From a3e5fa777d37e6d95f84b75befb6cd63019378a5 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Tue, 13 Jan 2026 12:32:47 +0200 Subject: [PATCH 01/21] add diskless managed rf feature design --- .../ts-unification/DISKLESS_MANAGED_RF.md | 571 ++++++++++++++++++ 1 file changed, 571 insertions(+) create mode 100644 docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md diff --git a/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md b/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md new file mode 100644 index 0000000000..b7c5d72136 --- /dev/null +++ b/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md @@ -0,0 +1,571 @@ +# Diskless-Managed Replication Factor + +## Table of Contents + +1. [Objectives](#objectives) +2. [Activation Model](#activation-model-binary-version) +3. [Placement Model](#placement-model) + - [Rack Cardinality](#rack-cardinality-and-missing-racks) + - [Rack Liveness](#rack-liveness-and-transience) +4. [Controller Behavior](#controller-behavior) + - [Placement Logic](#controller-placement-logic) + - [Topic Creation](#topic-creation-semantics) + - [Operator Override](#operator-override-and-reassignment) + - [Reconciliation](#reconciliation-algorithm) +5. [Broker Behavior](#broker-behavior) + - [Produce Path](#produce-path) + - [Consume Path](#consume-path) + - [ISR Semantics](#isr-semantics) +6. [Metadata Transformation](#metadata-transformation) + - [Current vs Target](#current-vs-target-behavior) + - [Transformer Changes](#transformer-implementation) +7. [Observability](#observability) +8. [Implementation Path](#implementation-path) +9. [Rejected Alternatives](#rejected-alternatives) +10. [Appendix: Migration Interactions](#appendix-migration-interactions) + +--- + +## Objectives + +Enable **rack-aware, dynamic, stable KRaft-managed replicas** for diskless topics: + +- **KRaft-managed replicas**: Diskless topics will have actual KRaft-managed replicas, not the current "faked" RF=1 metadata. These replicas are tracked by the controller, participate in leader election, and appear in standard Kafka tooling. The key difference from classic replicas: they don't replicate data (object storage handles durability), but they do provide partition assignments for RLM integration and operational consistency. +- **Rack-aware**: Enforce one replica per rack/AZ, ensuring geographic distribution +- **Dynamic**: Effective RF = rack cardinality; adjusts automatically as cluster topology changes +- **Stable**: Minimize placement churn; prioritize leader stability during rack additions/removals +- **Controller-managed**: Users don't specify RF; the controller computes optimal placement +- **Leader-agnostic produce**: Keep the diskless produce model where any broker can accept writes, while still having a leader for controller duties + +**Non-goal:** +Tiered Storage migration/cutover mechanics (sealing, offset boundary tracking). +Those are covered in [DESIGN.md](DESIGN.md). +This document covers **why** tiered storage integration requires managed RF, not **how** to implement the migration. + +--- + +## Activation Model (Binary Version) + +Managed RF is activated by **deploying a new binary version**. No feature flags, no topic attributes, no server configs. + +### Behavior Summary + +**Old binary:** +- Diskless topics use legacy "RF=1 / faked metadata" behavior +- `InklessTopicMetadataTransformer` calculates synthetic placement via hashing + +**New binary:** +- Diskless topics use KRaft-based placement (one replica per rack) +- Transformer filters KRaft placement by client AZ instead of calculating + +### How It Works + +1. Controller detects diskless topics via existing `diskless.enable=true` topic config +2. If diskless, controller applies managed RF placement (one replica per rack) +3. No new KRaft records needed; existing topic and partition records are sufficient +4. Transformer filters by client AZ instead of calculating synthetic placement + +### Rolling Upgrade + +During rolling restart: +- Old brokers return synthetic RF=1 metadata +- New brokers return KRaft-based filtered metadata +- Clients handle inconsistency via standard metadata refresh/retry +- After full upgrade, all brokers return consistent KRaft-based metadata + +### Existing Diskless Topics + +- Managed RF applies **immediately** to all diskless topics on upgrade +- No data movement required (diskless data lives in object storage) +- One-time metadata update as controller reconciles placement + +### Rationale + +Inkless is a new component with faster iteration cycles than upstream Kafka. +We control deployments and can coordinate version upgrades. +Complex feature gating adds overhead without proportional benefit for this internal enabler feature. +See [Rejected Alternative: Feature Flag Activation](#rejected-alternative-feature-flag-activation) for the more complex approach we considered. + +--- + +## Placement Model + +### Rack Cardinality and Missing Racks + +- **Effective RF** equals the number of racks (rack cardinality) +- Brokers without an explicit rack are treated as `rack=unknown` (a separate AZ for placement) + +### Rack Liveness and Transience + +**Problem:** The controller must distinguish between transient rack failures (network blip, rolling restart) and permanent rack removal to avoid: +- **Over-reacting:** Removing replicas during transient outages, causing unnecessary churn +- **Under-reacting:** Keeping replicas assigned to a permanently failed rack + +**Solution: Threshold-based Liveness** + +Config: `diskless.managed.rf.rack.unavailable.threshold.ms` +- Type: Long (milliseconds) +- Default: `300000` (5 minutes) +- Semantics: A rack is unavailable only after **all** brokers in that rack have been unreachable for longer than this threshold + +**Rack State Machine:** + +``` + broker unregisters / heartbeat timeout +HEALTHY ─────────────────────────────────────────────────► DEGRADED + ▲ │ + │ broker re-registers │ threshold exceeded + │ (any broker in rack) │ (all brokers in rack) + │ ▼ + └─────────────────────────────────────────────────────── UNAVAILABLE + broker re-registers (any) +``` + +**Controller Behavior by State:** + +**HEALTHY:** +- Normal operation +- One replica assigned per rack +- Eligible brokers available + +**DEGRADED:** +- Existing replicas retained +- No new replicas added to this rack +- `missingRackCapacity` exception surfaced +- Controller waits for recovery + +**UNAVAILABLE:** +- Replicas removed from this rack +- Effective RF decreases +- Reconciliation triggered + +**Rationale:** +- 5-minute default aligns with Kubernetes pod restart times and AWS AZ failover windows +- DEGRADED state prevents thrashing during rolling restarts +- Operators can tune threshold based on SLO requirements + +--- + +## Controller Behavior + +### Controller Placement Logic + +**Components:** `ReplicationControlManager`, `PartitionChangeBuilder`, `PartitionRegistration` + +**State tracked:** +- Topic: `effectiveRF` (derived from rack count) +- Partition: `placementState` (reconciling | steady | frozen) + +**Reconciliation strategy:** +- Input: rack map, broker capacities, current assignment, throttles +- Target: one eligible broker per rack +- Approach: reuse existing by rack, minimize movement, preserve leader + +### Topic Creation Semantics + +When creating diskless topics (`diskless.enable=true`): + +**Validation:** +- `replicationFactor` must be `-1` or `1` + - `-1`: Controller computes placement (recommended) + - `1`: Accepted for compatibility with automation tools that require an RF value + - Any value `> 1`: Rejected + - `0`: Rejected +- `replicaAssignments` must be empty (manual assignment not allowed) +- Controller computes actual placement as "one replica per rack" + +**Error behavior:** +``` +InvalidRequestException: replication factor is system-managed for diskless topics; +use replicationFactor=-1 or replicationFactor=1 +``` + +``` +InvalidRequestException: replica assignments cannot be specified for diskless topics; +placement is system-managed +``` + +**Rationale:** Many clients and automation tools always send an RF. Accepting `1` provides compatibility while still having the controller manage actual placement. + +### Operator Override and Reassignment + +Diskless topics use managed placement, but operators may still need to: +- Move a hot partition to a less loaded broker +- Drain a broker for maintenance +- Rebalance after adding brokers to a rack + +**Guiding principle:** Allow all standard Kafka reassignment operations. Observe drift from rack-aware placement via metrics/alerts rather than blocking. + +**Why not strict validation?** + +We considered rejecting any assignment that violates one-per-rack, but this adds complexity: +- Validation logic needed in multiple code paths (create, reassign, alter) +- Edge cases when rack topology changes mid-operation +- Divergent behavior from standard Kafka (harder to reason about) +- Blocks legitimate operator actions (intentional temporary placement) + +**Chosen approach: Observable Drift** + +All standard Kafka assignment operations work unchanged: +- `kafka-reassign-partitions.sh` works normally +- Manual replica assignment accepted +- Leader election works as expected + +When placement diverges from one-per-rack target: +- Partition flagged as having "placement drift" +- Metric: `kafka.controller.diskless.placement_drift{topic,partition}` = 1 +- Alert recommendation: "Partition X has placement drift from rack-aware target" + +**Reconciliation behavior:** + +The controller reconciliation loop can be configured to: +- **Auto-fix drift** (default): Gradually move partitions back to one-per-rack placement +- **Observe only**: Report drift but don't auto-correct (for operators who want manual control) + +Config: `diskless.managed.rf.auto.reconcile` (boolean, default: true) + +**Operator workflow:** + +1. Operator reassigns partition for operational reasons (e.g., drain broker) +2. Assignment accepted immediately +3. If placement violates one-per-rack: + - Metric shows drift + - Alert fires (if configured) + - If auto-reconcile enabled: controller will eventually fix it + - If auto-reconcile disabled: operator fixes manually or accepts drift + +This keeps Kafka API compatibility, reduces implementation complexity, and gives operators flexibility while maintaining observability. + +### Reconciliation Algorithm + +**Inputs:** rack map, broker capacities/eligibility, current assignment, `maxMovesPerSec` + +**Steps:** +1. Calculate required racks (all active racks) +2. Keep one replica per rack; evict extras from over-represented racks +3. For missing racks, add replica to least loaded eligible broker +4. Queue moves; enforce pacing and ISR health checks +5. If leader must move, ensure controlled election after catch-up + +**Exception: No Capacity in Rack** + +A rack has "no capacity" when all brokers in that rack are ineligible for replica assignment. This can happen when: +- All brokers in the rack are offline or unresponsive +- All brokers in the rack are in controlled shutdown +- All brokers in the rack have been fenced by the controller + +When this occurs: +- Controller marks the partition as temporarily under-replicated for that rack +- Surfaces `missingRackCapacity` exception in metrics and admin APIs +- Does **not** assign a replica to a different rack (would violate one-per-rack invariant) +- Retries assignment when a broker in that rack becomes available again + +This is distinct from "rack unavailable" (threshold exceeded, rack considered gone) where the replica is removed entirely. + +**Exception: Rack Removed** + +When a rack transitions to UNAVAILABLE (all brokers gone past threshold): +- Replicas in that rack are removed from partition assignments +- Effective RF decreases by 1 +- No replacement replica is added (no other rack should have >1 replica) + +**Guardrails:** +- Separate throttles for diskless vs classic reassignment +- Prefer "add then remove" to maintain availability +- Don't starve classic reassignment processing + +--- + +## Broker Behavior + +**Key point: Most broker behavior remains unchanged from today.** Diskless topics already work with the existing Inkless produce/consume handlers. This design changes **metadata and placement**, not the data path. + +### Produce Path + +**No changes to produce handling.** The existing Inkless produce path continues to work: +- Any broker can accept writes (leader-agnostic) +- Writes go to object storage via `AppendHandler` +- Batch metadata stored in Control Plane (PostgreSQL) + +The only difference is that KRaft now tracks replica assignments, but this doesn't affect how produces are processed. + +### Consume Path + +**No changes to consume handling.** The existing Inkless consume path continues to work: +- `FetchHandler` + `Reader` serve reads from object storage +- Data fetched via Control Plane batch lookups +- Object cache provides locality benefits + +What changes is **which broker the client talks to** (determined by transformer filtering), not how that broker serves the request. + +### ISR Semantics + +For diskless topics, **all replicas are always considered in-sync** because: +- Source of truth (batch metadata + WAL objects) is shared via object storage +- All replicas have immediate access to the same committed data +- No replication lag in the classic sense + +ISR membership is trivially "all assigned replicas" for diskless partitions. + +**This is unchanged from today** — diskless topics already have this property. The difference is that now there are multiple replicas in the ISR (one per rack) instead of a single faked replica. + +--- + +## Metadata Transformation + +### Current vs Target Behavior + +**Today (legacy):** +- `InklessTopicMetadataTransformer` intercepts responses +- Forces diskless partitions to look like RF=1 +- Calculates synthetic leader/ISR by hashing `(topicId, partition)` into alive brokers in client AZ + +**Target (managed RF):** +- KRaft placement is source of truth +- Replica sets reflect controller placement (one per rack) +- Transformer **filters** by client AZ instead of **calculating** + +### Transformer Implementation + +**Input changes:** +- For each diskless partition: KRaft replica set (brokerIds), KRaft leader id +- For each broker: `broker.rack` (AZ) via `Node.rack()` + +**Output (per request / per clientAZ):** +- Filtered `replicaNodes` / `isrNodes` for diskless partitions + +**Filtering logic:** +- If partition has replica in `clientAZ`: return only that local replica +- If no replica in `clientAZ`: fallback to cross-AZ view (full replica set or deterministic subset) + +**Detection:** +- New binary always uses KRaft-placement-based projection for diskless topics +- Check `diskless.enable=true` topic config to identify diskless topics + +--- + +## Observability + +### Metrics + +**Controller metrics:** +- `kafka.controller.diskless.effective_rf{topic}` - Current effective RF +- `kafka.controller.diskless.rack_coverage{topic,partition}` - Coverage percentage +- `kafka.controller.diskless.rack_state{rack}` - 0=HEALTHY, 1=DEGRADED, 2=UNAVAILABLE +- `kafka.controller.diskless.rack_degraded_since_ms{rack}` - Timestamp when degraded +- `kafka.controller.diskless.reassignment_backlog` - Pending reconciliation work +- `kafka.controller.diskless.placement_state{topic,partition}` - Current state +- `kafka.controller.diskless.placement_drift{topic,partition}` - 1 if placement differs from one-per-rack target +- `kafka.controller.diskless.exceptions{topic,partition}` - Active exceptions + +**Broker metrics:** +- `kafka.broker.diskless.coordinator_lag{topic,partition}` +- `kafka.broker.diskless.cache_build_lag{topic,partition}` +- `kafka.broker.diskless.preferred_broker_hit_ratio{topic}` + +### Admin Surfaces (Read-Only) + +**DescribeTopic extensions:** +- `IsDiskless: boolean` +- `effectiveRF: int` (current rack cardinality) + +**DescribePartition / Placement view:** +- `targetRacks: [rackId]` +- `assignedRacks: [rackId]` +- `rackCoverage: percent` +- `placementState: reconciling|steady|frozen` +- `exceptions: missingRackCapacity|none` + +**Client metadata (Metadata v14):** +- `IsDiskless=true` for diskless topics + +### Alert Recommendations + +- Rack in DEGRADED state for >50% of threshold +- `rackCoverage` below 100% for extended period +- `reassignment_backlog` growing continuously +- `placement_drift` = 1 for any partition (operator override detected) + +--- + +## Implementation Path + +### Phase 1: Controller Placement + +1. Modify `ReplicationControlManager` to detect `diskless.enable=true` topics +2. Compute rack-cardinality placement (one replica per rack) +3. Use existing partition records; no new KRaft record types +4. Implement rack liveness tracking with threshold-based state machine + +### Phase 2: Transformer Changes + +1. Update `InklessTopicMetadataTransformer` to filter by KRaft placement +2. Implement AZ filtering: local-AZ replica if present, else cross-AZ fallback +3. Remove synthetic placement calculation + +### Phase 3: Steady-State Reconciliation + +1. Implement reconciliation loop for rack topology changes +2. Add throttling and pacing controls +3. Add metrics and observability + +**Key invariant:** Only change client-visible metadata when KRaft placement is already correct. + +### Code References + +**Controller:** +- `metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java` +- `metadata/src/main/java/org/apache/kafka/controller/PartitionChangeBuilder.java` +- `metadata/src/main/java/org/apache/kafka/metadata/PartitionRegistration.java` + +**Inkless:** +- `storage/inkless/src/main/java/io/aiven/inkless/metadata/InklessTopicMetadataTransformer.java` +- `storage/inkless/src/main/java/io/aiven/inkless/consume/FetchHandler.java` +- `storage/inkless/src/main/java/io/aiven/inkless/control_plane/MetadataView.java` + +--- + +## Rejected Alternatives + +### Why RF > 1 Is Required + +All RF=1 alternatives fail because **RLM requires KRaft-managed partition leadership**: + +``` +Tiered storage integration (for PG scalability) + │ + ▼ +RLM expiration tasks (to clean up tiered segments) + │ + ▼ +onLeadershipChange() must fire (RLM's entry point) + │ + ▼ +KRaft-managed partition leadership (not faked/virtual) + │ + ▼ +RF > 1 with KRaft-managed replica assignments +``` + +The tiering pipeline's merge phase requires a partition leader to coordinate segment creation and RLM registration. There is no path to production-grade diskless topics without embracing KRaft-based replica placement. + +See [DESIGN.md](DESIGN.md) Stream 7 for tiering pipeline details. + +### Rejected Alternative: Feature Flag Activation + +**Concept:** Follow upstream Kafka patterns with `DisklessVersion` feature enum, server config, and KRaft records. + +**What it would have involved:** +- Policy flag (`disklessManagedRF`) as new KRaft record +- Feature enum (`DisklessVersion`) following `EligibleLeaderReplicasVersion` pattern +- Server config (`diskless.managed.rf.enable`) requiring cluster-wide coordination +- Manual activation via `kafka-features.sh --upgrade --feature diskless.version=1` + +**Why we dropped it:** + +- New KRaft record type → Not needed; `diskless.enable` config is sufficient +- Feature enum + registration → Not needed; binary version is the gate +- Server config + propagation → Adds operational step without benefit +- Complex rollback semantics → Binary rollback is simpler + +**When to reconsider:** +- If upstreaming to Apache Kafka (strict backward compatibility) +- If needing per-topic opt-in/opt-out +- If supporting long-lived mixed-version clusters + +### Rejected Alternative A: Virtual Leader for RLM Tasks + +**Concept:** Keep RF=1, designate deterministic "virtual leader" per partition for RLM tasks. + +**Why it fails:** +- Two sources of leadership creates confusion +- Virtual leader failover requires new coordination mechanism +- RLM assumes real `Partition` objects with `UnifiedLog` +- Tiering pipeline merge phase needs `ReplicaManager` context + +### Rejected Alternative B: Control Plane Manages Tiered Expiration + +**Concept:** Extend PostgreSQL to track tiered segment metadata and run expiration directly. + +**Why it fails:** +- Contradicts goal of reducing PG load +- Duplicates RLM retention logic +- Breaks RLMM integration and Kafka tooling +- Creates cross-system consistency problems + +### Rejected Alternative C: Direct Tiered Read via FetchHandler + +**Concept:** Extend `FetchHandler` to read tiered storage directly, bypassing RLM read path. + +**Why it fails:** +- Only solves read path, not expiration +- Must combine with Alternative A or B +- Duplicates RLM index handling + +### Rejected Alternative D: Treat Tiered Data as Read-Only Archival + +**Concept:** Freeze tiered portion, use S3 lifecycle policies for expiration. + +**Why it fails:** +- No programmatic retention (can't implement `retention.ms`/`retention.bytes`) +- Breaks topic deletion cleanup +- Doesn't address PG scalability for new diskless data + +--- + +## Appendix: Migration Interactions + +This section describes how the managed RF design interacts with topic migration from classic tiered storage to diskless mode. For full migration mechanics, see [DESIGN.md](DESIGN.md). + +### ISR Semantics During Migration + +When migrating a topic from classic tiered storage to diskless mode, ISR semantics transition through phases: + +**Phase 1: Pre-migration (Classic Tiered)** +- Standard Kafka ISR semantics apply +- Replicas must fetch from leader and stay within `replica.lag.time.max.ms` +- Leader manages replication to followers + +**Phase 2: During Migration (Sealing)** +- Active segment is rolled and copied to tiered storage +- ISR is maintained normally +- No new writes accepted until migration completes +- All replicas must be in-sync before proceeding + +**Phase 3: Post-migration (Hybrid/Diskless)** +- New writes go to diskless storage (object storage) +- ISR for diskless portion is "all replicas" (trivially in-sync) +- Reads from tiered portion use existing RLM machinery +- Topic uses diskless ISR semantics going forward + +### Replica Set Changes During Migration + +Migration may trigger replica set changes: + +**Before migration:** RF=3 with classic placement (leader + 2 followers) + +**After migration:** RF=rack_count with managed placement (one per rack) + +If rack_count differs from original RF: +- Controller reconciles to target placement +- Uses standard add-then-remove approach +- Paced to avoid disruption + +### Interaction with Tiering Pipeline + +After migration to diskless: +1. New writes accumulate in diskless storage +2. Aged batches become eligible for tiering (per `local.retention.ms`) +3. Tiering pipeline converts batches to tiered segments +4. Partition leader coordinates segment creation via RLM +5. Batch metadata deleted from PostgreSQL + +The managed RF design ensures a KRaft-managed partition leader exists for step 4. + +--- + +## Open Items + +- Where to surface `effectiveRF` in admin tooling (prefer existing DescribeTopic extensions) +- Detailed timing/pacing parameters for reconciliation throttling +- Interaction with `kafka-reassign-partitions.sh` - should it be blocked for managed-RF topics? From 522c7f81f6afe0104031604ae1ae5ec1c30d1a15 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Tue, 13 Jan 2026 12:59:17 +0200 Subject: [PATCH 02/21] fixup! add diskless managed rf feature design --- .../ts-unification/DISKLESS_MANAGED_RF.md | 509 ++++++++++-------- 1 file changed, 275 insertions(+), 234 deletions(-) diff --git a/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md b/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md index b7c5d72136..3febe67d67 100644 --- a/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md +++ b/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md @@ -5,24 +5,17 @@ 1. [Objectives](#objectives) 2. [Activation Model](#activation-model-binary-version) 3. [Placement Model](#placement-model) - - [Rack Cardinality](#rack-cardinality-and-missing-racks) - - [Rack Liveness](#rack-liveness-and-transience) 4. [Controller Behavior](#controller-behavior) - - [Placement Logic](#controller-placement-logic) - - [Topic Creation](#topic-creation-semantics) - - [Operator Override](#operator-override-and-reassignment) - - [Reconciliation](#reconciliation-algorithm) + - [Topic Creation](#topic-creation) + - [Add Partitions](#add-partitions) + - [Standard Operations](#standard-operations-after-creation) 5. [Broker Behavior](#broker-behavior) - - [Produce Path](#produce-path) - - [Consume Path](#consume-path) - - [ISR Semantics](#isr-semantics) + - [No Replica Fetcher for Diskless Topics](#no-replica-fetcher-for-diskless-topics) 6. [Metadata Transformation](#metadata-transformation) - - [Current vs Target](#current-vs-target-behavior) - - [Transformer Changes](#transformer-implementation) 7. [Observability](#observability) 8. [Implementation Path](#implementation-path) 9. [Rejected Alternatives](#rejected-alternatives) -10. [Appendix: Migration Interactions](#appendix-migration-interactions) +10. [Appendix: Topic Migration Interactions](#appendix-topic-migration-interactions) --- @@ -30,17 +23,15 @@ Enable **rack-aware, dynamic, stable KRaft-managed replicas** for diskless topics: -- **KRaft-managed replicas**: Diskless topics will have actual KRaft-managed replicas, not the current "faked" RF=1 metadata. These replicas are tracked by the controller, participate in leader election, and appear in standard Kafka tooling. The key difference from classic replicas: they don't replicate data (object storage handles durability), but they do provide partition assignments for RLM integration and operational consistency. -- **Rack-aware**: Enforce one replica per rack/AZ, ensuring geographic distribution -- **Dynamic**: Effective RF = rack cardinality; adjusts automatically as cluster topology changes -- **Stable**: Minimize placement churn; prioritize leader stability during rack additions/removals -- **Controller-managed**: Users don't specify RF; the controller computes optimal placement +- **KRaft-managed replicas**: Diskless topics will have actual KRaft-managed replicas, not the current "faked" RF=1 metadata. These replicas are tracked by the controller, participate in leader election, and appear in standard Kafka tooling. The key difference from classic replicas: **they don't replicate data** (object storage handles durability), but they do provide partition assignments for RLM integration and operational consistency. See [No Replica Fetcher for Diskless Topics](#no-replica-fetcher-for-diskless-topics) for implementation details. +- **Rack-aware at creation**: Topics are created with one replica per rack (RF = rack count) +- **Standard operations after creation**: Once created, diskless topics use standard Kafka replica management — no custom reconciliation or drift handling +- **Controller-managed RF**: Users don't specify RF at creation; the controller computes it from rack topology - **Leader-agnostic produce**: Keep the diskless produce model where any broker can accept writes, while still having a leader for controller duties **Non-goal:** -Tiered Storage migration/cutover mechanics (sealing, offset boundary tracking). +Topic migration mechanics (Tiered Classic → Diskless), including sealing and offset boundary tracking. Those are covered in [DESIGN.md](DESIGN.md). -This document covers **why** tiered storage integration requires managed RF, not **how** to implement the migration. --- @@ -55,15 +46,15 @@ Managed RF is activated by **deploying a new binary version**. No feature flags, - `InklessTopicMetadataTransformer` calculates synthetic placement via hashing **New binary:** -- Diskless topics use KRaft-based placement (one replica per rack) +- Diskless topics use KRaft-managed placement (one replica per rack at creation) - Transformer filters KRaft placement by client AZ instead of calculating ### How It Works 1. Controller detects diskless topics via existing `diskless.enable=true` topic config -2. If diskless, controller applies managed RF placement (one replica per rack) +2. At topic creation, controller computes RF = rack count and places one replica per rack 3. No new KRaft records needed; existing topic and partition records are sufficient -4. Transformer filters by client AZ instead of calculating synthetic placement +4. After creation, standard Kafka replica management applies ### Rolling Upgrade @@ -77,7 +68,7 @@ During rolling restart: - Managed RF applies **immediately** to all diskless topics on upgrade - No data movement required (diskless data lives in object storage) -- One-time metadata update as controller reconciles placement +- One-time metadata update as controller sets proper replica assignments ### Rationale @@ -90,91 +81,55 @@ See [Rejected Alternative: Feature Flag Activation](#rejected-alternative-featur ## Placement Model -### Rack Cardinality and Missing Racks +### Rack-Aware Placement at Creation -- **Effective RF** equals the number of racks (rack cardinality) -- Brokers without an explicit rack are treated as `rack=unknown` (a separate AZ for placement) +When a diskless topic is created: +- Controller determines current rack count from registered brokers +- RF is set to rack count (e.g., 3 racks → RF=3) +- One broker is selected per rack for each partition +- Broker selection within a rack uses load balancing (least loaded broker) -### Rack Liveness and Transience +### Brokers Without Rack -**Problem:** The controller must distinguish between transient rack failures (network blip, rolling restart) and permanent rack removal to avoid: -- **Over-reacting:** Removing replicas during transient outages, causing unnecessary churn -- **Under-reacting:** Keeping replicas assigned to a permanently failed rack +Brokers without an explicit `broker.rack` config are treated as belonging to `rack=unknown`. +- They are considered a separate "rack" for placement purposes +- If all brokers lack rack config, RF=1 (single "unknown" rack) -**Solution: Threshold-based Liveness** +### Placement Is Static After Creation -Config: `diskless.managed.rf.rack.unavailable.threshold.ms` -- Type: Long (milliseconds) -- Default: `300000` (5 minutes) -- Semantics: A rack is unavailable only after **all** brokers in that rack have been unreachable for longer than this threshold +Once a topic is created: +- RF does not automatically change if racks are added/removed +- Replica assignments don't automatically adjust +- Standard Kafka replica management applies -**Rack State Machine:** - -``` - broker unregisters / heartbeat timeout -HEALTHY ─────────────────────────────────────────────────► DEGRADED - ▲ │ - │ broker re-registers │ threshold exceeded - │ (any broker in rack) │ (all brokers in rack) - │ ▼ - └─────────────────────────────────────────────────────── UNAVAILABLE - broker re-registers (any) -``` - -**Controller Behavior by State:** - -**HEALTHY:** -- Normal operation -- One replica assigned per rack -- Eligible brokers available - -**DEGRADED:** -- Existing replicas retained -- No new replicas added to this rack -- `missingRackCapacity` exception surfaced -- Controller waits for recovery - -**UNAVAILABLE:** -- Replicas removed from this rack -- Effective RF decreases -- Reconciliation triggered - -**Rationale:** -- 5-minute default aligns with Kubernetes pod restart times and AWS AZ failover windows -- DEGRADED state prevents thrashing during rolling restarts -- Operators can tune threshold based on SLO requirements +This keeps the design simple and predictable. --- ## Controller Behavior -### Controller Placement Logic - -**Components:** `ReplicationControlManager`, `PartitionChangeBuilder`, `PartitionRegistration` +### Topic Creation -**State tracked:** -- Topic: `effectiveRF` (derived from rack count) -- Partition: `placementState` (reconciling | steady | frozen) - -**Reconciliation strategy:** -- Input: rack map, broker capacities, current assignment, throttles -- Target: one eligible broker per rack -- Approach: reuse existing by rack, minimize movement, preserve leader +When creating diskless topics (`diskless.enable=true`): -### Topic Creation Semantics +**RF Computation:** +- Controller counts distinct racks from registered brokers +- RF = rack count (e.g., 3 racks → RF=3) -When creating diskless topics (`diskless.enable=true`): +**Placement:** +- One replica assigned per rack +- Within each rack, select least loaded broker +- Leader is typically the first replica (standard Kafka behavior) **Validation:** -- `replicationFactor` must be `-1` or `1` +- `replicationFactor` in CreateTopics request must be `-1` or `1` - `-1`: Controller computes placement (recommended) - - `1`: Accepted for compatibility with automation tools that require an RF value - - Any value `> 1`: Rejected + - `1`: Accepted for compatibility with automation tools + - Any value `> 1`: Rejected (RF is system-managed) - `0`: Rejected -- `replicaAssignments` must be empty (manual assignment not allowed) -- Controller computes actual placement as "one replica per rack" +- `replicaAssignments` must be empty (manual assignment not allowed at creation) -**Error behavior:** +**Error messages:** ``` InvalidRequestException: replication factor is system-managed for diskless topics; use replicationFactor=-1 or replicationFactor=1 @@ -182,103 +137,97 @@ use replicationFactor=-1 or replicationFactor=1 ``` InvalidRequestException: replica assignments cannot be specified for diskless topics; -placement is system-managed +placement is system-managed at creation time ``` -**Rationale:** Many clients and automation tools always send an RF. Accepting `1` provides compatibility while still having the controller manage actual placement. - -### Operator Override and Reassignment - -Diskless topics use managed placement, but operators may still need to: -- Move a hot partition to a less loaded broker -- Drain a broker for maintenance -- Rebalance after adding brokers to a rack +### Add Partitions -**Guiding principle:** Allow all standard Kafka reassignment operations. Observe drift from rack-aware placement via metrics/alerts rather than blocking. +When adding partitions to an existing diskless topic: -**Why not strict validation?** - -We considered rejecting any assignment that violates one-per-rack, but this adds complexity: -- Validation logic needed in multiple code paths (create, reassign, alter) -- Edge cases when rack topology changes mid-operation -- Divergent behavior from standard Kafka (harder to reason about) -- Blocks legitimate operator actions (intentional temporary placement) +```bash +kafka-topics.sh --alter --topic foo --partitions 10 +``` -**Chosen approach: Observable Drift** +**Behavior:** +- New partitions are placed using the same one-per-rack logic as creation +- RF for new partitions = current rack count (may differ from original if racks changed) +- Existing partitions are not affected -All standard Kafka assignment operations work unchanged: -- `kafka-reassign-partitions.sh` works normally -- Manual replica assignment accepted -- Leader election works as expected +This ensures consistency: all partitions of a diskless topic use rack-aware placement. -When placement diverges from one-per-rack target: -- Partition flagged as having "placement drift" -- Metric: `kafka.controller.diskless.placement_drift{topic,partition}` = 1 -- Alert recommendation: "Partition X has placement drift from rack-aware target" +### Standard Operations (After Creation) -**Reconciliation behavior:** +After topic creation, diskless topics behave like classic Kafka topics for all operations: -The controller reconciliation loop can be configured to: -- **Auto-fix drift** (default): Gradually move partitions back to one-per-rack placement -- **Observe only**: Report drift but don't auto-correct (for operators who want manual control) +**Leader Election:** +- Standard Kafka leader election from ISR +- For diskless topics, all replicas are always in ISR (data is in object storage) +- `kafka-leader-election.sh` works normally -Config: `diskless.managed.rf.auto.reconcile` (boolean, default: true) +**Broker Failure:** +- Replicas on failed broker become offline +- Partition becomes under-replicated (reported in metrics) +- Leader fails over if leader was on failed broker +- When broker returns, replica immediately rejoins ISR (no catch-up needed) -**Operator workflow:** +**Broker Shutdown (Controlled):** +- Standard controlled shutdown +- Leaders moved away before shutdown +- No special handling for diskless -1. Operator reassigns partition for operational reasons (e.g., drain broker) -2. Assignment accepted immediately -3. If placement violates one-per-rack: - - Metric shows drift - - Alert fires (if configured) - - If auto-reconcile enabled: controller will eventually fix it - - If auto-reconcile disabled: operator fixes manually or accepts drift +**Reassignment:** +- `kafka-reassign-partitions.sh` works normally +- Operator can reassign replicas to any brokers +- No data movement (just metadata update for diskless) +- ⚠️ Operator can break rack-awareness (see below) -This keeps Kafka API compatibility, reduces implementation complexity, and gives operators flexibility while maintaining observability. +**Rolling Upgrades:** +- Standard Kafka rolling restart behavior +- No special handling for diskless topics -### Reconciliation Algorithm +### Operator Reassignment and Rack-Awareness -**Inputs:** rack map, broker capacities/eligibility, current assignment, `maxMovesPerSec` +Operators can use standard reassignment tools on diskless topics: -**Steps:** -1. Calculate required racks (all active racks) -2. Keep one replica per rack; evict extras from over-represented racks -3. For missing racks, add replica to least loaded eligible broker -4. Queue moves; enforce pacing and ISR health checks -5. If leader must move, ensure controlled election after catch-up +```bash +kafka-reassign-partitions.sh --bootstrap-server localhost:9092 \ + --reassignment-json-file reassignment.json \ + --execute +``` -**Exception: No Capacity in Rack** +**Important:** The controller does **not** validate rack-awareness for reassignments. -A rack has "no capacity" when all brokers in that rack are ineligible for replica assignment. This can happen when: -- All brokers in the rack are offline or unresponsive -- All brokers in the rack are in controlled shutdown -- All brokers in the rack have been fenced by the controller +If an operator reassigns replicas such that rack-awareness is broken (e.g., 2 replicas in same rack): +- Assignment is **accepted** (standard Kafka behavior) +- Warning logged: `WARN Diskless topic 'foo' partition 0 has non-rack-aware placement` +- Metric emitted: `kafka.controller.diskless.rack_aware{topic="foo",partition="0"} = 0` +- Operator is responsible for fixing if desired -When this occurs: -- Controller marks the partition as temporarily under-replicated for that rack -- Surfaces `missingRackCapacity` exception in metrics and admin APIs -- Does **not** assign a replica to a different rack (would violate one-per-rack invariant) -- Retries assignment when a broker in that rack becomes available again +**Rationale:** +- Keeps implementation simple (no validation logic in reassignment path) +- Same behavior as classic topics (Kafka doesn't enforce rack-awareness on reassignment) +- Operator has full control for edge cases (intentional non-rack-aware placement) -This is distinct from "rack unavailable" (threshold exceeded, rack considered gone) where the replica is removed entirely. +### Topology Changes (Racks Added/Removed) -**Exception: Rack Removed** +**New rack added:** +- Existing topics are NOT automatically updated +- RF stays at original value +- Operator can reassign to include new rack if desired +- New topics will include the new rack -When a rack transitions to UNAVAILABLE (all brokers gone past threshold): -- Replicas in that rack are removed from partition assignments -- Effective RF decreases by 1 -- No replacement replica is added (no other rack should have >1 replica) +**Rack removed (all brokers gone):** +- Partitions with replicas on that rack become under-replicated +- Standard Kafka under-replication handling +- Operator reassigns to remove references to gone rack -**Guardrails:** -- Separate throttles for diskless vs classic reassignment -- Prefer "add then remove" to maintain availability -- Don't starve classic reassignment processing +**Note:** These are edge cases that rarely happen in practice. The design prioritizes simplicity over automatic handling of rare topology changes. --- ## Broker Behavior -**Key point: Most broker behavior remains unchanged from today.** Diskless topics already work with the existing Inkless produce/consume handlers. This design changes **metadata and placement**, not the data path. +**Key point: Broker behavior is unchanged from today.** Diskless topics already work with existing Inkless produce/consume handlers. This design changes **metadata and placement**, not the data path. ### Produce Path @@ -309,6 +258,77 @@ ISR membership is trivially "all assigned replicas" for diskless partitions. **This is unchanged from today** — diskless topics already have this property. The difference is that now there are multiple replicas in the ISR (one per rack) instead of a single faked replica. +### No Replica Fetcher for Diskless Topics + +**Diskless topics do not start replica fetcher threads.** This is a critical design invariant that must be preserved. + +#### Current Implementation + +In `ReplicaManager.scala`, diskless topics are explicitly skipped when applying metadata changes: + +**`applyLocalLeadersDelta`** (line ~3118): +```scala +localLeaders.foreachEntry { (tp, info) => + if (!_inklessMetadataView.isDisklessTopic(tp.topic())) + getOrCreatePartition(tp, delta, info.topicId).foreach { ... } +} +``` + +**`applyLocalFollowersDelta`** (line ~3154): +```scala +localFollowers.foreachEntry { (tp, info) => + if (!_inklessMetadataView.isDisklessTopic(tp.topic())) + getOrCreatePartition(tp, delta, info.topicId).foreach { ... } +} +``` + +This means: +- No local `Partition` objects created for diskless topics +- No local log directories created +- No `ReplicaFetcherThread` started for followers +- No replication traffic between brokers for diskless data + +#### Why This Must Be Preserved + +1. **No data to replicate**: Diskless data lives in object storage, not local logs +2. **Performance**: Fetcher threads consume CPU and network; unnecessary for diskless +3. **Consistency**: All brokers read from the same object storage; no catch-up needed +4. **Simplicity**: Fewer moving parts means fewer failure modes + +#### Regression Prevention + +**What could cause a regression:** +- Code changes that don't check `isDisklessTopic()` before creating partitions +- New code paths that bypass the existing guards +- Feature flags that accidentally enable local storage for diskless topics + +**How to prevent regressions:** + +1. **Unit tests**: Ensure tests verify no fetcher is started for diskless topics + ```scala + // Example test assertion + assertEquals(None, replicaManager.replicaFetcherManager.getFetcher(disklessTopicPartition)) + ``` + +2. **Integration tests**: Verify no replication traffic for diskless topics + - Create diskless topic with RF=3 + - Produce messages + - Verify no inter-broker fetch requests for that topic + +3. **Code review checklist**: Any changes to `ReplicaManager.applyLocalLeadersDelta` or `applyLocalFollowersDelta` must preserve the `isDisklessTopic()` guard + +4. **Metrics**: Monitor `kafka.server:type=ReplicaFetcherManager,name=*` to detect unexpected fetcher activity + +#### Managed RF Interaction + +With managed RF, diskless topics will have multiple replicas (one per rack). However: +- These are **metadata-only replicas** in KRaft +- They exist for leader election and RLM coordination +- They do **not** trigger replica fetcher threads +- The `isDisklessTopic()` guards in `ReplicaManager` continue to apply + +The managed RF design **does not change** this behavior — it only changes how replica assignments are computed and stored in KRaft. + --- ## Metadata Transformation @@ -349,85 +369,87 @@ ISR membership is trivially "all assigned replicas" for diskless partitions. ### Metrics **Controller metrics:** -- `kafka.controller.diskless.effective_rf{topic}` - Current effective RF -- `kafka.controller.diskless.rack_coverage{topic,partition}` - Coverage percentage -- `kafka.controller.diskless.rack_state{rack}` - 0=HEALTHY, 1=DEGRADED, 2=UNAVAILABLE -- `kafka.controller.diskless.rack_degraded_since_ms{rack}` - Timestamp when degraded -- `kafka.controller.diskless.reassignment_backlog` - Pending reconciliation work -- `kafka.controller.diskless.placement_state{topic,partition}` - Current state -- `kafka.controller.diskless.placement_drift{topic,partition}` - 1 if placement differs from one-per-rack target -- `kafka.controller.diskless.exceptions{topic,partition}` - Active exceptions - -**Broker metrics:** -- `kafka.broker.diskless.coordinator_lag{topic,partition}` -- `kafka.broker.diskless.cache_build_lag{topic,partition}` -- `kafka.broker.diskless.preferred_broker_hit_ratio{topic}` - -### Admin Surfaces (Read-Only) - -**DescribeTopic extensions:** -- `IsDiskless: boolean` -- `effectiveRF: int` (current rack cardinality) - -**DescribePartition / Placement view:** -- `targetRacks: [rackId]` -- `assignedRacks: [rackId]` -- `rackCoverage: percent` -- `placementState: reconciling|steady|frozen` -- `exceptions: missingRackCapacity|none` - -**Client metadata (Metadata v14):** -- `IsDiskless=true` for diskless topics - -### Alert Recommendations - -- Rack in DEGRADED state for >50% of threshold -- `rackCoverage` below 100% for extended period -- `reassignment_backlog` growing continuously -- `placement_drift` = 1 for any partition (operator override detected) +- `kafka.controller.diskless.effective_rf{topic}` - RF assigned at creation +- `kafka.controller.diskless.rack_aware{topic,partition}` - 1 if one-per-rack, 0 if not +- `kafka.controller.diskless.rack_aware_total{topic}` - Count of rack-aware partitions +- `kafka.controller.diskless.non_rack_aware_total{topic}` - Count of non-rack-aware partitions + +**Standard Kafka metrics (already exist):** +- `kafka.server:type=ReplicaManager,name=UnderReplicatedPartitions` +- `kafka.controller:type=KafkaController,name=OfflinePartitionsCount` + +### Admin Surfaces + +**DescribeTopic shows actual replica assignments:** +```bash +kafka-topics.sh --bootstrap-server localhost:9092 --describe --topic foo + +# Output: +# Topic: foo TopicId: abc123 PartitionCount: 3 ReplicationFactor: 3 +# Partition: 0 Leader: 1 Replicas: 1,3,5 Isr: 1,3,5 +# Partition: 1 Leader: 3 Replicas: 3,5,1 Isr: 3,5,1 +# Partition: 2 Leader: 5 Replicas: 5,1,3 Isr: 5,1,3 +``` + +### Logs + +**Warning when rack-awareness is broken:** +``` +WARN [Controller] Diskless topic 'foo' partition 0 has non-rack-aware placement: + replicas [1,2,5] map to racks [rack-A,rack-A,rack-C]. Expected one replica per rack. +``` + +This warning is logged: +- When reassignment results in non-rack-aware placement +- Periodically if non-rack-aware placement persists (configurable interval) --- ## Implementation Path -### Phase 1: Controller Placement +### Phase 1: Topic Creation with Rack-Aware Placement 1. Modify `ReplicationControlManager` to detect `diskless.enable=true` topics -2. Compute rack-cardinality placement (one replica per rack) -3. Use existing partition records; no new KRaft record types -4. Implement rack liveness tracking with threshold-based state machine +2. Compute RF = rack count at creation time +3. Implement one-per-rack broker selection +4. Reject `replicationFactor > 1` and non-empty `replicaAssignments` + +**Estimate:** 2 weeks ### Phase 2: Transformer Changes -1. Update `InklessTopicMetadataTransformer` to filter by KRaft placement -2. Implement AZ filtering: local-AZ replica if present, else cross-AZ fallback -3. Remove synthetic placement calculation +1. Update `InklessTopicMetadataTransformer` to read KRaft placement +2. Implement AZ filtering logic +3. Remove synthetic hashing calculation + +**Estimate:** 2 weeks + +### Phase 3: Add Partitions Support -### Phase 3: Steady-State Reconciliation +1. Apply same one-per-rack logic when adding partitions +2. Handle case where rack count changed since topic creation -1. Implement reconciliation loop for rack topology changes -2. Add throttling and pacing controls -3. Add metrics and observability +**Estimate:** 1 week -**Key invariant:** Only change client-visible metadata when KRaft placement is already correct. +### Phase 4: Observability -### Code References +1. Add `rack_aware` metric +2. Add warning logs for non-rack-aware placement +3. Documentation -**Controller:** -- `metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java` -- `metadata/src/main/java/org/apache/kafka/controller/PartitionChangeBuilder.java` -- `metadata/src/main/java/org/apache/kafka/metadata/PartitionRegistration.java` +**Estimate:** 1 week -**Inkless:** -- `storage/inkless/src/main/java/io/aiven/inkless/metadata/InklessTopicMetadataTransformer.java` -- `storage/inkless/src/main/java/io/aiven/inkless/consume/FetchHandler.java` -- `storage/inkless/src/main/java/io/aiven/inkless/control_plane/MetadataView.java` +### Total Estimate + +**6 weeks with 1 engineer, or 3-4 weeks with 2 engineers** + +This is significantly simpler than the original design which included reconciliation loops and drift handling. --- ## Rejected Alternatives -### Why RF > 1 Is Required +### Why KRaft-Managed Replicas Are Required All RF=1 alternatives fail because **RLM requires KRaft-managed partition leadership**: @@ -462,17 +484,37 @@ See [DESIGN.md](DESIGN.md) Stream 7 for tiering pipeline details. - Manual activation via `kafka-features.sh --upgrade --feature diskless.version=1` **Why we dropped it:** - -- New KRaft record type → Not needed; `diskless.enable` config is sufficient -- Feature enum + registration → Not needed; binary version is the gate -- Server config + propagation → Adds operational step without benefit -- Complex rollback semantics → Binary rollback is simpler +- Not needed; `diskless.enable` config is sufficient +- Binary version is the gate +- Adds operational step without benefit +- Binary rollback is simpler **When to reconsider:** - If upstreaming to Apache Kafka (strict backward compatibility) - If needing per-topic opt-in/opt-out - If supporting long-lived mixed-version clusters +### Rejected Alternative: Dynamic Reconciliation + +**Concept:** Controller continuously reconciles replica placement to maintain one-per-rack as topology changes. + +**What it would have involved:** +- Reconciliation loop monitoring rack changes +- Automatic RF adjustment when racks added/removed +- Drift detection and auto-fix +- Rack liveness state machine (HEALTHY → DEGRADED → UNAVAILABLE) + +**Why we dropped it:** +- Significant complexity in controller +- Divergent behavior from standard Kafka operations +- Topology changes are rare in practice +- Operator can handle edge cases manually +- Standard Kafka reassignment tools work fine + +**When to reconsider:** +- If customers frequently add/remove racks +- If manual reassignment becomes burdensome + ### Rejected Alternative A: Virtual Leader for RLM Tasks **Concept:** Keep RF=1, designate deterministic "virtual leader" per partition for RLM tasks. @@ -513,34 +555,34 @@ See [DESIGN.md](DESIGN.md) Stream 7 for tiering pipeline details. --- -## Appendix: Migration Interactions +## Appendix: Topic Migration Interactions -This section describes how the managed RF design interacts with topic migration from classic tiered storage to diskless mode. For full migration mechanics, see [DESIGN.md](DESIGN.md). +This section describes how the managed RF design interacts with **topic migration (Tiered Classic → Diskless)**. For full migration mechanics, see [DESIGN.md](DESIGN.md). -### ISR Semantics During Migration +### ISR Semantics During Topic Migration -When migrating a topic from classic tiered storage to diskless mode, ISR semantics transition through phases: +When migrating a topic from Tiered Classic to Diskless, ISR semantics transition through phases: -**Phase 1: Pre-migration (Classic Tiered)** +**Phase 1: Pre-migration (Tiered Classic)** - Standard Kafka ISR semantics apply - Replicas must fetch from leader and stay within `replica.lag.time.max.ms` - Leader manages replication to followers -**Phase 2: During Migration (Sealing)** +**Phase 2: During Topic Migration (Sealing)** - Active segment is rolled and copied to tiered storage - ISR is maintained normally - No new writes accepted until migration completes - All replicas must be in-sync before proceeding -**Phase 3: Post-migration (Hybrid/Diskless)** +**Phase 3: Post-migration (Diskless)** - New writes go to diskless storage (object storage) - ISR for diskless portion is "all replicas" (trivially in-sync) - Reads from tiered portion use existing RLM machinery - Topic uses diskless ISR semantics going forward -### Replica Set Changes During Migration +### Replica Set Changes During Topic Migration -Migration may trigger replica set changes: +Topic migration (Tiered Classic → Diskless) may trigger replica set changes: **Before migration:** RF=3 with classic placement (leader + 2 followers) @@ -553,7 +595,7 @@ If rack_count differs from original RF: ### Interaction with Tiering Pipeline -After migration to diskless: +After topic migration to Diskless: 1. New writes accumulate in diskless storage 2. Aged batches become eligible for tiering (per `local.retention.ms`) 3. Tiering pipeline converts batches to tiered segments @@ -566,6 +608,5 @@ The managed RF design ensures a KRaft-managed partition leader exists for step 4 ## Open Items -- Where to surface `effectiveRF` in admin tooling (prefer existing DescribeTopic extensions) -- Detailed timing/pacing parameters for reconciliation throttling -- Interaction with `kafka-reassign-partitions.sh` - should it be blocked for managed-RF topics? +- Behavior when `broker.rack` config changes on a broker (rare edge case) +- Whether to add a tool to "re-rack-aware" a topic (reassign to restore one-per-rack) From 5e118305e567e3e8b5471fc76e4e6734d763940d Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Tue, 13 Jan 2026 15:12:29 +0200 Subject: [PATCH 03/21] fixup! add diskless managed rf feature design --- .../ts-unification/DISKLESS_MANAGED_RF.md | 453 +++++++++++++++--- 1 file changed, 388 insertions(+), 65 deletions(-) diff --git a/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md b/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md index 3febe67d67..fcff2ea26d 100644 --- a/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md +++ b/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md @@ -4,6 +4,7 @@ 1. [Objectives](#objectives) 2. [Activation Model](#activation-model-binary-version) + - [Existing Diskless Topics](#existing-diskless-topics) 3. [Placement Model](#placement-model) 4. [Controller Behavior](#controller-behavior) - [Topic Creation](#topic-creation) @@ -14,20 +15,24 @@ 6. [Metadata Transformation](#metadata-transformation) 7. [Observability](#observability) 8. [Implementation Path](#implementation-path) + - [Phase 0: Research and Validation](#phase-0-research-and-validation) + - [Testing Strategy](#testing-strategy) + - [Future Work: RLM Integration](#future-work-rlm-integration) 9. [Rejected Alternatives](#rejected-alternatives) 10. [Appendix: Topic Migration Interactions](#appendix-topic-migration-interactions) +11. [Open Items](#open-items) --- ## Objectives -Enable **rack-aware, dynamic, stable KRaft-managed replicas** for diskless topics: +Enable **rack-aware, stable KRaft-managed replicas** for diskless topics: - **KRaft-managed replicas**: Diskless topics will have actual KRaft-managed replicas, not the current "faked" RF=1 metadata. These replicas are tracked by the controller, participate in leader election, and appear in standard Kafka tooling. The key difference from classic replicas: **they don't replicate data** (object storage handles durability), but they do provide partition assignments for RLM integration and operational consistency. See [No Replica Fetcher for Diskless Topics](#no-replica-fetcher-for-diskless-topics) for implementation details. - **Rack-aware at creation**: Topics are created with one replica per rack (RF = rack count) - **Standard operations after creation**: Once created, diskless topics use standard Kafka replica management — no custom reconciliation or drift handling -- **Controller-managed RF**: Users don't specify RF at creation; the controller computes it from rack topology -- **Leader-agnostic produce**: Keep the diskless produce model where any broker can accept writes, while still having a leader for controller duties +- **Controller-managed RF**: Users don't specify RF at creation; the controller computes it from rack topology. Requests with `replicationFactor=-1` or `replicationFactor=1` are accepted for compatibility (both result in RF=rack_count). +- **Leader-agnostic produce and consume**: Keep the diskless model where any replica can accept writes and serve reads, while still having a KRaft leader for controller duties (e.g., future RLM coordination) **Non-goal:** Topic migration mechanics (Tiered Classic → Diskless), including sealing and offset boundary tracking. @@ -37,7 +42,7 @@ Those are covered in [DESIGN.md](DESIGN.md). ## Activation Model (Binary Version) -Managed RF is activated by **deploying a new binary version**. No feature flags, no topic attributes, no server configs. +Managed RF is activated by **deploying a new binary version**. No feature flags, no topic attributes, no server configs. See [Rejected Alternative: Feature Flag Activation](#rejected-alternative-feature-flag-activation) for why we chose this simpler approach. ### Behavior Summary @@ -56,26 +61,180 @@ Managed RF is activated by **deploying a new binary version**. No feature flags, 3. No new KRaft records needed; existing topic and partition records are sufficient 4. After creation, standard Kafka replica management applies -### Rolling Upgrade +### Mixed Binary Compatibility -During rolling restart: -- Old brokers return synthetic RF=1 metadata -- New brokers return KRaft-based filtered metadata -- Clients handle inconsistency via standard metadata refresh/retry -- After full upgrade, all brokers return consistent KRaft-based metadata +If a new controller updates partition metadata (e.g., RF=3 with replicas in multiple racks) while some brokers still run the old binary: + +- **Data plane is unaffected**: Produce and consume APIs are not modified. Old brokers continue serving diskless requests via existing `AppendHandler`/`FetchHandler`. +- **Metadata plane differs**: Old brokers use transformer hashing, new brokers filter KRaft metadata. Clients may see different leader/replicas depending on which broker they query. +- **No correctness issues**: Diskless data lives in object storage. Any broker can serve any partition regardless of what metadata says. + +This temporary inconsistency resolves once all brokers run the new binary. In our deployment model (new VMs), this period is brief or non-existent. + +### Cluster Migration + +In our deployment model, new binaries are deployed on **new VMs with new broker IDs** (not in-place rolling upgrade). This simplifies the transition: + +**Before migration:** +- Old cluster with old broker IDs +- Diskless topics have RF=1 pointing to old broker IDs +- Transformer uses hash-based leader selection + +**After migration:** +- New cluster with new broker IDs +- Controller detects orphaned replicas (old broker IDs not in cluster) +- Auto-migration expands RF to rack_count with new broker IDs +- Transformer filters KRaft metadata by client AZ + +**Client behavior:** +- Clients reconnect to new cluster via bootstrap servers +- Metadata requests return new broker IDs with rack-aware placement +- No inconsistency period (unlike rolling upgrade) ### Existing Diskless Topics -- Managed RF applies **immediately** to all diskless topics on upgrade -- No data movement required (diskless data lives in object storage) -- One-time metadata update as controller sets proper replica assignments +Existing diskless topics created before this feature have RF=1 in KRaft. These need to be expanded to RF=rack_count. + +**Migration approach: Automatic migration for orphaned replicas** + +Legacy diskless topics were created with RF=1 pointing to a broker that existed at creation time. In our deployment model, **new binaries are deployed on new VMs with new broker IDs**. This means legacy topics will have broker IDs that no longer exist in the cluster, allowing safe automatic detection. + +#### Deployment Model Assumption + +``` +Old cluster (legacy diskless topics created here): + Brokers: 1, 2, 3, 4, 5, 6 + +New cluster (after infrastructure migration): + Brokers: 101, 102, 103, 104, 105, 106 ← new VMs, new IDs + +Legacy topic in KRaft: + Topic: foo, Partition 0, Replicas=[3] ← broker 3 no longer exists +``` + +#### Detection Logic + +``` +IF diskless.enable=true + AND RF=1 + AND replica broker ID is NOT in active cluster +THEN auto-migrate to RF=rack_count +``` + +#### Why This Is Safe + +The detection has **no false positives** given the deployment model: + +| Scenario | RF | Broker ID | Auto-migrate? | +|----------|-----|-----------|---------------| +| Legacy diskless (old cluster) | 1 | Orphaned (old broker ID) | ✅ Yes | +| Already migrated diskless | rack_count | Valid | ❌ No (RF > 1) | +| Operator-managed diskless | 1 | Valid (current broker) | ❌ No (valid ID) | +| Classic topic | any | Valid | ❌ No (not diskless) | + +**Key insight:** The deployment model (new VMs = new broker IDs) ensures legacy topics have orphaned broker IDs. + +#### Other Deployment Models (Broker IDs Overlap) + +If the new cluster reuses broker IDs from the old cluster (e.g., in-place upgrade on same VMs), automatic migration won't trigger because the broker ID is still valid. + +**In this case:** +- Topics continue to work (RF=1 is functional, just not rack-aware) +- Operator uses standard replica placement tools to migrate manually +- Metric `kafka.controller.diskless.rf1_topics_total` tracks diskless topics with RF=1 +- Log warning: `WARN Diskless topic 'foo' has RF=1; consider migrating to rack-aware placement` + +**Manual migration steps:** + +1. Generate reassignment plan: +```bash +# List diskless topics with RF=1 +kafka-topics.sh --bootstrap-server localhost:9092 --describe \ + | grep -B1 "ReplicationFactor: 1" | grep "diskless.enable=true" + +# Generate reassignment JSON (one replica per rack) +cat > reassignment.json << 'EOF' +{ + "version": 1, + "partitions": [ + {"topic": "foo", "partition": 0, "replicas": [101, 103, 105]} + ] +} +EOF +``` + +2. Execute reassignment: +```bash +kafka-reassign-partitions.sh --bootstrap-server localhost:9092 \ + --reassignment-json-file reassignment.json \ + --execute +``` + +3. Verify: +```bash +kafka-reassign-partitions.sh --bootstrap-server localhost:9092 \ + --reassignment-json-file reassignment.json \ + --verify +``` + +**Note:** For diskless topics, reassignment is metadata-only (no data movement), so it completes instantly. + +#### Migration Process + +When controller detects an orphaned diskless topic: -### Rationale +1. Log: `INFO Detected orphaned diskless topic 'foo' (RF=1, broker 3 not in cluster)` +2. Compute target replica assignment (one broker per rack) +3. Issue `AlterPartitionReassignments` to set new replicas +4. Log: `INFO Migrated diskless topic 'foo' to RF=3, replicas=[1,3,5]` -Inkless is a new component with faster iteration cycles than upstream Kafka. -We control deployments and can coordinate version upgrades. -Complex feature gating adds overhead without proportional benefit for this internal enabler feature. -See [Rejected Alternative: Feature Flag Activation](#rejected-alternative-feature-flag-activation) for the more complex approach we considered. +**This is metadata-only** — no data movement required (data is in object storage). + +#### Example + +``` +Legacy topic (created on old cluster): + Topic: foo (diskless.enable=true) + Partition 0: Replicas=[3], Leader=3, ISR=[3] + ↑ + Broker 3 was valid on old cluster + +New cluster brokers: [101, 102, 103, 104, 105, 106] in racks [A, A, B, B, C, C] + ↑ + Broker 3 doesn't exist anymore + +After auto-migration: + Topic: foo (diskless.enable=true) + Partition 0: Replicas=[101,103,105], Leader=101, ISR=[101,103,105] + ↑ ↑ ↑ + One broker per rack (A, B, C) +``` + +#### When Migration Runs + +- **Trigger:** Controller startup, metadata change, or periodic scan +- **Pacing:** Batched to avoid overwhelming controller (configurable) +- **Idempotent:** Re-running on already-migrated topics is a no-op (RF > 1) + +#### Observability + +**Auto-migration:** +- Log: `INFO Detected orphaned diskless topic 'foo' (RF=1, broker 3 not in cluster)` +- Log: `INFO Migrated diskless topic 'foo' to RF=3` +- Metric: `kafka.controller.diskless.topics_migrated_total` (counter) +- Metric: `kafka.controller.diskless.orphaned_topics_total` (gauge) — pending migration + +**RF=1 with valid broker (needs manual migration):** +- Log: `WARN Diskless topic 'foo' has RF=1; consider migrating to rack-aware placement` +- Metric: `kafka.controller.diskless.rf1_topics_total` (gauge) — topics that need manual migration + +#### Rollback + +If rolled back to old binary: +- Expanded RF is preserved in KRaft metadata +- Old brokers ignore KRaft RF and use transformer hashing +- No harm — metadata is unused until re-upgraded +- Re-upgrade will see RF > 1 and skip migration (idempotent) --- @@ -152,9 +311,15 @@ kafka-topics.sh --alter --topic foo --partitions 10 - New partitions are placed using the same one-per-rack logic as creation - RF for new partitions = current rack count (may differ from original if racks changed) - Existing partitions are not affected +- Manual replica assignments are rejected (same as topic creation) This ensures consistency: all partitions of a diskless topic use rack-aware placement. +**Implementation note:** Currently `ReplicationControlManager.createPartitions()` uses the standard replica placer without diskless-specific handling. This needs to be updated to: +1. Detect diskless topics via `diskless.enable=true` config +2. Use rack-aware placement instead of standard placer +3. Reject manual replica assignments + ### Standard Operations (After Creation) After topic creation, diskless topics behave like classic Kafka topics for all operations: @@ -163,6 +328,9 @@ After topic creation, diskless topics behave like classic Kafka topics for all o - Standard Kafka leader election from ISR - For diskless topics, all replicas are always in ISR (data is in object storage) - `kafka-leader-election.sh` works normally +- Leader can be any replica in ISR (shuffled on election) + +**Important:** This is unchanged from current diskless behavior. For diskless topics with AZ-aware clients, leader election doesn't force cross-AZ traffic. The transformer returns the local replica to each client regardless of who the KRaft leader is. The KRaft leader is primarily for controller coordination (e.g., future RLM tasks), not for directing client traffic. **Broker Failure:** - Replicas on failed broker become offline @@ -254,7 +422,7 @@ For diskless topics, **all replicas are always considered in-sync** because: - All replicas have immediate access to the same committed data - No replication lag in the classic sense -ISR membership is trivially "all assigned replicas" for diskless partitions. +**Implementation:** ISR is stored in KRaft as all assigned replicas. At topic creation, controller sets `ISR = replicas`. No `AlterPartition` requests are needed since there's no lag tracking — replicas don't fetch from each other. **This is unchanged from today** — diskless topics already have this property. The difference is that now there are multiple replicas in the ISR (one per rack) instead of a single faked replica. @@ -329,6 +497,54 @@ With managed RF, diskless topics will have multiple replicas (one per rack). How The managed RF design **does not change** this behavior — it only changes how replica assignments are computed and stored in KRaft. +#### No Local Partition Objects — Implications + +**Current behavior:** `ReplicaManager` skips `getOrCreatePartition()` for diskless topics, meaning: +- No `Partition` objects on brokers +- No `UnifiedLog` objects +- No local log directories + +**This raises questions:** + +1. **How does leader election work without `Partition` objects?** + + Leader election is handled by the KRaft controller, not broker `Partition` objects. The controller updates partition metadata (leader, ISR) in KRaft records. Brokers observe these changes via metadata updates but don't need local `Partition` objects to "become" leader — they just serve requests for partitions where metadata says they're a replica. + +2. **How do brokers know they should serve requests for a partition?** + + Currently: `InklessTopicMetadataTransformer` returns metadata pointing clients to brokers. Brokers check `isDisklessTopic()` and route to Inkless handlers (`AppendHandler`, `FetchHandler`) instead of local log. + + With managed RF: Same flow, but transformer filters KRaft metadata instead of computing synthetic placement. + +3. **What about RLM integration?** + + RLM requires `Partition` and `UnifiedLog` objects to: + - Track local vs. tiered segment boundaries + - Coordinate segment uploads + - Run expiration tasks + + **This is out of scope for this design.** RLM integration will be addressed separately when implementing the tiering pipeline (see [DESIGN.md](DESIGN.md)). At that point, we may need to create `Partition` objects for the leader, but this can be deferred. + +4. **Can we avoid `Partition` objects entirely?** + + For the initial managed RF implementation: **yes**. We only need: + - KRaft metadata (replicas, leader, ISR) + - Transformer filtering + - Inkless handlers (already working) + + `Partition` objects become necessary when: + - Implementing local disk cache (future) + - Integrating with RLM for tiering (future) + + **Recommendation:** Start without `Partition` integration. Add it when implementing local cache or RLM integration. + +#### Research Needed + +Before implementation, verify: +1. Leader election works correctly without broker `Partition` objects (test with existing diskless topics) +2. ISR updates in KRaft don't require broker-side `Partition` state +3. `DescribeTopics` / `ListOffsets` work correctly for diskless topics with RF > 1 + --- ## Metadata Transformation @@ -355,8 +571,10 @@ The managed RF design **does not change** this behavior — it only changes how - Filtered `replicaNodes` / `isrNodes` for diskless partitions **Filtering logic:** -- If partition has replica in `clientAZ`: return only that local replica -- If no replica in `clientAZ`: fallback to cross-AZ view (full replica set or deterministic subset) +- If partition has replica in `clientAZ`: return only that local replica as leader/replicas/ISR +- If no replica in `clientAZ`: return full replica set (standard Kafka behavior — client talks to leader) + +**Note:** This is consistent with current diskless behavior. Today, the transformer returns one replica based on hash as a "fake" leader. With managed RF, we have one real replica per AZ and return the local one. The cross-AZ fallback (returning full replica set) handles edge cases like rack removal or misconfigured client AZ. **Detection:** - New binary always uses KRaft-placement-based projection for diskless topics @@ -368,11 +586,14 @@ The managed RF design **does not change** this behavior — it only changes how ### Metrics -**Controller metrics:** -- `kafka.controller.diskless.effective_rf{topic}` - RF assigned at creation -- `kafka.controller.diskless.rack_aware{topic,partition}` - 1 if one-per-rack, 0 if not -- `kafka.controller.diskless.rack_aware_total{topic}` - Count of rack-aware partitions -- `kafka.controller.diskless.non_rack_aware_total{topic}` - Count of non-rack-aware partitions +**Controller metrics (all prefixed `kafka.controller.diskless.`):** +- `effective_rf{topic}` - RF assigned at creation +- `rack_aware{topic,partition}` - 1 if one-per-rack, 0 if not +- `rack_aware_partitions_total{topic}` - Count of rack-aware partitions +- `non_rack_aware_partitions_total{topic}` - Count of non-rack-aware partitions +- `rf1_topics_total` - Count of diskless topics with RF=1 (need manual migration) +- `topics_migrated_total` - Count of topics auto-migrated from RF=1 +- `orphaned_topics_total` - Count of topics pending auto-migration **Standard Kafka metrics (already exist):** - `kafka.server:type=ReplicaManager,name=UnderReplicatedPartitions` @@ -407,6 +628,16 @@ This warning is logged: ## Implementation Path +### Phase 0: Research and Validation + +Verify assumptions before implementation: + +1. Leader election works correctly without broker `Partition` objects (test with existing diskless topics) +2. ISR updates in KRaft don't require broker-side `Partition` state +3. `DescribeTopics` / `ListOffsets` work correctly for diskless topics with RF > 1 + +**Estimate:** 1 week + ### Phase 1: Topic Creation with Rack-Aware Placement 1. Modify `ReplicationControlManager` to detect `diskless.enable=true` topics @@ -431,9 +662,18 @@ This warning is logged: **Estimate:** 1 week -### Phase 4: Observability +### Phase 4: Existing Topics Migration + +1. Implement orphaned replica detection (RF=1 with non-existent broker ID) +2. Add auto-migration logic in controller +3. Add pacing controls (batch size, interval) +4. Add metrics: `topics_migrated_total`, `orphaned_topics_total`, `rf1_topics_total` + +**Estimate:** 1 week + +### Phase 5: Observability -1. Add `rack_aware` metric +1. Add `rack_aware`, `rack_aware_partitions_total`, `non_rack_aware_partitions_total` metrics 2. Add warning logs for non-rack-aware placement 3. Documentation @@ -441,10 +681,54 @@ This warning is logged: ### Total Estimate -**6 weeks with 1 engineer, or 3-4 weeks with 2 engineers** +**8 weeks with 1 engineer, or 5 weeks with 2 engineers** This is significantly simpler than the original design which included reconciliation loops and drift handling. +### Testing Strategy + +**Unit Tests:** +- `ReplicationControlManager`: rack-aware placement logic, RF computation, validation +- `InklessTopicMetadataTransformer`: AZ filtering, cross-AZ fallback +- Migration detection: orphaned replica identification + +**Integration Tests:** +- Topic creation with `diskless.enable=true` results in RF=rack_count +- Add partitions uses rack-aware placement +- Existing topics with orphaned replicas are auto-migrated +- Reassignment works and logs warnings for non-rack-aware placement +- No replica fetcher threads started for diskless topics with RF > 1 + +**System Tests:** +- Multi-AZ cluster with diskless topics +- Client AZ awareness: verify clients talk to local replica +- Broker failure: verify leader election and continued availability +- Rolling restart: verify no disruption + +**Existing Test Coverage:** +See [DISKLESS_INTEGRATION_TEST_COVERAGE.md](../../DISKLESS_INTEGRATION_TEST_COVERAGE.md) for current Inkless test coverage. Managed RF tests should extend this framework. + +### Future Work: RLM Integration + +This design enables RLM integration but doesn't implement it. Key considerations for future RLM work: + +**Why RLM needs managed RF:** +- RLM's `onLeadershipChange()` is the entry point for tiering tasks +- Requires KRaft-managed partition leadership (not faked) +- Managed RF provides this foundation + +**What RLM integration will require:** +1. **Partition objects on leader:** RLM uses `Partition` and `UnifiedLog` APIs. The leader broker may need to create lightweight `Partition` objects for diskless topics. +2. **Segment boundary tracking:** RLM needs to know which offsets are local vs. tiered. For diskless, "local" means Control Plane (PostgreSQL), "tiered" means S3 segments. +3. **Expiration coordination:** Leader runs RLM expiration tasks to clean up tiered segments. + +**Design hints for RLM integration:** +- Create `Partition` objects only on the leader, not followers +- `UnifiedLog` can be a thin wrapper that delegates to Inkless handlers +- Alternatively, implement RLM hooks directly without full `Partition` — needs investigation + +**This is out of scope for managed RF** but the decisions here (KRaft-managed replicas, stable leader election) provide the foundation. See [DESIGN.md](DESIGN.md) Stream 7 for tiering pipeline details. + --- ## Rejected Alternatives @@ -477,6 +761,12 @@ See [DESIGN.md](DESIGN.md) Stream 7 for tiering pipeline details. **Concept:** Follow upstream Kafka patterns with `DisklessVersion` feature enum, server config, and KRaft records. +**Why we initially considered it:** +- Upstream Kafka uses feature flags for backward-compatible feature rollout (e.g., `EligibleLeaderReplicasVersion`) +- Allows gradual activation: deploy new binary first, then enable feature cluster-wide +- Provides explicit opt-in, reducing risk of unexpected behavior changes +- Supports mixed-version clusters where some brokers have the feature and others don't + **What it would have involved:** - Policy flag (`disklessManagedRF`) as new KRaft record - Feature enum (`DisklessVersion`) following `EligibleLeaderReplicasVersion` pattern @@ -484,74 +774,101 @@ See [DESIGN.md](DESIGN.md) Stream 7 for tiering pipeline details. - Manual activation via `kafka-features.sh --upgrade --feature diskless.version=1` **Why we dropped it:** -- Not needed; `diskless.enable` config is sufficient -- Binary version is the gate -- Adds operational step without benefit -- Binary rollback is simpler +- **Inkless deployment model is different**: We deploy new binaries on new VMs, not rolling upgrades on existing clusters. There's no mixed-version period to manage. +- **`diskless.enable` already gates behavior**: Topics must explicitly opt into diskless mode. The feature flag would be redundant. +- **Operational overhead without benefit**: Adds a manual activation step that doesn't provide additional safety in our deployment model. +- **Simpler rollback**: Binary rollback is sufficient; no need to coordinate feature flag state. **When to reconsider:** -- If upstreaming to Apache Kafka (strict backward compatibility) -- If needing per-topic opt-in/opt-out -- If supporting long-lived mixed-version clusters +- If upstreaming to Apache Kafka (strict backward compatibility requirements) +- If needing per-topic opt-in/opt-out for managed RF specifically +- If supporting long-lived mixed-version clusters (not our deployment model) ### Rejected Alternative: Dynamic Reconciliation **Concept:** Controller continuously reconciles replica placement to maintain one-per-rack as topology changes. +**Why we initially considered it:** +- Ensures rack-awareness is always maintained, even after topology changes +- Automatically adapts RF when racks are added (more availability) or removed (avoid under-replication) +- Reduces operational burden — no manual reassignment needed +- Provides "self-healing" behavior similar to Kubernetes operators + **What it would have involved:** - Reconciliation loop monitoring rack changes - Automatic RF adjustment when racks added/removed -- Drift detection and auto-fix -- Rack liveness state machine (HEALTHY → DEGRADED → UNAVAILABLE) +- Drift detection and auto-fix when operator breaks rack-awareness +- Rack liveness state machine (HEALTHY → DEGRADED → UNAVAILABLE) to distinguish transient failures from permanent rack loss **Why we dropped it:** -- Significant complexity in controller -- Divergent behavior from standard Kafka operations -- Topology changes are rare in practice -- Operator can handle edge cases manually -- Standard Kafka reassignment tools work fine +- **Significant complexity**: Adds new controller component with its own state machine, edge cases, and failure modes +- **Divergent from Kafka norms**: Standard Kafka doesn't auto-adjust RF or auto-reassign. Operators expect explicit control. +- **Rare scenarios**: Topology changes (adding/removing racks) are infrequent in practice. Optimizing for rare cases adds constant complexity. +- **Existing tools work**: `kafka-reassign-partitions.sh` handles all cases. Operators already know this workflow. +- **Harder to reason about**: Auto-reconciliation can surprise operators. "Why did my partition move?" is a common complaint with auto-rebalancing systems. **When to reconsider:** -- If customers frequently add/remove racks -- If manual reassignment becomes burdensome +- If customers frequently add/remove racks and manual reassignment becomes burdensome +- If we build a broader "Kafka operator" that manages cluster topology holistically ### Rejected Alternative A: Virtual Leader for RLM Tasks -**Concept:** Keep RF=1, designate deterministic "virtual leader" per partition for RLM tasks. +**Concept:** Keep RF=1 (current faked metadata), but designate a deterministic "virtual leader" per partition for RLM tasks. + +**Why we considered it:** +- Avoids changing the current RF=1 model that works for produce/consume +- RLM only needs *some* broker to run expiration tasks — could be a "virtual" designation +- Simpler than managing real replicas if we can make RLM work with virtual leadership **Why it fails:** -- Two sources of leadership creates confusion -- Virtual leader failover requires new coordination mechanism -- RLM assumes real `Partition` objects with `UnifiedLog` -- Tiering pipeline merge phase needs `ReplicaManager` context +- **Two sources of leadership**: KRaft has one leader (faked), RLM needs another (virtual). Confusing for operators and tooling. +- **Failover complexity**: Virtual leader failover requires new coordination mechanism outside KRaft. What happens when the virtual leader dies? +- **RLM assumptions**: RLM code assumes real `Partition` objects with `UnifiedLog`. Significant refactoring needed to work with virtual concept. +- **Tiering pipeline needs `ReplicaManager`**: The merge phase that creates tiered segments needs broker-side context that only exists with real partitions. ### Rejected Alternative B: Control Plane Manages Tiered Expiration -**Concept:** Extend PostgreSQL to track tiered segment metadata and run expiration directly. +**Concept:** Extend PostgreSQL (Control Plane) to track tiered segment metadata and run expiration directly, bypassing RLM. + +**Why we considered it:** +- Control Plane already tracks batch metadata — could extend to track tiered segments +- Avoids needing KRaft-managed replicas entirely +- Keeps all Inkless metadata in one place (PostgreSQL) **Why it fails:** -- Contradicts goal of reducing PG load -- Duplicates RLM retention logic -- Breaks RLMM integration and Kafka tooling -- Creates cross-system consistency problems +- **Contradicts PG scalability goal**: The whole point of tiering is to *reduce* PG load. Adding more metadata to PG defeats the purpose. +- **Duplicates RLM logic**: Retention policies (`retention.ms`, `retention.bytes`) are already implemented in RLM. Reimplementing in Control Plane doubles the code and bugs. +- **Breaks tooling**: Kafka admin tools expect RLM for tiered storage management. Custom Control Plane expiration wouldn't integrate. +- **Cross-system consistency**: Tiered segments in S3, metadata in PG, Kafka expecting RLM — three systems to keep consistent. Recipe for orphaned data. ### Rejected Alternative C: Direct Tiered Read via FetchHandler -**Concept:** Extend `FetchHandler` to read tiered storage directly, bypassing RLM read path. +**Concept:** Extend `FetchHandler` to read tiered storage (S3 segments) directly, bypassing RLM read path. + +**Why we considered it:** +- `FetchHandler` already serves diskless reads from object storage — could extend to tiered segments +- Avoids RLM dependency for reads +- Potentially simpler than full RLM integration **Why it fails:** -- Only solves read path, not expiration -- Must combine with Alternative A or B -- Duplicates RLM index handling +- **Only solves reads**: Expiration (the main RLM value) still needs a solution. Must combine with Alternative A or B, inheriting their problems. +- **Duplicates index handling**: RLM maintains indexes for tiered segments. `FetchHandler` would need to duplicate this or depend on RLM indexes anyway. +- **Partial solution**: Doesn't address the core problem (needing KRaft leadership for RLM). Just moves complexity around. ### Rejected Alternative D: Treat Tiered Data as Read-Only Archival -**Concept:** Freeze tiered portion, use S3 lifecycle policies for expiration. +**Concept:** Freeze tiered portion as read-only archive, use S3 lifecycle policies for expiration instead of RLM. + +**Why we considered it:** +- S3 lifecycle policies are simple and battle-tested +- Avoids RLM complexity entirely for expiration +- "Archival" use case doesn't need sophisticated retention **Why it fails:** -- No programmatic retention (can't implement `retention.ms`/`retention.bytes`) -- Breaks topic deletion cleanup -- Doesn't address PG scalability for new diskless data +- **No programmatic retention**: S3 lifecycle can't implement `retention.ms` or `retention.bytes` based on Kafka semantics. Can only do "delete after N days" globally. +- **Topic deletion broken**: Deleting a Kafka topic should clean up tiered data. S3 lifecycle doesn't know about Kafka topics. +- **Doesn't solve PG scalability**: The goal is to tier *new* diskless data to reduce PG load. This alternative only addresses old/archived data. +- **User expectations**: Users expect Kafka retention semantics to work. "Your retention.ms doesn't apply to tiered data" is a poor user experience. --- @@ -608,5 +925,11 @@ The managed RF design ensures a KRaft-managed partition leader exists for step 4 ## Open Items -- Behavior when `broker.rack` config changes on a broker (rare edge case) -- Whether to add a tool to "re-rack-aware" a topic (reassign to restore one-per-rack) +**Resolved in this design:** +- ~~Transformer fallback behavior~~ → Return full replica set (standard Kafka behavior) +- ~~ISR storage~~ → Stored in KRaft as all replicas +- ~~`broker.rack` config changes~~ → Use existing reassignment tooling to fix placement if needed (rare edge case) +- ~~Re-rack-aware tooling~~ → Use existing `kafka-reassign-partitions.sh` (no new tooling needed) + +**Research (Phase 0):** +- See [Phase 0: Research and Validation](#phase-0-research-and-validation) From 0b0d562d85bd447e2a4aa7a150ae17cfb80f8319 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Tue, 13 Jan 2026 15:51:47 +0200 Subject: [PATCH 04/21] fixup! add diskless managed rf feature design --- .../ts-unification/DISKLESS_MANAGED_RF.md | 243 ++++++++++++------ 1 file changed, 169 insertions(+), 74 deletions(-) diff --git a/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md b/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md index fcff2ea26d..e9c2bac967 100644 --- a/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md +++ b/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md @@ -14,8 +14,13 @@ - [No Replica Fetcher for Diskless Topics](#no-replica-fetcher-for-diskless-topics) 6. [Metadata Transformation](#metadata-transformation) 7. [Observability](#observability) -8. [Implementation Path](#implementation-path) +8. [Implementation Path](#implementation-path) *(9 weeks / 1 eng, 5-6 weeks / 2 eng)* - [Phase 0: Research and Validation](#phase-0-research-and-validation) + - [Phase 1: Topic Creation with Rack-Aware Placement](#phase-1-topic-creation-with-rack-aware-placement) + - [Phase 2: Transformer Changes](#phase-2-transformer-changes) + - [Phase 3: Add Partitions Support](#phase-3-add-partitions-support) + - [Phase 4: Offline Replica Auto-Reassignment](#phase-4-offline-replica-auto-reassignment) + - [Phase 5: Observability](#phase-5-observability) - [Testing Strategy](#testing-strategy) - [Future Work: RLM Integration](#future-work-rlm-integration) 9. [Rejected Alternatives](#rejected-alternatives) @@ -30,7 +35,7 @@ Enable **rack-aware, stable KRaft-managed replicas** for diskless topics: - **KRaft-managed replicas**: Diskless topics will have actual KRaft-managed replicas, not the current "faked" RF=1 metadata. These replicas are tracked by the controller, participate in leader election, and appear in standard Kafka tooling. The key difference from classic replicas: **they don't replicate data** (object storage handles durability), but they do provide partition assignments for RLM integration and operational consistency. See [No Replica Fetcher for Diskless Topics](#no-replica-fetcher-for-diskless-topics) for implementation details. - **Rack-aware at creation**: Topics are created with one replica per rack (RF = rack count) -- **Standard operations after creation**: Once created, diskless topics use standard Kafka replica management — no custom reconciliation or drift handling +- **Standard operations after creation**: Once created, diskless topics use standard Kafka replica management. The only automatic action is **offline replica reassignment** to preserve availability (see [Existing Diskless Topics](#existing-diskless-topics)). - **Controller-managed RF**: Users don't specify RF at creation; the controller computes it from rack topology. Requests with `replicationFactor=-1` or `replicationFactor=1` are accepted for compatibility (both result in RF=rack_count). - **Leader-agnostic produce and consume**: Keep the diskless model where any replica can accept writes and serve reads, while still having a KRaft leader for controller duties (e.g., future RLM coordination) @@ -116,42 +121,57 @@ Legacy topic in KRaft: ``` IF diskless.enable=true - AND RF=1 - AND replica broker ID is NOT in active cluster -THEN auto-migrate to RF=rack_count + AND ANY replica broker is offline +THEN: + IF broker ID is not registered in cluster metadata: + # Legacy topic from old cluster + Reassign ALL replicas, expand to RF=rack_count + ELSE: + # Broker exists but offline (failure/maintenance) + Reassign ONLY offline replica(s), preserve current RF ``` -#### Why This Is Safe +**Key distinction:** +- **Not registered:** Broker ID was never seen by this cluster (e.g., old cluster had brokers 1-6, new cluster has 101-106). This is a legacy topic that should be modernized. +- **Registered but offline:** Broker ID exists in cluster metadata but is currently unavailable. Operator chose this RF intentionally, so preserve it. -The detection has **no false positives** given the deployment model: +#### Why This Is Safe for Diskless -| Scenario | RF | Broker ID | Auto-migrate? | -|----------|-----|-----------|---------------| -| Legacy diskless (old cluster) | 1 | Orphaned (old broker ID) | ✅ Yes | -| Already migrated diskless | rack_count | Valid | ❌ No (RF > 1) | -| Operator-managed diskless | 1 | Valid (current broker) | ❌ No (valid ID) | -| Classic topic | any | Valid | ❌ No (not diskless) | +Auto-reassigning offline replicas is safe for diskless topics because: -**Key insight:** The deployment model (new VMs = new broker IDs) ensures legacy topics have orphaned broker IDs. +| Property | Classic Topics | Diskless Topics | +|----------|----------------|-----------------| +| Data location | Local broker disk | Object storage (shared) | +| Reassignment cost | Data copy (slow) | Metadata only (instant) | +| Data loss risk | Yes (if out of sync) | No (all brokers see same data) | +| ISR meaning | Replication lag tracking | Meaningless (all always in sync) | + +**This preserves "always available" semantics:** +- Current diskless: Transformer hashes to any alive broker → always available +- Managed RF diskless: Controller reassigns offline replicas to online brokers → always available + +| Scenario | Broker Status | RF Change | Action | +|----------|---------------|-----------|--------| +| Legacy topic, cluster migration | Not registered | Expand to rack_count | Modernize to rack-aware | +| RF=3, one broker failed | Registered, offline | Keep RF=3 | Replace offline replica | +| RF=1, broker maintenance | Registered, offline | Keep RF=1 | Move to online broker | +| Classic topic | Any | N/A | No action (not diskless) | #### Other Deployment Models (Broker IDs Overlap) -If the new cluster reuses broker IDs from the old cluster (e.g., in-place upgrade on same VMs), automatic migration won't trigger because the broker ID is still valid. +If the new cluster reuses broker IDs from the old cluster (e.g., in-place upgrade on same VMs): -**In this case:** -- Topics continue to work (RF=1 is functional, just not rack-aware) -- Operator uses standard replica placement tools to migrate manually +**If old brokers are offline:** Auto-reassignment triggers (offline replica detection) +**If old brokers are online:** Topics work normally with existing placement + +**For RF=1 topics that should be expanded to rack-aware:** - Metric `kafka.controller.diskless.rf1_topics_total` tracks diskless topics with RF=1 -- Log warning: `WARN Diskless topic 'foo' has RF=1; consider migrating to rack-aware placement` +- Log warning: `WARN Diskless topic 'foo' has RF=1; consider expanding to rack-aware placement` +- Operator can manually expand using `kafka-reassign-partitions.sh` -**Manual migration steps:** +**Manual expansion to rack-aware:** -1. Generate reassignment plan: ```bash -# List diskless topics with RF=1 -kafka-topics.sh --bootstrap-server localhost:9092 --describe \ - | grep -B1 "ReplicationFactor: 1" | grep "diskless.enable=true" - # Generate reassignment JSON (one replica per rack) cat > reassignment.json << 'EOF' { @@ -161,32 +181,42 @@ cat > reassignment.json << 'EOF' ] } EOF -``` -2. Execute reassignment: -```bash +# Execute (instant for diskless - metadata only) kafka-reassign-partitions.sh --bootstrap-server localhost:9092 \ --reassignment-json-file reassignment.json \ --execute ``` -3. Verify: -```bash -kafka-reassign-partitions.sh --bootstrap-server localhost:9092 \ - --reassignment-json-file reassignment.json \ - --verify -``` +#### Reassignment Process -**Note:** For diskless topics, reassignment is metadata-only (no data movement), so it completes instantly. +When controller detects a diskless partition with offline replica(s): -#### Migration Process +**Case 1: Broker ID does not exist in cluster (e.g., cluster migration)** -When controller detects an orphaned diskless topic: +This indicates the partition was created on an old cluster with different broker IDs. Expand to rack-aware placement. -1. Log: `INFO Detected orphaned diskless topic 'foo' (RF=1, broker 3 not in cluster)` -2. Compute target replica assignment (one broker per rack) +1. Log: `INFO Detected diskless partition 'foo-0' with non-existent broker (broker 3 not registered)` +2. Compute target: one broker per rack (RF=rack_count) 3. Issue `AlterPartitionReassignments` to set new replicas -4. Log: `INFO Migrated diskless topic 'foo' to RF=3, replicas=[1,3,5]` +4. Log: `INFO Reassigned diskless partition 'foo-0' to RF=3, replicas=[101,103,105]` + +**Case 2: Broker ID exists but is offline (e.g., broker failure, maintenance)** + +This indicates a temporary or permanent broker outage. Preserve current RF, just replace offline replica. + +1. Log: `INFO Detected diskless partition 'foo-0' with offline replica (broker 101)` +2. Find online broker in same rack as offline broker (preserve rack-awareness) +3. If no broker in same rack: pick any online broker (preserve RF, log warning about rack-awareness) +4. Issue `AlterPartitionReassignments` to replace offline replica +5. Log: `INFO Reassigned diskless partition 'foo-0' replica: broker 101 → 102` + +**Summary:** + +| Condition | RF Change | Rationale | +|-----------|-----------|-----------| +| Broker not registered in cluster | Expand to rack_count | Legacy topic from old cluster, modernize | +| Broker registered but offline | Keep current RF | Operator's RF choice respected | **This is metadata-only** — no data movement required (data is in object storage). @@ -210,23 +240,23 @@ After auto-migration: One broker per rack (A, B, C) ``` -#### When Migration Runs +#### When Auto-Reassignment Runs - **Trigger:** Controller startup, metadata change, or periodic scan - **Pacing:** Batched to avoid overwhelming controller (configurable) -- **Idempotent:** Re-running on already-migrated topics is a no-op (RF > 1) +- **Idempotent:** Re-running when no offline replicas exist is a no-op #### Observability -**Auto-migration:** -- Log: `INFO Detected orphaned diskless topic 'foo' (RF=1, broker 3 not in cluster)` -- Log: `INFO Migrated diskless topic 'foo' to RF=3` -- Metric: `kafka.controller.diskless.topics_migrated_total` (counter) -- Metric: `kafka.controller.diskless.orphaned_topics_total` (gauge) — pending migration +**Auto-reassignment of offline replicas:** +- Log: `INFO Detected diskless partition 'foo-0' with offline replica (broker 101)` +- Log: `INFO Reassigned diskless partition 'foo-0' replica: broker 101 → 102` +- Metric: `kafka.controller.diskless.replicas_reassigned_total` (counter) +- Metric: `kafka.controller.diskless.offline_replicas_total` (gauge) — pending reassignment -**RF=1 with valid broker (needs manual migration):** -- Log: `WARN Diskless topic 'foo' has RF=1; consider migrating to rack-aware placement` -- Metric: `kafka.controller.diskless.rf1_topics_total` (gauge) — topics that need manual migration +**RF=1 topics (informational):** +- Log: `WARN Diskless topic 'foo' has RF=1; consider expanding to rack-aware placement` +- Metric: `kafka.controller.diskless.rf1_topics_total` (gauge) — topics with RF=1 #### Rollback @@ -334,9 +364,10 @@ After topic creation, diskless topics behave like classic Kafka topics for all o **Broker Failure:** - Replicas on failed broker become offline -- Partition becomes under-replicated (reported in metrics) +- Controller detects offline replica and **auto-reassigns** to online broker (preserving RF and rack-awareness when possible) - Leader fails over if leader was on failed broker -- When broker returns, replica immediately rejoins ISR (no catch-up needed) +- Partition remains available (unlike classic topics which would be under-replicated) +- If original broker returns later, no automatic action — new assignment is kept **Broker Shutdown (Controlled):** - Standard controlled shutdown @@ -385,11 +416,12 @@ If an operator reassigns replicas such that rack-awareness is broken (e.g., 2 re - New topics will include the new rack **Rack removed (all brokers gone):** -- Partitions with replicas on that rack become under-replicated -- Standard Kafka under-replication handling -- Operator reassigns to remove references to gone rack +- Replicas on that rack become offline +- Controller auto-reassigns to online brokers in other racks (preserving RF) +- Rack-awareness may be temporarily broken (logged as warning) +- Operator can manually rebalance to restore rack-awareness if desired -**Note:** These are edge cases that rarely happen in practice. The design prioritizes simplicity over automatic handling of rare topology changes. +**Note:** These are edge cases that rarely happen in practice. Auto-reassignment ensures availability; rack-awareness restoration is optional. --- @@ -540,7 +572,7 @@ The managed RF design **does not change** this behavior — it only changes how #### Research Needed -Before implementation, verify: +These items are covered in [Phase 0: Research and Validation](#phase-0-research-and-validation): 1. Leader election works correctly without broker `Partition` objects (test with existing diskless topics) 2. ISR updates in KRaft don't require broker-side `Partition` state 3. `DescribeTopics` / `ListOffsets` work correctly for diskless topics with RF > 1 @@ -571,15 +603,36 @@ Before implementation, verify: - Filtered `replicaNodes` / `isrNodes` for diskless partitions **Filtering logic:** -- If partition has replica in `clientAZ`: return only that local replica as leader/replicas/ISR -- If no replica in `clientAZ`: return full replica set (standard Kafka behavior — client talks to leader) +1. Check if KRaft-assigned replicas exist in the cluster (are alive) +2. If replicas exist: + - If partition has replica in `clientAZ`: return only that local replica as leader/replicas/ISR + - If no replica in `clientAZ`: return full replica set (standard Kafka behavior — client talks to leader) +3. If replicas are orphaned (none exist in cluster): **fall back to legacy hash-based selection** + +**Why the orphaned fallback is needed:** -**Note:** This is consistent with current diskless behavior. Today, the transformer returns one replica based on hash as a "fake" leader. With managed RF, we have one real replica per AZ and return the local one. The cross-AZ fallback (returning full replica set) handles edge cases like rack removal or misconfigured client AZ. +During cluster migration, there's a brief window between: +1. New cluster starts with restored KRaft metadata (contains old broker IDs) +2. Controller completes auto-migration to new broker IDs + +In this window, legacy topics have `Replicas=[3]` where broker 3 no longer exists. Without the fallback: +- New transformer would return empty/invalid metadata +- Partition would be effectively offline for clients + +The fallback ensures partitions remain available during this migration window (typically seconds to minutes). Once auto-migration completes, the fallback is no longer triggered. + +**Note:** This is consistent with current diskless behavior. Today, the transformer returns one replica based on hash as a "fake" leader. With managed RF, we have one real replica per AZ and return the local one. The orphaned fallback preserves availability only during the migration window. **Detection:** - New binary always uses KRaft-placement-based projection for diskless topics - Check `diskless.enable=true` topic config to identify diskless topics +### Unclean Leader Election + +For diskless topics, `unclean.leader.election.enable=true` can be safely enabled — there is no data loss risk since all data is in object storage. + +However, **auto-reassignment of offline replicas** (not unclean election) is our primary availability mechanism. See [Rejected Alternative E: Unclean Leader Election for Availability](#rejected-alternative-e-unclean-leader-election-for-availability) for the full analysis. + --- ## Observability @@ -591,9 +644,9 @@ Before implementation, verify: - `rack_aware{topic,partition}` - 1 if one-per-rack, 0 if not - `rack_aware_partitions_total{topic}` - Count of rack-aware partitions - `non_rack_aware_partitions_total{topic}` - Count of non-rack-aware partitions -- `rf1_topics_total` - Count of diskless topics with RF=1 (need manual migration) -- `topics_migrated_total` - Count of topics auto-migrated from RF=1 -- `orphaned_topics_total` - Count of topics pending auto-migration +- `rf1_topics_total` - Count of diskless topics with RF=1 +- `replicas_reassigned_total` - Count of replicas auto-reassigned due to offline broker +- `offline_replicas_total` - Count of replicas pending reassignment **Standard Kafka metrics (already exist):** - `kafka.server:type=ReplicaManager,name=UnderReplicatedPartitions` @@ -628,6 +681,8 @@ This warning is logged: ## Implementation Path +**Total Estimate: 9 weeks with 1 engineer, or 5-6 weeks with 2 engineers** + ### Phase 0: Research and Validation Verify assumptions before implementation: @@ -651,7 +706,8 @@ Verify assumptions before implementation: 1. Update `InklessTopicMetadataTransformer` to read KRaft placement 2. Implement AZ filtering logic -3. Remove synthetic hashing calculation +3. **Add orphaned replica fallback** — if KRaft replicas don't exist, fall back to legacy hash-based selection +4. Remove synthetic hashing calculation (but keep as fallback path) **Estimate:** 2 weeks @@ -662,28 +718,38 @@ Verify assumptions before implementation: **Estimate:** 1 week -### Phase 4: Existing Topics Migration +### Phase 4: Offline Replica Auto-Reassignment -1. Implement orphaned replica detection (RF=1 with non-existent broker ID) -2. Add auto-migration logic in controller +1. Implement offline replica detection (any diskless partition with offline broker) +2. Add auto-reassignment logic in controller: + - Broker ID not registered in cluster → expand to RF=rack_count (legacy topic modernization) + - Broker ID registered but offline → keep current RF, replace offline replica (same rack if possible) 3. Add pacing controls (batch size, interval) -4. Add metrics: `topics_migrated_total`, `orphaned_topics_total`, `rf1_topics_total` +4. Add metrics: `replicas_reassigned_total`, `offline_replicas_total`, `rf1_topics_total` -**Estimate:** 1 week +**Estimate:** 2 weeks (increased due to broader scope) ### Phase 5: Observability 1. Add `rack_aware`, `rack_aware_partitions_total`, `non_rack_aware_partitions_total` metrics -2. Add warning logs for non-rack-aware placement +2. Add warning logs for non-rack-aware placement and RF=1 topics 3. Documentation **Estimate:** 1 week -### Total Estimate +### Summary -**8 weeks with 1 engineer, or 5 weeks with 2 engineers** +| Phase | Scope | Estimate | +|-------|-------|----------| +| 0 | Research and Validation | 1 week | +| 1 | Topic Creation with Rack-Aware Placement | 2 weeks | +| 2 | Transformer Changes | 2 weeks | +| 3 | Add Partitions Support | 1 week | +| 4 | Offline Replica Auto-Reassignment | 2 weeks | +| 5 | Observability | 1 week | +| **Total** | | **9 weeks (1 eng) / 5-6 weeks (2 eng)** | -This is significantly simpler than the original design which included reconciliation loops and drift handling. +This includes the offline replica auto-reassignment which preserves "always available" semantics. ### Testing Strategy @@ -698,6 +764,9 @@ This is significantly simpler than the original design which included reconcilia - Existing topics with orphaned replicas are auto-migrated - Reassignment works and logs warnings for non-rack-aware placement - No replica fetcher threads started for diskless topics with RF > 1 +- **Orphaned replica fallback**: Transformer returns valid broker when KRaft replicas don't exist +- **Offline replica auto-reassignment**: Broker goes offline → replica reassigned to online broker +- **Legacy topic modernization**: Broker ID not in cluster → RF expanded to rack_count **System Tests:** - Multi-AZ cluster with diskless topics @@ -870,6 +939,30 @@ See [DESIGN.md](DESIGN.md) Stream 7 for tiering pipeline details. - **Doesn't solve PG scalability**: The goal is to tier *new* diskless data to reduce PG load. This alternative only addresses old/archived data. - **User expectations**: Users expect Kafka retention semantics to work. "Your retention.ms doesn't apply to tiered data" is a poor user experience. +### Rejected Alternative E: Unclean Leader Election for Availability + +**Concept:** Enable `unclean.leader.election.enable=true` for diskless topics to ensure availability when replicas go offline. + +**Why we considered it:** +- For diskless topics, there is no data loss risk from unclean election (all data is in object storage) +- ISR membership is a metadata concept, not a data consistency concept +- Simple configuration change, no new code required +- Standard Kafka mechanism + +**Why it's not the primary solution:** +- **Doesn't work for RF=1**: Unclean election needs multiple replicas to elect from. With RF=1, there's no alternative replica. +- **Reactive, not proactive**: Waits for election to happen rather than proactively ensuring availability +- **Doesn't preserve rack-awareness**: Elects from existing (possibly degraded) replica set + +**What we do instead:** +Auto-reassignment of offline replicas is more powerful: +- Works for any RF (including RF=1) +- Proactively moves replica to online broker +- Preserves rack-awareness when possible +- Metadata-only operation (instant) + +**Note:** Unclean leader election *can* be enabled for diskless topics (no downside), but auto-reassignment is the primary availability mechanism. + --- ## Appendix: Topic Migration Interactions @@ -906,10 +999,12 @@ Topic migration (Tiered Classic → Diskless) may trigger replica set changes: **After migration:** RF=rack_count with managed placement (one per rack) If rack_count differs from original RF: -- Controller reconciles to target placement +- Controller adjusts to target placement as part of the migration process - Uses standard add-then-remove approach - Paced to avoid disruption +**Note:** This is a one-time adjustment during topic migration, not ongoing reconciliation. + ### Interaction with Tiering Pipeline After topic migration to Diskless: From e0a62ef5c9bfae57ebe98a970c506af29064bd33 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Wed, 14 Jan 2026 10:45:00 +0200 Subject: [PATCH 05/21] fixup! add diskless managed rf feature design --- .../ts-unification/DISKLESS_MANAGED_RF.md | 69 +- .../DISKLESS_MANAGED_RF_SIMPLIFIED.md | 1109 +++++++++++++++++ 2 files changed, 1167 insertions(+), 11 deletions(-) create mode 100644 docs/inkless/ts-unification/DISKLESS_MANAGED_RF_SIMPLIFIED.md diff --git a/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md b/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md index e9c2bac967..6840445c54 100644 --- a/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md +++ b/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md @@ -1,20 +1,23 @@ # Diskless-Managed Replication Factor +> **Alternative design under review:** [DISKLESS_MANAGED_RF_SIMPLIFIED.md](DISKLESS_MANAGED_RF_SIMPLIFIED.md) proposes a simpler approach using transformer-first availability instead of controller auto-reassignment. Review both before deciding. + ## Table of Contents -1. [Objectives](#objectives) -2. [Activation Model](#activation-model-binary-version) +1. [Purpose](#purpose) +2. [Objectives](#objectives) +3. [Activation Model](#activation-model-binary-version) - [Existing Diskless Topics](#existing-diskless-topics) -3. [Placement Model](#placement-model) -4. [Controller Behavior](#controller-behavior) +4. [Placement Model](#placement-model) +5. [Controller Behavior](#controller-behavior) - [Topic Creation](#topic-creation) - [Add Partitions](#add-partitions) - [Standard Operations](#standard-operations-after-creation) -5. [Broker Behavior](#broker-behavior) +6. [Broker Behavior](#broker-behavior) - [No Replica Fetcher for Diskless Topics](#no-replica-fetcher-for-diskless-topics) -6. [Metadata Transformation](#metadata-transformation) -7. [Observability](#observability) -8. [Implementation Path](#implementation-path) *(9 weeks / 1 eng, 5-6 weeks / 2 eng)* +7. [Metadata Transformation](#metadata-transformation) +8. [Observability](#observability) +9. [Implementation Path](#implementation-path) *(9 weeks / 1 eng, 5-6 weeks / 2 eng)* - [Phase 0: Research and Validation](#phase-0-research-and-validation) - [Phase 1: Topic Creation with Rack-Aware Placement](#phase-1-topic-creation-with-rack-aware-placement) - [Phase 2: Transformer Changes](#phase-2-transformer-changes) @@ -23,9 +26,53 @@ - [Phase 5: Observability](#phase-5-observability) - [Testing Strategy](#testing-strategy) - [Future Work: RLM Integration](#future-work-rlm-integration) -9. [Rejected Alternatives](#rejected-alternatives) -10. [Appendix: Topic Migration Interactions](#appendix-topic-migration-interactions) -11. [Open Items](#open-items) +10. [Rejected Alternatives](#rejected-alternatives) +11. [Appendix: Topic Migration Interactions](#appendix-topic-migration-interactions) +12. [Open Items](#open-items) + +--- + +## Purpose + +### Current State: RF=1 with Transformer Override + +Current diskless topics use **RF=1 in KRaft** with the transformer routing requests to any alive broker. This was a deliberate decision that enabled: +- Fast iteration without controller changes +- Simple implementation +- Flexibility in routing + +**This has worked well for pure diskless topics.** + +### The Question: What Does Topic Migration Require? + +We need to support **bidirectional topic migration**: +1. **Classic → Diskless** (immediate need) +2. **Diskless → Classic** (future need) + +### Why Real Replicas Matter + +| Capability | RF=1 (current) | RF=rack_count (this design) | +|-------------------------------|-----------------------------|-----------------------------| +| Topic migration coordination | Custom coordinator needed | Leader coordinates | +| RLM integration | Custom hooks (~11 weeks) | Standard (~0 weeks) | +| Background job assignment | Racing/randomized | Deterministic (leader) | +| Diskless → Classic migration | RF expansion + migration | Migration only | +| Operational debugging | "Which broker ran this?" | "Leader of partition X" | + +### What This Feature Enables + +| Capability | How | Status | +|---------------------------|----------------------------------------|----------------| +| Rack-aware placement | One replica per AZ at creation | ✅ This design | +| Topic migration | Real replicas to migrate to | ✅ This design | +| RLM integration | Stable partition leader | 🔜 Future | +| Standard Kafka operations | Reassignment, election work normally | ✅ This design | +| Deterministic job owner | Leader owns partition operations | ✅ This design | + +### Related Documents + +- [DESIGN.md](DESIGN.md) — Overall tiered storage unification design +- [DISKLESS_MANAGED_RF_SIMPLIFIED.md](DISKLESS_MANAGED_RF_SIMPLIFIED.md) — Simplified variant (transformer-first availability) --- diff --git a/docs/inkless/ts-unification/DISKLESS_MANAGED_RF_SIMPLIFIED.md b/docs/inkless/ts-unification/DISKLESS_MANAGED_RF_SIMPLIFIED.md new file mode 100644 index 0000000000..9c273bf1bd --- /dev/null +++ b/docs/inkless/ts-unification/DISKLESS_MANAGED_RF_SIMPLIFIED.md @@ -0,0 +1,1109 @@ +# Diskless-Managed Replication Factor (Simplified Design) + +> **This is a draft alternative to [DISKLESS_MANAGED_RF.md](DISKLESS_MANAGED_RF.md)** that reduces complexity by using transformer-first availability instead of controller auto-reassignment. + +## Table of Contents + +1. [Purpose](#purpose) +2. [Objectives](#objectives) +3. [Key Simplification: Transformer-First Availability](#key-simplification-transformer-first-availability) +4. [Activation Model](#activation-model-binary-version) + - [Existing Diskless Topics](#existing-diskless-topics) +5. [Placement Model](#placement-model) +6. [Controller Behavior](#controller-behavior) + - [Topic Creation](#topic-creation) + - [Add Partitions](#add-partitions) + - [Standard Operations](#standard-operations-after-creation) +7. [Broker Behavior](#broker-behavior) +8. [Metadata Transformation](#metadata-transformation) +9. [Observability](#observability) +10. [Implementation Path](#implementation-path) *(6 weeks / 1 eng, 4 weeks / 2 eng)* +11. [Trade-offs vs Original Design](#trade-offs-vs-original-design) +12. [Rejected Alternatives](#rejected-alternatives) + - [F: Keep Classic RF, Continue Faking](#rejected-alternative-f-keep-classic-rf-continue-faking-metadata) + - [G: Controller Auto-Reassignment](#rejected-alternative-g-controller-auto-reassignment-from-original-design) + - [H: Shrink RF During Migration](#rejected-alternative-h-shrink-rf-during-migration) + - [I: Custom Job Coordinator](#rejected-alternative-i-custom-job-coordinator-without-kraft-leadership) + +--- + +## Purpose + +### Current State: RF=1 with Transformer Override + +Current diskless topics use **RF=1 in KRaft** with the transformer routing requests to any alive broker. The single replica may point to an **offline or non-existent broker** — the transformer ignores KRaft metadata entirely and hashes to any alive broker. + +``` +Current diskless: + KRaft: Partition 0 → Replicas=[1], Leader=1 ← may be offline/stale + Transformer: Ignores KRaft, hashes to any alive broker +``` + +This was a deliberate decision that enabled: +- Fast iteration without controller changes +- Simple implementation +- "Always available" semantics (transformer routes around any failure) + +**This has worked well for pure diskless topics.** + +### The Question: What Does Topic Migration Require? + +We need to support **bidirectional topic migration**: +1. **Classic → Diskless** (immediate need) +2. **Diskless → Classic** (future need) + +The key question: **how does each approach handle reverse migration?** + +#### Diskless → Classic Migration (Future) + +Before comparing approaches for Classic → Diskless, consider what's needed for the reverse: + +| Aspect | A: RF=1 | B: RF=3 faked | C: RF=rack_count | +|-------------------|-----------------------|-------------------------|-------------------------| +| Starting RF | 1 | 3 (ghost replicas) | 3 (real replicas) | +| Before migration | Must expand RF=1 → RF=3| Must "un-ghost" replicas| **Ready as-is** | +| Replica state | Create from scratch | Create `Partition` objects| Already correct | +| Complexity | High (2 operations) | Medium (fix ghosts) | **Low (just migrate)** | + +**Key insight:** Only approach C is ready for reverse migration without additional work. + +#### How Critical Is Replica-Awareness for Topic Migration? + +Now compare three approaches for Classic → Diskless migration: + +| Aspect | A: Shrink to RF=1 | B: Keep RF=3, fake | C: RF=rack_count (this) | +|-------------------------|--------------------------|--------------------------|--------------------------| +| KRaft RF after migrate | 1 | 3 (unchanged) | 3 (managed, rack-aware) | +| Transformer behavior | Hash to any broker | Hash to any broker | Filter by AZ (KRaft-sourced) | +| Replica objects | None (offline in KRaft) | Ghost (no `Partition`) | Real (metadata-only) | +| Migration coordinator | Custom (who?) | Custom (who?) | **Leader** | +| RLM integration | Custom hooks | Custom hooks | **Standard** | +| Job assignment | Racing | Racing | **Leader-based** | +| Reverse migration ready | No (must expand RF) | No (must fix ghosts) | **Yes** | + +**Terminology:** +- **Offline replica**: KRaft shows replica on broker X, but broker X doesn't exist or is down. No `Partition` object. +- **Ghost replica**: KRaft shows replica on broker X, broker X is alive, but `ReplicaManager` skips creating `Partition` object for diskless. Broker doesn't know it's a replica. +- **Real (metadata-only) replica**: KRaft shows replica on broker X, broker X recognizes it as a replica, but no local log (data in object storage). + +**Approach A: Shrink to RF=1 during migration** + +``` +Classic: Replicas=[1,2,3], Leader=1, data on all brokers + ↓ migration starts +Shrink: Replicas=[1], Leader=1 ← When does this happen? + ↓ Who removes replicas 2,3? + ↓ What if broker 1 fails mid-shrink? +Diskless: Replicas=[1], data in object storage + Transformer hashes to any alive broker (ignores KRaft) +``` + +Problems: +- Timing of RF shrink is complex (before seal? during? after?) +- Single point of failure during migration +- Loses rack-awareness from classic topic +- **Reverse migration requires RF expansion first** + +**Approach B: Keep classic RF=3, continue faking metadata** + +``` +Classic: Replicas=[1,2,3], Leader=1, data on all brokers + ↓ migration starts (no RF change!) +Diskless: Replicas=[1,2,3], Leader=1, data in object storage + ↑ KRaft unchanged, transformer ignores and hashes + Brokers 1,2,3 are "ghost replicas" +``` + +What "ghost replica" means: +- KRaft says broker is a replica with ISR membership +- But `ReplicaManager.isDisklessTopic()` skips `getOrCreatePartition()` +- No `Partition` object, no fetcher, no local log +- Broker doesn't know it's a "replica" — just serves requests via Inkless handlers +- ISR never shrinks (no lag tracking for diskless) + +Problems: +- **RLM still blocked** — `onLeadershipChange()` needs `Partition` objects to run tiering tasks. Ghost replicas don't have them. +- **Reverse migration requires "un-ghosting"** — must create real `Partition` objects before migrating back +- Side effects: misleading tooling output (`describe topic` shows healthy replicas that aren't real) +- See [What Happens to "Ignored" Replicas?](#what-happens-to-ignored-replicas) + +**Approach C: RF=rack_count with real replicas (this design)** + +``` +Classic: Replicas=[1,2,3], Leader=1, data on all brokers + ↓ migration starts +Diskless: Replicas=[1,3,5], Leader=1, data in object storage + ↑ rack-aware placement, real metadata-only replicas + Leader coordinates migration, RLM tasks, background jobs +``` + +What "real (metadata-only) replica" means: +- KRaft says broker is a replica with ISR membership +- Broker recognizes it's a replica for this partition +- No local log (data in object storage) — but could have `Partition` object for coordination +- Leader can run RLM tasks, coordinate migration, own background jobs + +Benefits: +- **Leader coordinates migration** — deterministic owner +- **Standard RLM integration** — `onLeadershipChange()` works +- **Deterministic job assignment** — leader owns partition operations +- **Accurate tooling output** — replicas are real +- **Reverse migration ready** — already has RF=3 with real replicas + +### Cost of Continuing with RF=1 + +If we keep RF=1 and use transformer override for topic migration: + +**Challenges:** + +1. **When does RF shrink happen?** + - Classic topic has RF=3 with replicas [1,2,3] + - Control Plane marks migration point, seals the topic + - But when do we remove replicas 2 and 3? + - Before seal? During? After data is in object storage? + - What if broker 1 fails mid-transition? + +2. **Migration coordination without real leadership:** + - With RF=1, the single broker "owns" the migration + - If it fails, how do we pick a new coordinator? + - Custom leader election outside KRaft? Or use KRaft but ignore it elsewhere? + +3. **RLM integration blocked:** + - `onLeadershipChange()` won't fire without real KRaft leadership + - Tiering pipeline needs a leader to coordinate segment merging + - Custom hooks needed to bypass RLM's standard entry points + - See [RLM Integration: Critical Decision](#rlm-integration-critical-decision) below + +4. **Diskless → Classic complexity:** + - Must expand RF=1 → RF=3 as part of migration + - Two complex operations combined: RF expansion + data migration + - More failure modes, harder to reason about + +5. **Alternative: Keep existing placement, keep faking?** + - Classic RF=3 becomes diskless RF=3, but transformer still fakes + - Zero metadata changes, simplest migration + - But: RLM still blocked, tooling still confused + - Defers the problem, doesn't solve it + - See [What Happens to "Ignored" Replicas?](#what-happens-to-ignored-replicas) below + +**What RF=1 continuation requires:** +- Custom migration coordinator (not partition leader) +- Custom RLM integration hooks (bypass `onLeadershipChange()`) +- RF expansion logic embedded in Diskless → Classic migration +- Ongoing maintenance of parallel code paths + +### Cost of Moving to RF=rack_count + +**One-time investment:** +- Controller changes for rack-aware placement (~2 weeks) +- Transformer changes for AZ filtering (~2 weeks) +- Testing and validation (~2 weeks) + +**What we get:** +- Topic migration uses standard replica handoff patterns +- RLM integration uses standard `onLeadershipChange()` +- Diskless → Classic migration starts from correct RF +- Single code path for replica management + +### Decision Framework + +| Factor | Keep RF=1 | Move to RF=rack_count | +|--------------------------|--------------------------|----------------------------| +| Short-term cost | Lower | ~6 weeks | +| Classic → Diskless | Custom coordination | Standard replica handoff | +| RLM integration | Custom hooks | Standard leadership hook | +| Diskless → Classic | RF expansion + migration | Migration only | +| Long-term maintenance | Two code paths | One code path | +| Kafka alignment | Divergent | Aligned | + +### What Happens to "Ignored" Replicas? + +If we keep RF=3 in KRaft but the transformer ignores it, what does Kafka do with those replicas? + +**Current diskless behavior in `ReplicaManager.scala`:** + +```scala +// applyLocalLeadersDelta - when broker becomes leader +localLeaders.foreachEntry { (tp, info) => + if (!_inklessMetadataView.isDisklessTopic(tp.topic())) + getOrCreatePartition(tp, delta, info.topicId) // SKIPPED for diskless! +} + +// applyLocalFollowersDelta - when broker becomes follower +localFollowers.foreachEntry { (tp, info) => + if (!_inklessMetadataView.isDisklessTopic(tp.topic())) + getOrCreatePartition(tp, delta, info.topicId) // SKIPPED for diskless! +} +``` + +**What this means:** + +| Aspect | Classic Topic | Diskless Topic (current) | +|-------------------------|----------------------------|------------------------------| +| `Partition` objects | Created on each replica | **Never created** | +| Local log directories | Created on each replica | **Never created** | +| Replica fetcher threads | Started for followers | **Never started** | +| ISR tracking | Leader tracks follower lag | **Skipped** (no lag to track)| +| Leader election | Uses `Partition` state | KRaft metadata only | + +**The "ghost replica" problem:** + +If KRaft says `Replicas=[1,2,3]` but diskless skips `Partition` creation: + +``` +KRaft metadata: + Partition 0: Replicas=[1,2,3], Leader=1, ISR=[1,2,3] + +Broker 1 (leader): + - No Partition object for this topic + - No local log + - Serves requests via Inkless handlers + +Broker 2 (follower): + - No Partition object + - No fetcher thread (nothing to fetch) + - Doesn't know it's a "follower" + +Broker 3 (follower): + - Same as broker 2 +``` + +**Will they become offline?** + +| Scenario | What Happens | +|------------------------------|-------------------------------------------| +| Broker 2 restarts | KRaft still shows it as replica, ISR=all | +| Broker 2 decommissioned | KRaft shows offline, but no data lost | +| Leader election triggered | KRaft picks new leader, but no `Partition`| +| ISR shrink check | **Skipped** for diskless (no lag to check)| + +**Key insight:** Replicas won't become "offline" in the Kafka sense because: +1. ISR shrink is skipped for diskless topics +2. No fetcher means no lag detection +3. KRaft ISR stays as "all replicas" forever + +**But this creates confusion:** + +```bash +$ kafka-topics.sh --describe --topic diskless-foo + +Topic: diskless-foo Replicas: 1,2,3 ISR: 1,2,3 + ↑ + Looks healthy! + But brokers 2,3 aren't really "in sync" + They don't even know they're replicas +``` + +**The operational risk:** + +| Risk | Impact | +|------------------------------|-------------------------------------------| +| False sense of redundancy | Operator thinks RF=3 provides safety | +| Confusing metrics | Under-replicated = 0, but replicas unused | +| Tooling mismatch | `describe` shows healthy, reality differs | +| Decommission confusion | Remove broker 2, ISR still shows it | +| Debugging difficulty | "Why isn't replication working?" | + +**Summary:** + +Keeping replicas but ignoring them creates **ghost replicas**: +- KRaft thinks they exist and are healthy +- Brokers don't know they're replicas +- ISR never shrinks (no lag tracking) +- Operators get misleading information +- No actual redundancy benefit + +This is a form of **technical debt** that compounds over time. + +--- + +### Current Diskless Job Assignment: Racing Brokers + +**How diskless background jobs work today:** + +Diskless has background jobs (merger, cleaner, etc.) that need to run periodically. Currently, these use a **racing/randomized** approach: + +``` +Current model (no leader): + Broker 0: "I'll try to clean partition X" ──┐ + Broker 1: "I'll try to clean partition X" ──┼── Race! First one wins + Broker 2: "I'll try to clean partition X" ──┘ + + Coordination via: PostgreSQL locks, randomized delays, distributed claims +``` + +**Why this was chosen:** +- Simpler implementation (no leader election needed) +- Works without real KRaft replicas +- Good enough for initial diskless MVP + +**The operational complexity:** + +| Aspect | Racing Model | +|---------------------------|---------------------------------------| +| Job assignment | Random/first-come-first-serve | +| Duplicate work | Possible (mitigated by PG locks) | +| Debugging | "Which broker ran this job?" | +| Monitoring | Jobs scattered across brokers | +| Failure handling | Another broker eventually picks up | +| Load distribution | Probabilistic, not guaranteed | + +**What leadership enables:** + +With real KRaft-managed leaders, background jobs have a **home**: + +``` +Leader-based model: + Partition X leader = Broker 1 + + Broker 0: "Not leader of X, skip" + Broker 1: "I'm leader of X, I run jobs for X" ← Deterministic! + Broker 2: "Not leader of X, skip" + + Coordination via: KRaft leadership (standard Kafka) +``` + +| Aspect | Leader Model | +|---------------------------|---------------------------------------| +| Job assignment | Deterministic (leader owns it) | +| Duplicate work | Impossible (single leader) | +| Debugging | "Leader of partition X ran this" | +| Monitoring | Jobs grouped by leader | +| Failure handling | Leader election, new leader takes over| +| Load distribution | Controlled by partition assignment | + +**Jobs that benefit from leadership:** + +| Job | Current (racing) | With Leader | +|---------------------------|----------------------------|---------------------------------| +| Batch merging | Any broker can try | Leader merges its partitions | +| Segment cleanup | Race to delete | Leader cleans its partitions | +| Tiering (diskless→TS) | Complex coordination | Leader uploads via RLM | +| Expiration | Distributed claims | Leader runs RLM expiration | +| Offset tracking | Scattered | Leader maintains offsets | + +**The TS migration benefit:** + +Tiered Storage migration requires uploading segments in TS format. With leadership: + +``` +Without leader (current): + Who uploads segment for partition X? + → Need custom coordinator + → Race conditions possible + → Custom logic to track progress + +With leader: + Partition X leader uploads segments + → Reuse RLM upload logic directly + → Standard Kafka pattern + → Progress tracked via RLM APIs +``` + +**Key insight:** + +Leadership isn't just about RLM integration — it's about having a **deterministic owner** for partition-level operations. This simplifies: +- Implementation (reuse Kafka patterns) +- Operations (clear ownership) +- Debugging (know who did what) +- Future features (upload, compaction, etc.) + +### Operational Wins from Leadership + +**1. Clear accountability:** +``` +Racing model: + Alert: "Partition X merger failed" + Operator: "Which broker? Let me check all broker logs..." + +Leader model: + Alert: "Partition X merger failed on broker 2 (leader)" + Operator: "Check broker 2 logs for partition X" +``` + +**2. Predictable load distribution:** +``` +Racing model: + Broker 0: 45% of merger jobs (got lucky) + Broker 1: 35% of merger jobs + Broker 2: 20% of merger jobs (got unlucky) + +Leader model: + Broker 0: Jobs for partitions it leads (balanced by assignment) + Broker 1: Jobs for partitions it leads + Broker 2: Jobs for partitions it leads +``` + +**3. Simpler capacity planning:** + +| Aspect | Racing | Leader | +|---------------------|--------------------------------|----------------------------------| +| Job count per broker| Unpredictable | = partitions led | +| Resource usage | Spiky, varies by luck | Proportional to leadership | +| Scaling decision | "Add brokers, hope it helps" | "Rebalance partition leaders" | + +**4. Easier incident response:** + +| Scenario | Racing | Leader | +|---------------------------|----------------------------------|----------------------------------| +| Job stuck | Which broker has it? | Leader of partition X | +| Job running slow | Check all brokers | Check leader broker | +| Want to pause jobs | Disable on all brokers | Move leadership away | +| Debug job history | Correlate logs across brokers | Single broker's logs | + +**5. Standard Kafka operations:** + +```bash +# Racing model: No good way to move jobs +# "Jobs run wherever, hope for the best" + +# Leader model: Standard Kafka tooling +kafka-leader-election.sh --topic foo --partition 0 --election-type preferred +# Jobs for partition 0 now run on the new leader +``` + +**6. Monitoring alignment with Kafka:** + +| Metric | Racing | Leader | +|-------------------------------|-------------------------|-------------------------------| +| Jobs per broker | Custom metric | Partition leadership count | +| Job failures | Custom alerting | Under-replicated partitions | +| Load imbalance | Custom detection | Leader skew metrics | + +**Summary: Operational benefits of leadership** + +| Benefit | Impact | +|----------------------------|------------------------------------------------| +| Single point of truth | Leader owns partition, no ambiguity | +| Standard tooling | `kafka-leader-election`, `kafka-reassign` | +| Predictable load | Jobs proportional to leadership | +| Simpler debugging | One broker to check per partition | +| Kafka-native monitoring | Reuse existing metrics and alerts | +| Incident response | Move leadership to isolate/debug | + +--- + +### RLM Integration: Critical Decision + +**Why RLM matters for diskless:** + +The tiering pipeline (diskless batches → tiered segments) is critical for PostgreSQL scalability. Without it, all batch metadata stays in PG forever, which becomes the bottleneck. + +``` +Current diskless data flow: + Produce → Object Storage (WAL) → PostgreSQL (batch metadata) + ↑ + This grows unbounded! + +Target diskless data flow: + Produce → Object Storage (WAL) → PostgreSQL (recent batches) + ↓ tiering pipeline + Tiered Storage (aged batches) + ↓ RLM expiration + Deleted (per retention policy) +``` + +**How RLM works:** + +```java +// RemoteLogManager.java - entry point +public void onLeadershipChange(Set partitions, ...) { + for (Partition partition : partitions) { + // Start tiering tasks for this partition + // Only runs on the LEADER + scheduleTieringTasks(partition); + } +} +``` + +RLM requires: +1. **Real KRaft leadership** — `onLeadershipChange()` must fire +2. **`Partition` objects** — RLM uses `Partition` and `UnifiedLog` APIs +3. **Leader coordination** — Only leader writes tiered segments + +**Cost of deferring RLM integration (keeping RF=1/faked):** + +| Aspect | Custom Solution Required | Effort | +|---------------------------|-----------------------------------|-----------| +| Tiering entry point | Bypass `onLeadershipChange()` | ~2 weeks | +| Leader coordination | Custom leader election for tiering| ~2 weeks | +| Segment merging | Who merges? How to coordinate? | ~3 weeks | +| RLM API compatibility | Fake `Partition` objects or fork | ~2 weeks | +| Expiration tasks | Custom expiration outside RLM | ~2 weeks | +| **Total custom work** | | **~11 weeks** | + +**Cost of doing RF=rack_count now:** + +| Aspect | Standard Solution | Effort | +|---------------------------|-----------------------------------|-----------| +| Rack-aware placement | Modify `ReplicationControlManager`| ~2 weeks | +| Transformer filtering | Update `InklessTopicMetadataTransformer` | ~2 weeks | +| Add partitions | Same logic as creation | ~1 week | +| RLM integration | Standard `onLeadershipChange()` | ~0 weeks* | +| **Total work** | | **~6 weeks** | + +*RLM integration comes "for free" once we have real leadership. + +**The math:** + +``` +Option A: Keep RF=1, defer RLM = 0 weeks now + 11 weeks later = 11 weeks total +Option B: Do RF=rack_count now = 6 weeks now + 0 weeks later = 6 weeks total + ───────────────────────────── + Savings: 5 weeks +``` + +**But the real cost is worse:** + +If we defer, we carry technical debt: +- Two code paths for leadership (KRaft vs. custom) +- Custom tiering pipeline that diverges from Kafka +- Harder to maintain, harder to reason about +- Every new feature asks "does this work with faked RF?" + +### Recommendation + +**Move to RF=rack_count now** because: +1. Topic migration (both directions) benefits from real replicas +2. RLM integration becomes standard rather than custom +3. One-time 6-week investment vs. 11+ weeks of custom work +4. Avoids accumulating technical debt +5. Aligns diskless with Kafka's replica model + +### Related Documents + +- [DESIGN.md](DESIGN.md) — Overall tiered storage unification design (includes topic migration, tiering pipeline details) + +--- + +## Objectives + +Enable **rack-aware, stable KRaft-managed replicas** for diskless topics: + +- **KRaft-managed replicas**: Diskless topics will have actual KRaft-managed replicas (RF = rack count) +- **Rack-aware at creation**: Topics are created with one replica per rack +- **Standard operations after creation**: Once created, standard Kafka replica management applies +- **Controller-managed RF**: Users don't specify RF; controller computes from rack topology +- **Leader-agnostic produce and consume**: Any replica can accept writes and serve reads +- **Always available via transformer**: Transformer ensures availability by routing to alive brokers + +**Key difference from original design:** Availability is handled by the transformer (instant), not controller auto-reassignment. Metadata accuracy is eventual. + +--- + +## Key Simplification: Transformer-First Availability + +### The Insight + +For diskless topics, **availability** and **metadata accuracy** can be decoupled: + +| Concern | Priority | Who Handles | Speed | +|-------------------|--------------|-------------------|------------| +| Availability | Critical | Transformer | Instant | +| Metadata accuracy | Nice-to-have | Controller (lazy) | Eventually | + +### How It Works + +**Current diskless behavior (RF=1, faked):** +- Transformer hashes to any alive broker +- Partition is always available +- KRaft metadata is ignored for routing + +**Simplified managed RF behavior:** +- KRaft stores RF=rack_count with real broker IDs +- Transformer filters by client AZ from KRaft replicas +- **If assigned replica is offline → fall back to any alive broker in AZ** +- Partition is always available (same as today!) +- KRaft metadata may be temporarily stale (shows offline broker) + +### Why This Works for Diskless + +``` +Classic topic: + Data on broker X → client MUST talk to broker X + Broker X offline → partition unavailable (until reassigned) + +Diskless topic: + Data in object storage → ANY broker can serve + Broker X offline → transformer routes to broker Y + Partition stays available (no reassignment needed!) +``` + +The transformer already has all the information it needs: +1. KRaft replica assignments (preferred brokers) +2. Alive brokers in cluster +3. Client AZ + +It can make instant routing decisions without waiting for controller. + +--- + +## Activation Model (Binary Version) + +*Unchanged from original design.* + +Managed RF is activated by **deploying a new binary version**. No feature flags, no topic attributes, no server configs. + +### Behavior Summary + +**Old binary:** +- Diskless topics use legacy "RF=1 / faked metadata" behavior +- Transformer calculates synthetic placement via hashing + +**New binary:** +- Diskless topics use KRaft-managed placement (one replica per rack at creation) +- Transformer filters KRaft placement by client AZ +- **Falls back to alive brokers if assigned replicas are offline** + +### Existing Diskless Topics + +Existing diskless topics (RF=1) continue to work: + +**Immediate availability:** +- Transformer sees RF=1 with potentially offline broker +- Falls back to hash-based selection (same as today) +- No downtime + +**Eventual modernization (optional):** +- Controller can lazily detect RF=1 diskless topics +- Background task expands to RF=rack_count when convenient +- Not urgent — availability is already handled by transformer + +**Manual modernization (alternative):** +- Operator uses `kafka-reassign-partitions.sh` to expand RF +- Same tooling as classic topics + +```bash +# Optional: manually expand legacy topic to rack-aware +kafka-reassign-partitions.sh --bootstrap-server localhost:9092 \ + --reassignment-json-file reassignment.json \ + --execute +``` + +**Key difference from original:** No urgent auto-migration needed. Transformer handles availability immediately. + +--- + +## Placement Model + +*Unchanged from original design.* + +### Rack-Aware Placement at Creation + +When a diskless topic is created: +- Controller determines current rack count from registered brokers +- RF is set to rack count (e.g., 3 racks → RF=3) +- One broker is selected per rack for each partition +- Broker selection within a rack uses load balancing (least loaded broker) + +### Placement Is Static After Creation + +Once a topic is created: +- RF does not automatically change if racks are added/removed +- Replica assignments don't automatically adjust +- Standard Kafka replica management applies + +--- + +## Controller Behavior + +### Topic Creation + +*Unchanged from original design.* + +When creating diskless topics (`diskless.enable=true`): +- Controller counts distinct racks from registered brokers +- RF = rack count +- One replica assigned per rack +- Reject `replicationFactor > 1` and manual assignments + +### Add Partitions + +*Unchanged from original design.* + +New partitions use same one-per-rack logic as creation. + +### Standard Operations (After Creation) + +**Key difference: No auto-reassignment on broker failure.** + +**Leader Election:** +- Standard Kafka leader election from ISR +- For diskless topics, all replicas are always in ISR + +**Broker Failure:** +- Replicas on failed broker become offline in KRaft metadata +- **Transformer routes around offline broker** (instant) +- Partition remains available +- KRaft shows under-replicated (informational only for diskless) +- When broker returns, it's back in ISR immediately + +**Reassignment:** +- `kafka-reassign-partitions.sh` works normally +- Operator can reassign replicas if desired +- **Not required for availability** — transformer handles routing + +### What We Remove + +No controller logic for: +- ❌ Proactive offline replica detection +- ❌ Immediate auto-reassignment +- ❌ "Not registered vs offline" distinction +- ❌ Pacing controls for reassignment +- ❌ Complex state machine + +--- + +## Broker Behavior + +*Unchanged from original design.* + +- No replica fetcher threads for diskless topics +- No local `Partition` objects +- Inkless handlers (`AppendHandler`, `FetchHandler`) serve requests + +--- + +## Metadata Transformation + +### Updated Filtering Logic + +``` +FOR each diskless partition: + assigned_replicas = KRaft replica set + alive_replicas = assigned_replicas ∩ alive_brokers + + IF alive_replicas is not empty: + # Normal case: use KRaft placement + IF any alive_replica in clientAZ: + RETURN local replica (AZ-aware routing) + ELSE: + RETURN all alive_replicas (cross-AZ fallback) + ELSE: + # All assigned replicas offline: fall back to hash + RETURN hash-based selection from all alive brokers in clientAZ +``` + +### Key Properties + +1. **Instant availability**: No waiting for controller +2. **AZ-aware when possible**: Uses KRaft placement if alive +3. **Graceful degradation**: Falls back to hash if needed +4. **No metadata dependency for availability**: Stale metadata doesn't cause downtime + +### Comparison + +| Scenario | Original Design | Simplified Design | +|----------------------|----------------------------|------------------------------| +| Broker fails | Wait for controller reassign| Instant transformer fallback | +| All replicas offline | Wait for controller | Instant hash fallback | +| Client routing | After KRaft updated | Immediate | + +--- + +## Observability + +### Metrics + +**Controller metrics:** +- `kafka.controller.diskless.effective_rf{topic}` - RF assigned at creation +- `kafka.controller.diskless.rack_aware{topic,partition}` - 1 if one-per-rack, 0 if not +- `kafka.controller.diskless.rf1_topics_total` - Legacy topics not yet modernized + +**Transformer metrics (new):** +- `kafka.server.diskless.transformer.fallback_total` - Count of hash fallbacks +- `kafka.server.diskless.transformer.offline_replicas_routed_around` - Routing decisions + +**Standard Kafka metrics:** +- `UnderReplicatedPartitions` - Will show diskless partitions with offline brokers +- Note: For diskless, "under-replicated" is informational, not critical + +### Logs + +``` +INFO [Transformer] Diskless partition 'foo-0' has offline replica (broker 101), + routing to alive replica (broker 103) + +WARN [Transformer] Diskless partition 'foo-0' has ALL replicas offline, + using hash-based fallback to broker 105 +``` + +### Admin Surfaces + +**`kafka-topics.sh --describe` may show stale info:** +``` +Topic: foo PartitionCount: 3 ReplicationFactor: 3 + Partition: 0 Leader: 101 Replicas: 101,103,105 Isr: 101,103,105 + ↑ + Broker 101 may be offline, but transformer routes to 103 +``` + +This is acceptable because: +1. Availability is not affected +2. Operator can see actual state via broker metrics +3. Eventually consistent (when broker returns or is reassigned) + +--- + +## Implementation Path + +**Total Estimate: 6 weeks with 1 engineer, or 4 weeks with 2 engineers** + +### Phase 0: Research and Validation (1 week) + +*Same as original.* + +1. Leader election works without broker `Partition` objects +2. ISR updates don't require broker-side `Partition` state +3. `DescribeTopics` / `ListOffsets` work with RF > 1 + +### Phase 1: Topic Creation with Rack-Aware Placement (2 weeks) + +*Same as original.* + +1. Modify `ReplicationControlManager` to detect diskless topics +2. Compute RF = rack count at creation +3. Implement one-per-rack broker selection +4. Reject `replicationFactor > 1` and manual assignments + +### Phase 2: Transformer Changes (2 weeks) + +Updated scope: + +1. Update `InklessTopicMetadataTransformer` to read KRaft placement +2. Implement AZ filtering logic +3. **Add offline replica routing** — if KRaft replica offline, route to alive replica +4. **Add hash fallback** — if all replicas offline, use legacy hash +5. Add metrics for fallback tracking + +### Phase 3: Add Partitions Support (1 week) + +*Same as original.* + +1. Apply same one-per-rack logic when adding partitions +2. Handle case where rack count changed since topic creation + +### ~~Phase 4: Offline Replica Auto-Reassignment~~ (REMOVED) + +**Not needed.** Transformer handles availability. + +### ~~Phase 5: Observability~~ → Phase 4: Observability (included in Phase 2) + +Simplified — mostly transformer metrics. + +### Summary + +| Phase | Scope | Estimate | +|-----------|------------------------------------------|--------------------------------| +| 0 | Research and Validation | 1 week | +| 1 | Topic Creation with Rack-Aware Placement | 2 weeks | +| 2 | Transformer Changes (with fallback) | 2 weeks | +| 3 | Add Partitions Support | 1 week | +| **Total** | | **6 weeks (1 eng) / 4 weeks (2 eng)** | + +**Savings: 3 weeks** (removed Phase 4: Auto-Reassignment) + +--- + +## Trade-offs vs Original Design + +### What We Gain + +| Benefit | Impact | +|-------------------------------|---------------------------------------------| +| Simpler implementation | 3 weeks saved, less code to maintain | +| No controller complexity | No state machine, pacing, edge cases | +| Instant availability | Transformer decides immediately | +| Consistent with current | Transformer fallback = what diskless does | +| Fewer moving parts | Less to debug, less to break | + +### What We Accept + +| Trade-off | Impact | Mitigation | +|---------------------------|---------------------------------|----------------------------------| +| Stale KRaft metadata | `describe` shows offline broker | Transformer metrics show reality | +| Under-replicated alerts | Metrics fire for diskless | Document as informational | +| Manual legacy modernization| RF=1 stays until reassigned | Provide runbook | +| No auto rack restoration | Broken rack-awareness persists | Same as original (operator fixes)| + +### When Original Design Is Better + +Choose original design if: +- KRaft metadata accuracy is critical for tooling/automation +- You want `describe topic` to always show current state +- Operators expect under-replicated alerts to always require action + +Choose simplified design if: +- You prioritize simplicity and faster delivery +- You accept "eventually consistent" metadata +- Current diskless transformer behavior is acceptable baseline + +--- + +## Rejected Alternatives + +*Includes alternatives from original design, plus alternatives specific to the "keep faking" approach.* + +### Rejected Alternative F: Keep Classic RF, Continue Faking Metadata + +**Concept:** When migrating Classic → Diskless, keep the existing RF (e.g., RF=3) in KRaft but continue using the transformer to fake metadata. No replica management changes. + +**Variations considered:** + +1. **Keep RF=3, transformer ignores:** Classic RF=3 becomes diskless RF=3, transformer hashes to any alive broker +2. **Shrink to RF=1 during migration:** Classic RF=3 shrinks to RF=1, then uses current diskless model +3. **Keep placement, add custom coordination:** Keep RF=3, add custom job coordinator outside KRaft + +**Why we considered it:** +- Zero controller changes for migration +- Simplest migration path (just flip `diskless.enable`) +- Works with existing diskless implementation +- Faster time to initial migration + +**Why we rejected it:** + +1. **Ghost replicas problem:** + - KRaft shows `Replicas=[1,2,3], ISR=[1,2,3]` + - But brokers 2,3 have no `Partition` objects, no fetchers + - ISR never shrinks (no lag tracking for diskless) + - Operators get misleading information + - See [What Happens to "Ignored" Replicas?](#what-happens-to-ignored-replicas) + +2. **RLM integration blocked:** + - `onLeadershipChange()` won't fire without real leadership + - Custom hooks needed (~11 weeks vs ~6 weeks for real replicas) + - See [RLM Integration: Critical Decision](#rlm-integration-critical-decision) + +3. **Racing job assignment continues:** + - Background jobs (merger, cleaner) remain randomized + - No deterministic owner for partition operations + - See [Current Diskless Job Assignment: Racing Brokers](#current-diskless-job-assignment-racing-brokers) + +4. **RF shrink complexity:** + - If shrinking RF=3 → RF=1, when does this happen? + - Who coordinates the shrink? What if coordinator fails? + - See [Cost of Continuing with RF=1](#cost-of-continuing-with-rf1) + +5. **Diskless → Classic migration harder:** + - Must expand RF=1 → RF=3 as part of reverse migration + - Two complex operations combined + +6. **Technical debt compounds:** + - Two systems: KRaft replicas (classic) vs transformer magic (diskless) + - Every new feature asks "does this work with faked RF?" + - Harder to maintain, harder to onboard + +**Cost comparison:** + +| Approach | Now | Later | Total | +|-------------------------|---------|----------|-----------| +| Keep faking + custom RLM| 0 weeks | ~11 weeks| 11+ weeks | +| RF=rack_count now | 6 weeks | 0 weeks | 6 weeks | + +**When this might make sense:** +- If RLM integration is not needed (PG can scale indefinitely) +- If bidirectional migration is not needed +- If operational confusion from ghost replicas is acceptable +- None of these apply to our use case + +--- + +### Rejected Alternative G: Controller Auto-Reassignment (from original design) + +**Concept:** Controller proactively detects offline replicas and reassigns to online brokers immediately. + +**Why we considered it:** +- KRaft metadata always accurate +- Standard tooling shows correct state +- Consistent with future direction (KRaft as source of truth) + +**Why we chose transformer-first instead:** +- **3 weeks additional complexity** for something transformer already handles +- **Instant vs eventually consistent** — transformer is faster +- **Consistent with today** — diskless already uses transformer for availability +- **Fewer failure modes** — no controller state machine to debug + +**When to reconsider:** +- If KRaft metadata accuracy becomes critical requirement +- If RLM integration needs accurate replica state for leader selection +- If operators require `describe topic` to always reflect reality + +--- + +### Rejected Alternative H: Shrink RF During Migration + +**Concept:** When migrating Classic → Diskless, shrink RF=3 to RF=1 as part of the migration process, then use current diskless model. + +**Why we considered it:** +- Maintains consistency with current RF=1 diskless model +- No need to change existing diskless implementation +- Single replica = simpler mental model + +**Why we rejected it:** + +1. **Timing complexity:** + - When does RF shrink happen? Before seal? During? After? + - Who removes replicas 2 and 3? + - What if coordinator fails mid-shrink? + +2. **Reverse migration harder:** + - Diskless → Classic requires RF=1 → RF=3 expansion + - Two operations: expand RF + migrate data + +3. **Loses rack-awareness:** + - Classic topics are often rack-aware (RF=3 across 3 racks) + - Shrinking to RF=1 loses this property + - Must re-establish rack-awareness later + +4. **All problems of RF=1:** + - Ghost replicas if we keep KRaft RF=3 + - RLM integration blocked + - Racing job assignment + +--- + +### Rejected Alternative I: Custom Job Coordinator (Without KRaft Leadership) + +**Concept:** Keep faking metadata, but add a custom coordinator (outside KRaft) to assign background jobs deterministically. + +**Why we considered it:** +- Gets deterministic job ownership without changing replica model +- Could use PG-based leader election +- Avoids KRaft changes + +**Why we rejected it:** + +1. **Two leadership systems:** + - KRaft has its own leader concept + - Custom coordinator has different leader + - Which one is authoritative? + +2. **RLM still blocked:** + - RLM uses KRaft leadership, not custom coordinator + - Still need custom hooks for tiering + +3. **More complexity, not less:** + - Now have KRaft + transformer + custom coordinator + - Three systems to keep consistent + +4. **Defeats the purpose:** + - If we're adding a coordinator, why not use KRaft's? + - KRaft leadership is battle-tested, custom is not + +--- + +## Migration Path: Original → Simplified + +If we start with simplified and later need original: + +1. Simplified design is a **subset** of original +2. Can add controller auto-reassignment later without breaking changes +3. Transformer fallback remains as safety net + +If we start with original and want to simplify: + +1. Remove controller auto-reassignment code +2. Transformer already has fallback logic +3. Accept stale metadata + +**Recommendation:** Start with simplified. Add controller complexity only if proven necessary. From ba838f516dee38e75dd3f979db5b28795239d664 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Wed, 14 Jan 2026 11:15:21 +0200 Subject: [PATCH 06/21] fixup! add diskless managed rf feature design --- .../DISKLESS_MANAGED_RF_FINAL.md | 635 ++++++++++++++++++ 1 file changed, 635 insertions(+) create mode 100644 docs/inkless/ts-unification/DISKLESS_MANAGED_RF_FINAL.md diff --git a/docs/inkless/ts-unification/DISKLESS_MANAGED_RF_FINAL.md b/docs/inkless/ts-unification/DISKLESS_MANAGED_RF_FINAL.md new file mode 100644 index 0000000000..8f65d14f20 --- /dev/null +++ b/docs/inkless/ts-unification/DISKLESS_MANAGED_RF_FINAL.md @@ -0,0 +1,635 @@ +# Diskless-Managed Replication Factor + +## Table of Contents + +1. [Purpose](#purpose) +2. [Objectives](#objectives) +3. [Design: Transformer-First Availability](#design-transformer-first-availability) +4. [Activation Model](#activation-model-binary-version) +5. [Placement Model](#placement-model) +6. [Controller Behavior](#controller-behavior) +7. [Broker Behavior](#broker-behavior) +8. [Metadata Transformation](#metadata-transformation) +9. [Observability](#observability) +10. [Implementation Path](#implementation-path) *(6 weeks / 1 eng, 4 weeks / 2 eng)* +11. [Rejected Alternatives](#rejected-alternatives) +12. [Appendix](#appendix) + - [Ghost Replicas Problem](#ghost-replicas-problem) + - [Operational Benefits Summary](#operational-benefits-summary) + - [Leadership Benefits for Operations](#leadership-benefits-for-operations) + +--- + +## Purpose + +### Current State: RF=1 with Transformer Override + +Current diskless topics use **RF=1 in KRaft** with the transformer routing requests to any alive broker. The single replica may point to an **offline or non-existent broker** — the transformer ignores KRaft metadata entirely and hashes to any alive broker. + +``` +Current diskless: + KRaft: Partition 0 → Replicas=[1], Leader=1 ← may be offline/stale + Transformer: Ignores KRaft, hashes to any alive broker +``` + +This was a deliberate decision that enabled: +- Fast iteration without controller changes +- Simple implementation +- "Always available" semantics (transformer routes around any failure) + +**This has worked well for pure diskless topics.** + +### The Question: What Does Topic Migration Require? + +We need to support **bidirectional topic migration**: +1. **Classic → Diskless** (immediate need) +2. **Diskless → Classic** (future need) + +The key question: **how does each approach handle both directions?** + +We evaluate three approaches (A, B, C) below, starting with reverse migration requirements. The proposed design (Approach C) is then detailed in the rest of this document. + +### Approach Comparison + +**Terminology:** +- **Offline replica**: KRaft shows replica on broker X, but broker X doesn't exist or is down. No `Partition` object. +- **Ghost replica**: KRaft shows replica on broker X, broker X is alive, but `ReplicaManager` skips creating `Partition` object for diskless. Broker doesn't know it's a replica. +- **Real (metadata-only) replica**: KRaft shows replica on broker X, broker X recognizes it as a replica, but no local log (data in object storage). + +#### Summary Table + +| Aspect | A: Shrink to RF=1 | B: Keep RF=3, fake | C: RF=rack_count (proposed) | +|-------------------------|--------------------------|--------------------------|------------------------------| +| KRaft RF after migrate | 1 | 3 (unchanged) | 3 (managed, rack-aware) | +| Transformer behavior | Hash to any broker | Hash to any broker | Filter by AZ (KRaft-sourced) | +| Replica objects | None (offline in KRaft) | Ghost (no `Partition`) | Real (metadata-only) | +| Migration coordinator | Custom (who?) | Custom (who?) | **Leader** | +| RLM integration | Custom hooks (~11 wks) | Custom hooks (~11 wks) | **Standard (~0 wks)** | +| Job assignment | Racing | Racing | **Leader-based** | +| Reverse migration ready | No (must expand RF) | No (must fix ghosts) | **Yes** | + +#### Approach A: Shrink to RF=1 During Migration + +``` +Classic: Replicas=[1,2,3], Leader=1, data on all brokers + ↓ migration starts +Shrink: Replicas=[1], Leader=1 ← When? Who coordinates? + ↓ +Diskless: Replicas=[1], data in object storage +``` + +**Problems:** +- Timing of RF shrink is complex (before seal? during? after?) +- Single point of failure during migration +- Loses rack-awareness — original placement lost, must recreate for reverse migration +- **Reverse migration requires RF expansion first** (two operations) + +#### Approach B: Keep Classic RF=3, Continue Faking + +``` +Classic: Replicas=[1,2,3], Leader=1, data on all brokers + ↓ migration starts (no RF change!) +Diskless: Replicas=[1,2,3], Leader=1, data in object storage + ↑ KRaft unchanged, transformer ignores and hashes +``` + +What "ghost replica" means: +- KRaft says broker is a replica with ISR membership +- But `ReplicaManager.isDisklessTopic()` skips `getOrCreatePartition()` +- No `Partition` object, no fetcher, no local log +- Broker doesn't know it's a "replica" — just serves requests via Inkless handlers +- ISR never shrinks (no lag tracking for diskless) + +**Benefit:** If users migrate back to classic, they keep the same partition assignments. + +**Problems:** +- **RLM still blocked** — `onLeadershipChange()` needs `Partition` objects to run tiering tasks. Ghost replicas don't have them. Adding conditional `Partition` creation for diskless would be similar complexity to the proposed approach. +- **Reverse migration requires "un-ghosting"** — must create real `Partition` objects before migrating back +- Misleading tooling output (`describe topic` shows healthy replicas that aren't real) — operational burden on operators +- See [Ghost Replicas Problem](#ghost-replicas-problem) + +#### Approach C: RF=rack_count with Real Replicas (Proposed) + +``` +Classic: Replicas=[1,2,3], Leader=1, data on all brokers + ↓ migration starts +Diskless: Replicas=[1,3,5], Leader=1, data in object storage + ↑ rack-aware placement, real metadata-only replicas + Leader coordinates migration, RLM tasks, background jobs +``` + +What "real (metadata-only) replica" means: +- KRaft says broker is a replica with ISR membership +- Broker recognizes it's a replica for this partition +- No local log (data in object storage) — but could have `Partition` object for coordination +- Leader can run RLM tasks, coordinate migration, own background jobs + +**Benefits:** +- **Leader coordinates migration** — deterministic owner +- **Standard RLM integration** — `onLeadershipChange()` works +- **Deterministic job assignment** — leader owns partition operations +- **Accurate tooling output** — replicas are real +- **Reverse migration ready** — already has RF=3 with real replicas + +### Diskless → Classic Migration Readiness + +| Aspect | A: RF=1 | B: RF=3 faked | C: RF=rack_count | +|-------------------|-----------------------|-------------------------|-------------------------| +| Starting RF | 1 | 3 (ghost replicas) | 3 (real replicas) | +| Before migration | Must expand RF=1 → RF=3| Must "un-ghost" replicas| **Ready as-is** | +| Replica state | Create from scratch | Create `Partition` objects| Already correct | +| Complexity | High (2 operations) | Medium (fix ghosts) | **Low (just migrate)** | + +**Key insight:** Only Approach C is ready for reverse migration without additional work. + +### Cost Analysis + +#### Cost of Keeping RF=1/Faked (Approaches A and B) + +| Aspect | Custom Solution Required | Effort | +|---------------------------|-----------------------------------|-----------| +| Tiering entry point | Bypass `onLeadershipChange()` | ~2 weeks | +| Leader coordination | Custom leader election for tiering| ~2 weeks | +| Segment merging | Custom coordination | ~3 weeks | +| RLM API compatibility | Fake `Partition` objects or fork | ~2 weeks | +| Expiration tasks | Custom expiration outside RLM | ~2 weeks | +| **Total custom work** | | **~11 weeks** | + +#### Cost of Proposed Design (Approach C: RF=rack_count) + +| Aspect | Standard Solution | Effort | +|---------------------------|-----------------------------------|-----------| +| Rack-aware placement | Modify `ReplicationControlManager`| ~2 weeks | +| Transformer filtering | Update `InklessTopicMetadataTransformer` | ~2 weeks | +| Add partitions | Same logic as creation | ~1 week | +| RLM integration | Standard `onLeadershipChange()` | ~0 weeks* | +| **Total work** | | **~6 weeks** | + +*RLM integration comes "for free" once we have real leadership. + +#### Decision Framework + +| Factor | A: Shrink to RF=1 | B: Keep RF=3, fake | C: RF=rack_count (proposed) | +|--------------------------|--------------------------|--------------------------|------------------------------| +| Short-term cost | Lower | Lower | ~6 weeks | +| Classic → Diskless | Custom coordination | Custom coordination | Standard replica handoff | +| RLM integration | Custom hooks (~11 wks) | Custom hooks (~11 wks) | Standard (~0 wks) | +| Diskless → Classic | RF expansion + migration | Un-ghost + migration | Migration only | +| Long-term maintenance | Two code paths | Two code paths | One code path | +| Kafka alignment | Divergent | Divergent | Aligned | + +#### Summary + +``` +Approach A/B: 0 weeks now + ~11 weeks later = 11+ weeks total +Approach C: 6 weeks now + 0 weeks later = 6 weeks total + ───────────── + Savings: 5+ weeks +``` + +Plus, deferring creates technical debt: +- Two code paths for leadership (KRaft vs. custom) +- Custom tiering pipeline that diverges from Kafka +- Every new feature asks "does this work with faked RF?" + +### Recommendation + +**Implement the proposed design (Approach C: RF=rack_count)** because: +1. Topic migration (both directions) benefits from real replicas +2. RLM integration becomes standard rather than custom +3. One-time 6-week investment vs. 11+ weeks of custom work +4. Avoids accumulating technical debt +5. Aligns diskless with Kafka's replica model + +### Related Documents + +- [DESIGN.md](DESIGN.md) — Overall tiered storage unification design + +--- + +## Objectives + +Enable **rack-aware, stable KRaft-managed replicas** for diskless topics: + +- **KRaft-managed replicas**: Diskless topics will have actual KRaft-managed replicas (RF = rack count) +- **Rack-aware at creation**: Topics are created with one replica per rack +- **Standard operations after creation**: Once created, standard Kafka replica management applies +- **Controller-managed RF**: Users don't specify RF; controller computes from rack topology +- **Leader-agnostic produce and consume**: Any replica can accept writes and serve reads +- **Always available via transformer**: Transformer ensures availability by routing to alive brokers + +--- + +## Design: Transformer-First Availability + +### The Insight + +For diskless topics, **availability** and **metadata accuracy** can be decoupled: + +| Concern | Priority | Who Handles | Speed | +|-------------------|--------------|-------------------|------------| +| Availability | Critical | Transformer | Instant | +| Metadata accuracy | Nice-to-have | Controller (lazy) | Eventually | + +### How It Works + +**Current diskless behavior (RF=1, faked):** +- Transformer hashes to any alive broker +- Partition is always available +- KRaft metadata is ignored for routing + +**Proposed behavior (RF=rack_count, real replicas):** +- KRaft stores RF=rack_count with real broker IDs +- Transformer filters by client AZ from KRaft replicas +- **If assigned replica is offline → fall back to any alive broker in AZ** +- Partition is always available (same as today!) +- KRaft metadata may be temporarily stale (shows offline broker) + +### Why This Works for Diskless + +``` +Classic topic: + Data on broker X → client MUST talk to broker X + Broker X offline → partition unavailable (until reassigned) + +Diskless topic: + Data in object storage → ANY broker can serve + Broker X offline → transformer routes to broker Y + Partition stays available (no reassignment needed!) +``` + +The transformer already has all the information it needs: +1. KRaft replica assignments (preferred brokers) +2. Alive brokers in cluster +3. Client AZ + +It can make instant routing decisions without waiting for controller. + +--- + +## Activation Model (Binary Version) + +Managed RF is activated by **deploying a new binary version**. No feature flags, no topic attributes, no server configs. + +### Behavior Summary + +**Old binary:** +- Diskless topics use legacy "RF=1 / faked metadata" behavior +- Transformer calculates synthetic placement via hashing + +**New binary:** +- Diskless topics use KRaft-managed placement (one replica per rack at creation) +- Transformer filters KRaft placement by client AZ +- **Falls back to alive brokers if assigned replicas are offline** + +### Existing Diskless Topics + +Existing diskless topics (RF=1) continue to work: + +**Immediate availability:** +- Transformer sees RF=1 with potentially offline broker +- Falls back to hash-based selection (same as today) +- No downtime + +**Eventual modernization (optional):** +- Controller can lazily detect RF=1 diskless topics +- Background task expands to RF=rack_count when convenient +- Not urgent — availability is already handled by transformer + +**Manual modernization (alternative):** +- Operator uses `kafka-reassign-partitions.sh` to expand RF +- Same tooling as classic topics + +--- + +## Placement Model + +### Rack-Aware Placement at Creation + +When a diskless topic is created: +- Controller determines current rack count from registered brokers +- RF is set to rack count (e.g., 3 racks → RF=3) +- One broker is selected per rack for each partition +- Broker selection within a rack uses load balancing (least loaded broker) + +### Placement Is Static After Creation + +Once a topic is created: +- RF does not automatically change if racks are added/removed +- Replica assignments don't automatically adjust +- Standard Kafka replica management applies + +--- + +## Controller Behavior + +### Topic Creation + +When creating diskless topics (`diskless.enable=true`): +- Controller counts distinct racks from registered brokers +- RF = rack count +- One replica assigned per rack +- Reject `replicationFactor > 1` and manual assignments + +### Add Partitions + +New partitions use same one-per-rack logic as creation. + +### Standard Operations (After Creation) + +**No auto-reassignment on broker failure.** Transformer handles availability. + +**Leader Election:** +- Standard Kafka leader election from ISR +- For diskless topics, all replicas are always in ISR + +**Broker Failure:** +- Replicas on failed broker become offline in KRaft metadata +- **Transformer routes around offline broker** (instant) +- Partition remains available +- KRaft shows under-replicated (informational only for diskless) +- When broker returns, it's back in ISR immediately + +**Reassignment:** +- `kafka-reassign-partitions.sh` works normally +- Operator can reassign replicas if desired +- **Not required for availability** — transformer handles routing + +--- + +## Broker Behavior + +- No replica fetcher threads for diskless topics +- No local `Partition` objects (for now — may add for RLM coordination later) +- Inkless handlers (`AppendHandler`, `FetchHandler`) serve requests + +--- + +## Metadata Transformation + +### Filtering Logic + +``` +FOR each diskless partition: + assigned_replicas = KRaft replica set + alive_replicas = assigned_replicas ∩ alive_brokers + + IF alive_replicas is not empty: + # Normal case: use KRaft placement + IF any alive_replica in clientAZ: + RETURN local replica (AZ-aware routing) + ELSE: + RETURN all alive_replicas (cross-AZ fallback) + ELSE: + # All assigned replicas offline: fall back to hash + RETURN hash-based selection from all alive brokers in clientAZ +``` + +### Key Properties + +1. **Instant availability**: No waiting for controller +2. **AZ-aware when possible**: Uses KRaft placement if alive +3. **Graceful degradation**: Falls back to hash if needed +4. **No metadata dependency for availability**: Stale metadata doesn't cause downtime + +--- + +## Observability + +### Metrics + +**Controller metrics:** +- `kafka.controller.diskless.effective_rf{topic}` - RF assigned at creation +- `kafka.controller.diskless.rack_aware{topic,partition}` - 1 if one-per-rack, 0 if not +- `kafka.controller.diskless.rf1_topics_total` - Legacy topics not yet modernized + +**Transformer metrics:** +- `kafka.server.diskless.transformer.fallback_total` - Count of hash fallbacks +- `kafka.server.diskless.transformer.offline_replicas_routed_around` - Routing decisions + +**Standard Kafka metrics:** +- `UnderReplicatedPartitions` - Will show diskless partitions with offline brokers +- Note: For diskless, "under-replicated" is informational, not critical + +### Logs + +``` +INFO [Transformer] Diskless partition 'foo-0' has offline replica (broker 101), + routing to alive replica (broker 103) + +WARN [Transformer] Diskless partition 'foo-0' has ALL replicas offline, + using hash-based fallback to broker 105 +``` + +### Operational Notes + +**`kafka-topics.sh --describe` may show stale info:** +``` +Topic: foo PartitionCount: 3 ReplicationFactor: 3 + Partition: 0 Leader: 101 Replicas: 101,103,105 Isr: 101,103,105 + ↑ + Broker 101 may be offline, but transformer routes to 103 +``` + +This is acceptable because: +1. Availability is not affected +2. Operator can see actual state via broker metrics +3. Eventually consistent (when broker returns or is reassigned) + +--- + +## Implementation Path + +**Total Estimate: 6 weeks with 1 engineer, or 4 weeks with 2 engineers** + +### Phase 0: Research and Validation (1 week) + +1. Leader election works without broker `Partition` objects +2. ISR updates don't require broker-side `Partition` state +3. `DescribeTopics` / `ListOffsets` work with RF > 1 + +### Phase 1: Topic Creation with Rack-Aware Placement (2 weeks) + +1. Modify `ReplicationControlManager` to detect diskless topics +2. Compute RF = rack count at creation +3. Implement one-per-rack broker selection +4. Reject `replicationFactor > 1` and manual assignments + +### Phase 2: Transformer Changes (2 weeks) + +1. Update `InklessTopicMetadataTransformer` to read KRaft placement +2. Implement AZ filtering logic +3. Add offline replica routing — if KRaft replica offline, route to alive replica +4. Add hash fallback — if all replicas offline, use legacy hash +5. Add metrics for fallback tracking + +### Phase 3: Add Partitions Support (1 week) + +1. Apply same one-per-rack logic when adding partitions +2. Handle case where rack count changed since topic creation + +### Summary + +| Phase | Scope | Estimate | +|-----------|------------------------------------------|----------| +| 0 | Research and Validation | 1 week | +| 1 | Topic Creation with Rack-Aware Placement | 2 weeks | +| 2 | Transformer Changes (with fallback) | 2 weeks | +| 3 | Add Partitions Support | 1 week | +| **Total** | | **6 weeks (1 eng) / 4 weeks (2 eng)** | + +--- + +## Rejected Alternatives + +### Alternative A: Shrink RF During Migration + +**Concept:** When migrating Classic → Diskless, shrink RF=3 to RF=1, then use current diskless model. + +**Why considered:** +- Maintains consistency with current RF=1 diskless model +- No need to change existing diskless implementation + +**Why rejected:** +1. Timing complexity (when does RF shrink happen?) +2. Loses rack-awareness — must recreate for reverse migration +3. **Reverse migration requires RF expansion** (two operations) +4. RLM integration still blocked +5. Racing job assignment continues + +See [Approach Comparison](#approach-comparison) for details. + +--- + +### Alternative B: Keep Classic RF, Continue Faking Metadata + +**Concept:** Keep existing RF (e.g., RF=3) in KRaft but continue using transformer to fake metadata. + +**Why considered:** +- Zero controller changes for migration +- Simplest migration path +- If users migrate back, they keep same partition assignments + +**Why rejected:** +1. **Ghost replicas** — KRaft shows healthy replicas that aren't real +2. **RLM blocked** — `onLeadershipChange()` needs `Partition` objects. Adding them would be similar complexity to proposed approach. +3. **Reverse migration requires "un-ghosting"** replicas +4. Misleading tooling output +5. Racing job assignment continues + +See [Ghost Replicas Problem](#ghost-replicas-problem) for details. + +--- + +### Alternative C: Controller Auto-Reassignment + +**Concept:** Controller proactively detects offline replicas and reassigns to online brokers immediately. + +**Why considered:** +- KRaft metadata always accurate +- Standard tooling shows correct state + +**Why rejected:** +- **3 weeks additional complexity** for something transformer already handles +- **Instant vs eventually consistent** — transformer is faster +- **Consistent with today** — diskless already uses transformer for availability +- **Fewer failure modes** — no controller state machine to debug + +**When to reconsider:** +- If KRaft metadata accuracy becomes critical requirement +- If operators require `describe topic` to always reflect reality + +--- + +### Alternative D: Custom Job Coordinator + +**Concept:** Keep faking metadata, add custom coordinator (outside KRaft) for background jobs. + +**Why considered:** +- Gets deterministic job ownership without changing replica model +- Could use PG-based leader election + +**Why rejected:** +1. **Two leadership systems** — KRaft vs. custom coordinator +2. **RLM still blocked** — RLM uses KRaft leadership +3. **More complexity** — three systems (KRaft + transformer + coordinator) +4. **Defeats purpose** — if adding coordinator, why not use KRaft's? + +--- + +## Appendix + +### Ghost Replicas Problem + +If we keep RF=3 in KRaft but transformer ignores it, Kafka creates "ghost replicas": + +**Current diskless behavior in `ReplicaManager.scala`:** + +```scala +// applyLocalLeadersDelta - when broker becomes leader +localLeaders.foreachEntry { (tp, info) => + if (!_inklessMetadataView.isDisklessTopic(tp.topic())) + getOrCreatePartition(tp, delta, info.topicId) // SKIPPED for diskless! +} +``` + +**What this means:** + +| Aspect | Classic Topic | Diskless Topic (ghost) | +|-------------------------|----------------------------|------------------------------| +| `Partition` objects | Created on each replica | **Never created** | +| Replica fetcher threads | Started for followers | **Never started** | +| ISR tracking | Leader tracks follower lag | **Skipped** (no lag) | + +**The problem:** + +```bash +$ kafka-topics.sh --describe --topic diskless-foo + +Topic: diskless-foo Replicas: 1,2,3 ISR: 1,2,3 + ↑ + Looks healthy! + But brokers 2,3 don't know they're replicas +``` + +**Operational risks:** + +| Risk | Impact | +|------------------------------|-------------------------------------------| +| False sense of redundancy | Operator thinks RF=3 provides safety | +| Confusing metrics | Under-replicated = 0, but replicas unused | +| Tooling mismatch | `describe` shows healthy, reality differs | +| Debugging difficulty | "Why isn't replication working?" | + +This creates technical debt and operational confusion that compounds over time — essentially throwing problems at operators. + +--- + +### Operational Benefits Summary + +The proposed design (Approach C) provides clear operational benefits compared to alternatives: + +| Aspect | Approaches A/B (faking) | Approach C (proposed) | +|---------------------------|--------------------------------|--------------------------------| +| Tooling accuracy | `describe topic` misleading | `describe topic` accurate | +| Under-replicated alerts | False positives | Meaningful (informational) | +| Job debugging | "Which broker ran this?" | "Leader of partition X" | +| Incident response | Check all broker logs | Check leader broker | +| Capacity planning | Unpredictable load | Proportional to leadership | +| Standard Kafka ops | May not work as expected | Work normally | + +--- + +### Leadership Benefits for Operations + +Current diskless uses **racing/randomized** job assignment. Leadership provides: + +| Benefit | Racing Model | Leader Model | +|-------------------------|----------------------------|-------------------------------| +| Accountability | "Which broker ran this?" | "Leader of partition X" | +| Load distribution | Random, unpredictable | Proportional to leadership | +| Capacity planning | Add brokers, hope it helps | Rebalance partition leaders | +| Incident response | Check all broker logs | Check leader broker | +| Tooling | Custom | Standard Kafka tooling | + +Leadership isn't just about RLM — it's about having a **deterministic owner** for partition-level operations. From 4593e7a29f1f22eb7f309268a73d1edfe2410785 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Wed, 14 Jan 2026 11:16:31 +0200 Subject: [PATCH 07/21] fixup! add diskless managed rf feature design --- .../ts-unification/DISKLESS_MANAGED_RF.md | 1246 ++++++----------- .../DISKLESS_MANAGED_RF_FINAL.md | 635 --------- .../DISKLESS_MANAGED_RF_SIMPLIFIED.md | 1109 --------------- 3 files changed, 402 insertions(+), 2588 deletions(-) delete mode 100644 docs/inkless/ts-unification/DISKLESS_MANAGED_RF_FINAL.md delete mode 100644 docs/inkless/ts-unification/DISKLESS_MANAGED_RF_SIMPLIFIED.md diff --git a/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md b/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md index 6840445c54..8f65d14f20 100644 --- a/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md +++ b/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md @@ -1,34 +1,22 @@ # Diskless-Managed Replication Factor -> **Alternative design under review:** [DISKLESS_MANAGED_RF_SIMPLIFIED.md](DISKLESS_MANAGED_RF_SIMPLIFIED.md) proposes a simpler approach using transformer-first availability instead of controller auto-reassignment. Review both before deciding. - ## Table of Contents 1. [Purpose](#purpose) 2. [Objectives](#objectives) -3. [Activation Model](#activation-model-binary-version) - - [Existing Diskless Topics](#existing-diskless-topics) -4. [Placement Model](#placement-model) -5. [Controller Behavior](#controller-behavior) - - [Topic Creation](#topic-creation) - - [Add Partitions](#add-partitions) - - [Standard Operations](#standard-operations-after-creation) -6. [Broker Behavior](#broker-behavior) - - [No Replica Fetcher for Diskless Topics](#no-replica-fetcher-for-diskless-topics) -7. [Metadata Transformation](#metadata-transformation) -8. [Observability](#observability) -9. [Implementation Path](#implementation-path) *(9 weeks / 1 eng, 5-6 weeks / 2 eng)* - - [Phase 0: Research and Validation](#phase-0-research-and-validation) - - [Phase 1: Topic Creation with Rack-Aware Placement](#phase-1-topic-creation-with-rack-aware-placement) - - [Phase 2: Transformer Changes](#phase-2-transformer-changes) - - [Phase 3: Add Partitions Support](#phase-3-add-partitions-support) - - [Phase 4: Offline Replica Auto-Reassignment](#phase-4-offline-replica-auto-reassignment) - - [Phase 5: Observability](#phase-5-observability) - - [Testing Strategy](#testing-strategy) - - [Future Work: RLM Integration](#future-work-rlm-integration) -10. [Rejected Alternatives](#rejected-alternatives) -11. [Appendix: Topic Migration Interactions](#appendix-topic-migration-interactions) -12. [Open Items](#open-items) +3. [Design: Transformer-First Availability](#design-transformer-first-availability) +4. [Activation Model](#activation-model-binary-version) +5. [Placement Model](#placement-model) +6. [Controller Behavior](#controller-behavior) +7. [Broker Behavior](#broker-behavior) +8. [Metadata Transformation](#metadata-transformation) +9. [Observability](#observability) +10. [Implementation Path](#implementation-path) *(6 weeks / 1 eng, 4 weeks / 2 eng)* +11. [Rejected Alternatives](#rejected-alternatives) +12. [Appendix](#appendix) + - [Ghost Replicas Problem](#ghost-replicas-problem) + - [Operational Benefits Summary](#operational-benefits-summary) + - [Leadership Benefits for Operations](#leadership-benefits-for-operations) --- @@ -36,10 +24,18 @@ ### Current State: RF=1 with Transformer Override -Current diskless topics use **RF=1 in KRaft** with the transformer routing requests to any alive broker. This was a deliberate decision that enabled: +Current diskless topics use **RF=1 in KRaft** with the transformer routing requests to any alive broker. The single replica may point to an **offline or non-existent broker** — the transformer ignores KRaft metadata entirely and hashes to any alive broker. + +``` +Current diskless: + KRaft: Partition 0 → Replicas=[1], Leader=1 ← may be offline/stale + Transformer: Ignores KRaft, hashes to any alive broker +``` + +This was a deliberate decision that enabled: - Fast iteration without controller changes - Simple implementation -- Flexibility in routing +- "Always available" semantics (transformer routes around any failure) **This has worked well for pure diskless topics.** @@ -49,269 +45,260 @@ We need to support **bidirectional topic migration**: 1. **Classic → Diskless** (immediate need) 2. **Diskless → Classic** (future need) -### Why Real Replicas Matter - -| Capability | RF=1 (current) | RF=rack_count (this design) | -|-------------------------------|-----------------------------|-----------------------------| -| Topic migration coordination | Custom coordinator needed | Leader coordinates | -| RLM integration | Custom hooks (~11 weeks) | Standard (~0 weeks) | -| Background job assignment | Racing/randomized | Deterministic (leader) | -| Diskless → Classic migration | RF expansion + migration | Migration only | -| Operational debugging | "Which broker ran this?" | "Leader of partition X" | - -### What This Feature Enables - -| Capability | How | Status | -|---------------------------|----------------------------------------|----------------| -| Rack-aware placement | One replica per AZ at creation | ✅ This design | -| Topic migration | Real replicas to migrate to | ✅ This design | -| RLM integration | Stable partition leader | 🔜 Future | -| Standard Kafka operations | Reassignment, election work normally | ✅ This design | -| Deterministic job owner | Leader owns partition operations | ✅ This design | - -### Related Documents - -- [DESIGN.md](DESIGN.md) — Overall tiered storage unification design -- [DISKLESS_MANAGED_RF_SIMPLIFIED.md](DISKLESS_MANAGED_RF_SIMPLIFIED.md) — Simplified variant (transformer-first availability) +The key question: **how does each approach handle both directions?** ---- +We evaluate three approaches (A, B, C) below, starting with reverse migration requirements. The proposed design (Approach C) is then detailed in the rest of this document. -## Objectives +### Approach Comparison -Enable **rack-aware, stable KRaft-managed replicas** for diskless topics: +**Terminology:** +- **Offline replica**: KRaft shows replica on broker X, but broker X doesn't exist or is down. No `Partition` object. +- **Ghost replica**: KRaft shows replica on broker X, broker X is alive, but `ReplicaManager` skips creating `Partition` object for diskless. Broker doesn't know it's a replica. +- **Real (metadata-only) replica**: KRaft shows replica on broker X, broker X recognizes it as a replica, but no local log (data in object storage). -- **KRaft-managed replicas**: Diskless topics will have actual KRaft-managed replicas, not the current "faked" RF=1 metadata. These replicas are tracked by the controller, participate in leader election, and appear in standard Kafka tooling. The key difference from classic replicas: **they don't replicate data** (object storage handles durability), but they do provide partition assignments for RLM integration and operational consistency. See [No Replica Fetcher for Diskless Topics](#no-replica-fetcher-for-diskless-topics) for implementation details. -- **Rack-aware at creation**: Topics are created with one replica per rack (RF = rack count) -- **Standard operations after creation**: Once created, diskless topics use standard Kafka replica management. The only automatic action is **offline replica reassignment** to preserve availability (see [Existing Diskless Topics](#existing-diskless-topics)). -- **Controller-managed RF**: Users don't specify RF at creation; the controller computes it from rack topology. Requests with `replicationFactor=-1` or `replicationFactor=1` are accepted for compatibility (both result in RF=rack_count). -- **Leader-agnostic produce and consume**: Keep the diskless model where any replica can accept writes and serve reads, while still having a KRaft leader for controller duties (e.g., future RLM coordination) +#### Summary Table -**Non-goal:** -Topic migration mechanics (Tiered Classic → Diskless), including sealing and offset boundary tracking. -Those are covered in [DESIGN.md](DESIGN.md). +| Aspect | A: Shrink to RF=1 | B: Keep RF=3, fake | C: RF=rack_count (proposed) | +|-------------------------|--------------------------|--------------------------|------------------------------| +| KRaft RF after migrate | 1 | 3 (unchanged) | 3 (managed, rack-aware) | +| Transformer behavior | Hash to any broker | Hash to any broker | Filter by AZ (KRaft-sourced) | +| Replica objects | None (offline in KRaft) | Ghost (no `Partition`) | Real (metadata-only) | +| Migration coordinator | Custom (who?) | Custom (who?) | **Leader** | +| RLM integration | Custom hooks (~11 wks) | Custom hooks (~11 wks) | **Standard (~0 wks)** | +| Job assignment | Racing | Racing | **Leader-based** | +| Reverse migration ready | No (must expand RF) | No (must fix ghosts) | **Yes** | ---- +#### Approach A: Shrink to RF=1 During Migration -## Activation Model (Binary Version) +``` +Classic: Replicas=[1,2,3], Leader=1, data on all brokers + ↓ migration starts +Shrink: Replicas=[1], Leader=1 ← When? Who coordinates? + ↓ +Diskless: Replicas=[1], data in object storage +``` -Managed RF is activated by **deploying a new binary version**. No feature flags, no topic attributes, no server configs. See [Rejected Alternative: Feature Flag Activation](#rejected-alternative-feature-flag-activation) for why we chose this simpler approach. +**Problems:** +- Timing of RF shrink is complex (before seal? during? after?) +- Single point of failure during migration +- Loses rack-awareness — original placement lost, must recreate for reverse migration +- **Reverse migration requires RF expansion first** (two operations) -### Behavior Summary - -**Old binary:** -- Diskless topics use legacy "RF=1 / faked metadata" behavior -- `InklessTopicMetadataTransformer` calculates synthetic placement via hashing +#### Approach B: Keep Classic RF=3, Continue Faking -**New binary:** -- Diskless topics use KRaft-managed placement (one replica per rack at creation) -- Transformer filters KRaft placement by client AZ instead of calculating +``` +Classic: Replicas=[1,2,3], Leader=1, data on all brokers + ↓ migration starts (no RF change!) +Diskless: Replicas=[1,2,3], Leader=1, data in object storage + ↑ KRaft unchanged, transformer ignores and hashes +``` -### How It Works +What "ghost replica" means: +- KRaft says broker is a replica with ISR membership +- But `ReplicaManager.isDisklessTopic()` skips `getOrCreatePartition()` +- No `Partition` object, no fetcher, no local log +- Broker doesn't know it's a "replica" — just serves requests via Inkless handlers +- ISR never shrinks (no lag tracking for diskless) -1. Controller detects diskless topics via existing `diskless.enable=true` topic config -2. At topic creation, controller computes RF = rack count and places one replica per rack -3. No new KRaft records needed; existing topic and partition records are sufficient -4. After creation, standard Kafka replica management applies +**Benefit:** If users migrate back to classic, they keep the same partition assignments. -### Mixed Binary Compatibility +**Problems:** +- **RLM still blocked** — `onLeadershipChange()` needs `Partition` objects to run tiering tasks. Ghost replicas don't have them. Adding conditional `Partition` creation for diskless would be similar complexity to the proposed approach. +- **Reverse migration requires "un-ghosting"** — must create real `Partition` objects before migrating back +- Misleading tooling output (`describe topic` shows healthy replicas that aren't real) — operational burden on operators +- See [Ghost Replicas Problem](#ghost-replicas-problem) -If a new controller updates partition metadata (e.g., RF=3 with replicas in multiple racks) while some brokers still run the old binary: +#### Approach C: RF=rack_count with Real Replicas (Proposed) -- **Data plane is unaffected**: Produce and consume APIs are not modified. Old brokers continue serving diskless requests via existing `AppendHandler`/`FetchHandler`. -- **Metadata plane differs**: Old brokers use transformer hashing, new brokers filter KRaft metadata. Clients may see different leader/replicas depending on which broker they query. -- **No correctness issues**: Diskless data lives in object storage. Any broker can serve any partition regardless of what metadata says. +``` +Classic: Replicas=[1,2,3], Leader=1, data on all brokers + ↓ migration starts +Diskless: Replicas=[1,3,5], Leader=1, data in object storage + ↑ rack-aware placement, real metadata-only replicas + Leader coordinates migration, RLM tasks, background jobs +``` -This temporary inconsistency resolves once all brokers run the new binary. In our deployment model (new VMs), this period is brief or non-existent. +What "real (metadata-only) replica" means: +- KRaft says broker is a replica with ISR membership +- Broker recognizes it's a replica for this partition +- No local log (data in object storage) — but could have `Partition` object for coordination +- Leader can run RLM tasks, coordinate migration, own background jobs -### Cluster Migration +**Benefits:** +- **Leader coordinates migration** — deterministic owner +- **Standard RLM integration** — `onLeadershipChange()` works +- **Deterministic job assignment** — leader owns partition operations +- **Accurate tooling output** — replicas are real +- **Reverse migration ready** — already has RF=3 with real replicas -In our deployment model, new binaries are deployed on **new VMs with new broker IDs** (not in-place rolling upgrade). This simplifies the transition: +### Diskless → Classic Migration Readiness -**Before migration:** -- Old cluster with old broker IDs -- Diskless topics have RF=1 pointing to old broker IDs -- Transformer uses hash-based leader selection +| Aspect | A: RF=1 | B: RF=3 faked | C: RF=rack_count | +|-------------------|-----------------------|-------------------------|-------------------------| +| Starting RF | 1 | 3 (ghost replicas) | 3 (real replicas) | +| Before migration | Must expand RF=1 → RF=3| Must "un-ghost" replicas| **Ready as-is** | +| Replica state | Create from scratch | Create `Partition` objects| Already correct | +| Complexity | High (2 operations) | Medium (fix ghosts) | **Low (just migrate)** | -**After migration:** -- New cluster with new broker IDs -- Controller detects orphaned replicas (old broker IDs not in cluster) -- Auto-migration expands RF to rack_count with new broker IDs -- Transformer filters KRaft metadata by client AZ +**Key insight:** Only Approach C is ready for reverse migration without additional work. -**Client behavior:** -- Clients reconnect to new cluster via bootstrap servers -- Metadata requests return new broker IDs with rack-aware placement -- No inconsistency period (unlike rolling upgrade) +### Cost Analysis -### Existing Diskless Topics +#### Cost of Keeping RF=1/Faked (Approaches A and B) -Existing diskless topics created before this feature have RF=1 in KRaft. These need to be expanded to RF=rack_count. +| Aspect | Custom Solution Required | Effort | +|---------------------------|-----------------------------------|-----------| +| Tiering entry point | Bypass `onLeadershipChange()` | ~2 weeks | +| Leader coordination | Custom leader election for tiering| ~2 weeks | +| Segment merging | Custom coordination | ~3 weeks | +| RLM API compatibility | Fake `Partition` objects or fork | ~2 weeks | +| Expiration tasks | Custom expiration outside RLM | ~2 weeks | +| **Total custom work** | | **~11 weeks** | -**Migration approach: Automatic migration for orphaned replicas** +#### Cost of Proposed Design (Approach C: RF=rack_count) -Legacy diskless topics were created with RF=1 pointing to a broker that existed at creation time. In our deployment model, **new binaries are deployed on new VMs with new broker IDs**. This means legacy topics will have broker IDs that no longer exist in the cluster, allowing safe automatic detection. +| Aspect | Standard Solution | Effort | +|---------------------------|-----------------------------------|-----------| +| Rack-aware placement | Modify `ReplicationControlManager`| ~2 weeks | +| Transformer filtering | Update `InklessTopicMetadataTransformer` | ~2 weeks | +| Add partitions | Same logic as creation | ~1 week | +| RLM integration | Standard `onLeadershipChange()` | ~0 weeks* | +| **Total work** | | **~6 weeks** | -#### Deployment Model Assumption +*RLM integration comes "for free" once we have real leadership. -``` -Old cluster (legacy diskless topics created here): - Brokers: 1, 2, 3, 4, 5, 6 +#### Decision Framework -New cluster (after infrastructure migration): - Brokers: 101, 102, 103, 104, 105, 106 ← new VMs, new IDs - -Legacy topic in KRaft: - Topic: foo, Partition 0, Replicas=[3] ← broker 3 no longer exists -``` +| Factor | A: Shrink to RF=1 | B: Keep RF=3, fake | C: RF=rack_count (proposed) | +|--------------------------|--------------------------|--------------------------|------------------------------| +| Short-term cost | Lower | Lower | ~6 weeks | +| Classic → Diskless | Custom coordination | Custom coordination | Standard replica handoff | +| RLM integration | Custom hooks (~11 wks) | Custom hooks (~11 wks) | Standard (~0 wks) | +| Diskless → Classic | RF expansion + migration | Un-ghost + migration | Migration only | +| Long-term maintenance | Two code paths | Two code paths | One code path | +| Kafka alignment | Divergent | Divergent | Aligned | -#### Detection Logic +#### Summary ``` -IF diskless.enable=true - AND ANY replica broker is offline -THEN: - IF broker ID is not registered in cluster metadata: - # Legacy topic from old cluster - Reassign ALL replicas, expand to RF=rack_count - ELSE: - # Broker exists but offline (failure/maintenance) - Reassign ONLY offline replica(s), preserve current RF +Approach A/B: 0 weeks now + ~11 weeks later = 11+ weeks total +Approach C: 6 weeks now + 0 weeks later = 6 weeks total + ───────────── + Savings: 5+ weeks ``` -**Key distinction:** -- **Not registered:** Broker ID was never seen by this cluster (e.g., old cluster had brokers 1-6, new cluster has 101-106). This is a legacy topic that should be modernized. -- **Registered but offline:** Broker ID exists in cluster metadata but is currently unavailable. Operator chose this RF intentionally, so preserve it. - -#### Why This Is Safe for Diskless +Plus, deferring creates technical debt: +- Two code paths for leadership (KRaft vs. custom) +- Custom tiering pipeline that diverges from Kafka +- Every new feature asks "does this work with faked RF?" -Auto-reassigning offline replicas is safe for diskless topics because: +### Recommendation -| Property | Classic Topics | Diskless Topics | -|----------|----------------|-----------------| -| Data location | Local broker disk | Object storage (shared) | -| Reassignment cost | Data copy (slow) | Metadata only (instant) | -| Data loss risk | Yes (if out of sync) | No (all brokers see same data) | -| ISR meaning | Replication lag tracking | Meaningless (all always in sync) | +**Implement the proposed design (Approach C: RF=rack_count)** because: +1. Topic migration (both directions) benefits from real replicas +2. RLM integration becomes standard rather than custom +3. One-time 6-week investment vs. 11+ weeks of custom work +4. Avoids accumulating technical debt +5. Aligns diskless with Kafka's replica model -**This preserves "always available" semantics:** -- Current diskless: Transformer hashes to any alive broker → always available -- Managed RF diskless: Controller reassigns offline replicas to online brokers → always available +### Related Documents -| Scenario | Broker Status | RF Change | Action | -|----------|---------------|-----------|--------| -| Legacy topic, cluster migration | Not registered | Expand to rack_count | Modernize to rack-aware | -| RF=3, one broker failed | Registered, offline | Keep RF=3 | Replace offline replica | -| RF=1, broker maintenance | Registered, offline | Keep RF=1 | Move to online broker | -| Classic topic | Any | N/A | No action (not diskless) | +- [DESIGN.md](DESIGN.md) — Overall tiered storage unification design -#### Other Deployment Models (Broker IDs Overlap) +--- -If the new cluster reuses broker IDs from the old cluster (e.g., in-place upgrade on same VMs): +## Objectives -**If old brokers are offline:** Auto-reassignment triggers (offline replica detection) -**If old brokers are online:** Topics work normally with existing placement +Enable **rack-aware, stable KRaft-managed replicas** for diskless topics: -**For RF=1 topics that should be expanded to rack-aware:** -- Metric `kafka.controller.diskless.rf1_topics_total` tracks diskless topics with RF=1 -- Log warning: `WARN Diskless topic 'foo' has RF=1; consider expanding to rack-aware placement` -- Operator can manually expand using `kafka-reassign-partitions.sh` +- **KRaft-managed replicas**: Diskless topics will have actual KRaft-managed replicas (RF = rack count) +- **Rack-aware at creation**: Topics are created with one replica per rack +- **Standard operations after creation**: Once created, standard Kafka replica management applies +- **Controller-managed RF**: Users don't specify RF; controller computes from rack topology +- **Leader-agnostic produce and consume**: Any replica can accept writes and serve reads +- **Always available via transformer**: Transformer ensures availability by routing to alive brokers -**Manual expansion to rack-aware:** +--- -```bash -# Generate reassignment JSON (one replica per rack) -cat > reassignment.json << 'EOF' -{ - "version": 1, - "partitions": [ - {"topic": "foo", "partition": 0, "replicas": [101, 103, 105]} - ] -} -EOF +## Design: Transformer-First Availability -# Execute (instant for diskless - metadata only) -kafka-reassign-partitions.sh --bootstrap-server localhost:9092 \ - --reassignment-json-file reassignment.json \ - --execute -``` +### The Insight -#### Reassignment Process +For diskless topics, **availability** and **metadata accuracy** can be decoupled: -When controller detects a diskless partition with offline replica(s): +| Concern | Priority | Who Handles | Speed | +|-------------------|--------------|-------------------|------------| +| Availability | Critical | Transformer | Instant | +| Metadata accuracy | Nice-to-have | Controller (lazy) | Eventually | -**Case 1: Broker ID does not exist in cluster (e.g., cluster migration)** +### How It Works -This indicates the partition was created on an old cluster with different broker IDs. Expand to rack-aware placement. +**Current diskless behavior (RF=1, faked):** +- Transformer hashes to any alive broker +- Partition is always available +- KRaft metadata is ignored for routing -1. Log: `INFO Detected diskless partition 'foo-0' with non-existent broker (broker 3 not registered)` -2. Compute target: one broker per rack (RF=rack_count) -3. Issue `AlterPartitionReassignments` to set new replicas -4. Log: `INFO Reassigned diskless partition 'foo-0' to RF=3, replicas=[101,103,105]` +**Proposed behavior (RF=rack_count, real replicas):** +- KRaft stores RF=rack_count with real broker IDs +- Transformer filters by client AZ from KRaft replicas +- **If assigned replica is offline → fall back to any alive broker in AZ** +- Partition is always available (same as today!) +- KRaft metadata may be temporarily stale (shows offline broker) -**Case 2: Broker ID exists but is offline (e.g., broker failure, maintenance)** +### Why This Works for Diskless -This indicates a temporary or permanent broker outage. Preserve current RF, just replace offline replica. +``` +Classic topic: + Data on broker X → client MUST talk to broker X + Broker X offline → partition unavailable (until reassigned) + +Diskless topic: + Data in object storage → ANY broker can serve + Broker X offline → transformer routes to broker Y + Partition stays available (no reassignment needed!) +``` -1. Log: `INFO Detected diskless partition 'foo-0' with offline replica (broker 101)` -2. Find online broker in same rack as offline broker (preserve rack-awareness) -3. If no broker in same rack: pick any online broker (preserve RF, log warning about rack-awareness) -4. Issue `AlterPartitionReassignments` to replace offline replica -5. Log: `INFO Reassigned diskless partition 'foo-0' replica: broker 101 → 102` +The transformer already has all the information it needs: +1. KRaft replica assignments (preferred brokers) +2. Alive brokers in cluster +3. Client AZ -**Summary:** +It can make instant routing decisions without waiting for controller. -| Condition | RF Change | Rationale | -|-----------|-----------|-----------| -| Broker not registered in cluster | Expand to rack_count | Legacy topic from old cluster, modernize | -| Broker registered but offline | Keep current RF | Operator's RF choice respected | +--- -**This is metadata-only** — no data movement required (data is in object storage). +## Activation Model (Binary Version) -#### Example +Managed RF is activated by **deploying a new binary version**. No feature flags, no topic attributes, no server configs. -``` -Legacy topic (created on old cluster): - Topic: foo (diskless.enable=true) - Partition 0: Replicas=[3], Leader=3, ISR=[3] - ↑ - Broker 3 was valid on old cluster - -New cluster brokers: [101, 102, 103, 104, 105, 106] in racks [A, A, B, B, C, C] - ↑ - Broker 3 doesn't exist anymore - -After auto-migration: - Topic: foo (diskless.enable=true) - Partition 0: Replicas=[101,103,105], Leader=101, ISR=[101,103,105] - ↑ ↑ ↑ - One broker per rack (A, B, C) -``` +### Behavior Summary -#### When Auto-Reassignment Runs +**Old binary:** +- Diskless topics use legacy "RF=1 / faked metadata" behavior +- Transformer calculates synthetic placement via hashing -- **Trigger:** Controller startup, metadata change, or periodic scan -- **Pacing:** Batched to avoid overwhelming controller (configurable) -- **Idempotent:** Re-running when no offline replicas exist is a no-op +**New binary:** +- Diskless topics use KRaft-managed placement (one replica per rack at creation) +- Transformer filters KRaft placement by client AZ +- **Falls back to alive brokers if assigned replicas are offline** -#### Observability +### Existing Diskless Topics -**Auto-reassignment of offline replicas:** -- Log: `INFO Detected diskless partition 'foo-0' with offline replica (broker 101)` -- Log: `INFO Reassigned diskless partition 'foo-0' replica: broker 101 → 102` -- Metric: `kafka.controller.diskless.replicas_reassigned_total` (counter) -- Metric: `kafka.controller.diskless.offline_replicas_total` (gauge) — pending reassignment +Existing diskless topics (RF=1) continue to work: -**RF=1 topics (informational):** -- Log: `WARN Diskless topic 'foo' has RF=1; consider expanding to rack-aware placement` -- Metric: `kafka.controller.diskless.rf1_topics_total` (gauge) — topics with RF=1 +**Immediate availability:** +- Transformer sees RF=1 with potentially offline broker +- Falls back to hash-based selection (same as today) +- No downtime -#### Rollback +**Eventual modernization (optional):** +- Controller can lazily detect RF=1 diskless topics +- Background task expands to RF=rack_count when convenient +- Not urgent — availability is already handled by transformer -If rolled back to old binary: -- Expanded RF is preserved in KRaft metadata -- Old brokers ignore KRaft RF and use transformer hashing -- No harm — metadata is unused until re-upgraded -- Re-upgrade will see RF > 1 and skip migration (idempotent) +**Manual modernization (alternative):** +- Operator uses `kafka-reassign-partitions.sh` to expand RF +- Same tooling as classic topics --- @@ -325,12 +312,6 @@ When a diskless topic is created: - One broker is selected per rack for each partition - Broker selection within a rack uses load balancing (least loaded broker) -### Brokers Without Rack - -Brokers without an explicit `broker.rack` config are treated as belonging to `rack=unknown`. -- They are considered a separate "rack" for placement purposes -- If all brokers lack rack config, RF=1 (single "unknown" rack) - ### Placement Is Static After Creation Once a topic is created: @@ -338,8 +319,6 @@ Once a topic is created: - Replica assignments don't automatically adjust - Standard Kafka replica management applies -This keeps the design simple and predictable. - --- ## Controller Behavior @@ -347,338 +326,71 @@ This keeps the design simple and predictable. ### Topic Creation When creating diskless topics (`diskless.enable=true`): - -**RF Computation:** - Controller counts distinct racks from registered brokers -- RF = rack count (e.g., 3 racks → RF=3) - -**Placement:** +- RF = rack count - One replica assigned per rack -- Within each rack, select least loaded broker -- Leader is typically the first replica (standard Kafka behavior) - -**Validation:** -- `replicationFactor` in CreateTopics request must be `-1` or `1` - - `-1`: Controller computes placement (recommended) - - `1`: Accepted for compatibility with automation tools - - Any value `> 1`: Rejected (RF is system-managed) - - `0`: Rejected -- `replicaAssignments` must be empty (manual assignment not allowed at creation) - -**Error messages:** -``` -InvalidRequestException: replication factor is system-managed for diskless topics; -use replicationFactor=-1 or replicationFactor=1 -``` - -``` -InvalidRequestException: replica assignments cannot be specified for diskless topics; -placement is system-managed at creation time -``` +- Reject `replicationFactor > 1` and manual assignments ### Add Partitions -When adding partitions to an existing diskless topic: - -```bash -kafka-topics.sh --alter --topic foo --partitions 10 -``` - -**Behavior:** -- New partitions are placed using the same one-per-rack logic as creation -- RF for new partitions = current rack count (may differ from original if racks changed) -- Existing partitions are not affected -- Manual replica assignments are rejected (same as topic creation) - -This ensures consistency: all partitions of a diskless topic use rack-aware placement. - -**Implementation note:** Currently `ReplicationControlManager.createPartitions()` uses the standard replica placer without diskless-specific handling. This needs to be updated to: -1. Detect diskless topics via `diskless.enable=true` config -2. Use rack-aware placement instead of standard placer -3. Reject manual replica assignments +New partitions use same one-per-rack logic as creation. ### Standard Operations (After Creation) -After topic creation, diskless topics behave like classic Kafka topics for all operations: +**No auto-reassignment on broker failure.** Transformer handles availability. **Leader Election:** - Standard Kafka leader election from ISR -- For diskless topics, all replicas are always in ISR (data is in object storage) -- `kafka-leader-election.sh` works normally -- Leader can be any replica in ISR (shuffled on election) - -**Important:** This is unchanged from current diskless behavior. For diskless topics with AZ-aware clients, leader election doesn't force cross-AZ traffic. The transformer returns the local replica to each client regardless of who the KRaft leader is. The KRaft leader is primarily for controller coordination (e.g., future RLM tasks), not for directing client traffic. +- For diskless topics, all replicas are always in ISR **Broker Failure:** -- Replicas on failed broker become offline -- Controller detects offline replica and **auto-reassigns** to online broker (preserving RF and rack-awareness when possible) -- Leader fails over if leader was on failed broker -- Partition remains available (unlike classic topics which would be under-replicated) -- If original broker returns later, no automatic action — new assignment is kept - -**Broker Shutdown (Controlled):** -- Standard controlled shutdown -- Leaders moved away before shutdown -- No special handling for diskless +- Replicas on failed broker become offline in KRaft metadata +- **Transformer routes around offline broker** (instant) +- Partition remains available +- KRaft shows under-replicated (informational only for diskless) +- When broker returns, it's back in ISR immediately **Reassignment:** - `kafka-reassign-partitions.sh` works normally -- Operator can reassign replicas to any brokers -- No data movement (just metadata update for diskless) -- ⚠️ Operator can break rack-awareness (see below) - -**Rolling Upgrades:** -- Standard Kafka rolling restart behavior -- No special handling for diskless topics - -### Operator Reassignment and Rack-Awareness - -Operators can use standard reassignment tools on diskless topics: - -```bash -kafka-reassign-partitions.sh --bootstrap-server localhost:9092 \ - --reassignment-json-file reassignment.json \ - --execute -``` - -**Important:** The controller does **not** validate rack-awareness for reassignments. - -If an operator reassigns replicas such that rack-awareness is broken (e.g., 2 replicas in same rack): -- Assignment is **accepted** (standard Kafka behavior) -- Warning logged: `WARN Diskless topic 'foo' partition 0 has non-rack-aware placement` -- Metric emitted: `kafka.controller.diskless.rack_aware{topic="foo",partition="0"} = 0` -- Operator is responsible for fixing if desired - -**Rationale:** -- Keeps implementation simple (no validation logic in reassignment path) -- Same behavior as classic topics (Kafka doesn't enforce rack-awareness on reassignment) -- Operator has full control for edge cases (intentional non-rack-aware placement) - -### Topology Changes (Racks Added/Removed) - -**New rack added:** -- Existing topics are NOT automatically updated -- RF stays at original value -- Operator can reassign to include new rack if desired -- New topics will include the new rack - -**Rack removed (all brokers gone):** -- Replicas on that rack become offline -- Controller auto-reassigns to online brokers in other racks (preserving RF) -- Rack-awareness may be temporarily broken (logged as warning) -- Operator can manually rebalance to restore rack-awareness if desired - -**Note:** These are edge cases that rarely happen in practice. Auto-reassignment ensures availability; rack-awareness restoration is optional. +- Operator can reassign replicas if desired +- **Not required for availability** — transformer handles routing --- ## Broker Behavior -**Key point: Broker behavior is unchanged from today.** Diskless topics already work with existing Inkless produce/consume handlers. This design changes **metadata and placement**, not the data path. - -### Produce Path - -**No changes to produce handling.** The existing Inkless produce path continues to work: -- Any broker can accept writes (leader-agnostic) -- Writes go to object storage via `AppendHandler` -- Batch metadata stored in Control Plane (PostgreSQL) - -The only difference is that KRaft now tracks replica assignments, but this doesn't affect how produces are processed. - -### Consume Path - -**No changes to consume handling.** The existing Inkless consume path continues to work: -- `FetchHandler` + `Reader` serve reads from object storage -- Data fetched via Control Plane batch lookups -- Object cache provides locality benefits - -What changes is **which broker the client talks to** (determined by transformer filtering), not how that broker serves the request. - -### ISR Semantics - -For diskless topics, **all replicas are always considered in-sync** because: -- Source of truth (batch metadata + WAL objects) is shared via object storage -- All replicas have immediate access to the same committed data -- No replication lag in the classic sense - -**Implementation:** ISR is stored in KRaft as all assigned replicas. At topic creation, controller sets `ISR = replicas`. No `AlterPartition` requests are needed since there's no lag tracking — replicas don't fetch from each other. - -**This is unchanged from today** — diskless topics already have this property. The difference is that now there are multiple replicas in the ISR (one per rack) instead of a single faked replica. - -### No Replica Fetcher for Diskless Topics - -**Diskless topics do not start replica fetcher threads.** This is a critical design invariant that must be preserved. - -#### Current Implementation - -In `ReplicaManager.scala`, diskless topics are explicitly skipped when applying metadata changes: - -**`applyLocalLeadersDelta`** (line ~3118): -```scala -localLeaders.foreachEntry { (tp, info) => - if (!_inklessMetadataView.isDisklessTopic(tp.topic())) - getOrCreatePartition(tp, delta, info.topicId).foreach { ... } -} -``` - -**`applyLocalFollowersDelta`** (line ~3154): -```scala -localFollowers.foreachEntry { (tp, info) => - if (!_inklessMetadataView.isDisklessTopic(tp.topic())) - getOrCreatePartition(tp, delta, info.topicId).foreach { ... } -} -``` - -This means: -- No local `Partition` objects created for diskless topics -- No local log directories created -- No `ReplicaFetcherThread` started for followers -- No replication traffic between brokers for diskless data - -#### Why This Must Be Preserved - -1. **No data to replicate**: Diskless data lives in object storage, not local logs -2. **Performance**: Fetcher threads consume CPU and network; unnecessary for diskless -3. **Consistency**: All brokers read from the same object storage; no catch-up needed -4. **Simplicity**: Fewer moving parts means fewer failure modes - -#### Regression Prevention - -**What could cause a regression:** -- Code changes that don't check `isDisklessTopic()` before creating partitions -- New code paths that bypass the existing guards -- Feature flags that accidentally enable local storage for diskless topics - -**How to prevent regressions:** - -1. **Unit tests**: Ensure tests verify no fetcher is started for diskless topics - ```scala - // Example test assertion - assertEquals(None, replicaManager.replicaFetcherManager.getFetcher(disklessTopicPartition)) - ``` - -2. **Integration tests**: Verify no replication traffic for diskless topics - - Create diskless topic with RF=3 - - Produce messages - - Verify no inter-broker fetch requests for that topic - -3. **Code review checklist**: Any changes to `ReplicaManager.applyLocalLeadersDelta` or `applyLocalFollowersDelta` must preserve the `isDisklessTopic()` guard - -4. **Metrics**: Monitor `kafka.server:type=ReplicaFetcherManager,name=*` to detect unexpected fetcher activity - -#### Managed RF Interaction - -With managed RF, diskless topics will have multiple replicas (one per rack). However: -- These are **metadata-only replicas** in KRaft -- They exist for leader election and RLM coordination -- They do **not** trigger replica fetcher threads -- The `isDisklessTopic()` guards in `ReplicaManager` continue to apply - -The managed RF design **does not change** this behavior — it only changes how replica assignments are computed and stored in KRaft. - -#### No Local Partition Objects — Implications - -**Current behavior:** `ReplicaManager` skips `getOrCreatePartition()` for diskless topics, meaning: -- No `Partition` objects on brokers -- No `UnifiedLog` objects -- No local log directories - -**This raises questions:** - -1. **How does leader election work without `Partition` objects?** - - Leader election is handled by the KRaft controller, not broker `Partition` objects. The controller updates partition metadata (leader, ISR) in KRaft records. Brokers observe these changes via metadata updates but don't need local `Partition` objects to "become" leader — they just serve requests for partitions where metadata says they're a replica. - -2. **How do brokers know they should serve requests for a partition?** - - Currently: `InklessTopicMetadataTransformer` returns metadata pointing clients to brokers. Brokers check `isDisklessTopic()` and route to Inkless handlers (`AppendHandler`, `FetchHandler`) instead of local log. - - With managed RF: Same flow, but transformer filters KRaft metadata instead of computing synthetic placement. - -3. **What about RLM integration?** - - RLM requires `Partition` and `UnifiedLog` objects to: - - Track local vs. tiered segment boundaries - - Coordinate segment uploads - - Run expiration tasks - - **This is out of scope for this design.** RLM integration will be addressed separately when implementing the tiering pipeline (see [DESIGN.md](DESIGN.md)). At that point, we may need to create `Partition` objects for the leader, but this can be deferred. - -4. **Can we avoid `Partition` objects entirely?** - - For the initial managed RF implementation: **yes**. We only need: - - KRaft metadata (replicas, leader, ISR) - - Transformer filtering - - Inkless handlers (already working) - - `Partition` objects become necessary when: - - Implementing local disk cache (future) - - Integrating with RLM for tiering (future) - - **Recommendation:** Start without `Partition` integration. Add it when implementing local cache or RLM integration. - -#### Research Needed - -These items are covered in [Phase 0: Research and Validation](#phase-0-research-and-validation): -1. Leader election works correctly without broker `Partition` objects (test with existing diskless topics) -2. ISR updates in KRaft don't require broker-side `Partition` state -3. `DescribeTopics` / `ListOffsets` work correctly for diskless topics with RF > 1 +- No replica fetcher threads for diskless topics +- No local `Partition` objects (for now — may add for RLM coordination later) +- Inkless handlers (`AppendHandler`, `FetchHandler`) serve requests --- ## Metadata Transformation -### Current vs Target Behavior - -**Today (legacy):** -- `InklessTopicMetadataTransformer` intercepts responses -- Forces diskless partitions to look like RF=1 -- Calculates synthetic leader/ISR by hashing `(topicId, partition)` into alive brokers in client AZ - -**Target (managed RF):** -- KRaft placement is source of truth -- Replica sets reflect controller placement (one per rack) -- Transformer **filters** by client AZ instead of **calculating** - -### Transformer Implementation - -**Input changes:** -- For each diskless partition: KRaft replica set (brokerIds), KRaft leader id -- For each broker: `broker.rack` (AZ) via `Node.rack()` - -**Output (per request / per clientAZ):** -- Filtered `replicaNodes` / `isrNodes` for diskless partitions - -**Filtering logic:** -1. Check if KRaft-assigned replicas exist in the cluster (are alive) -2. If replicas exist: - - If partition has replica in `clientAZ`: return only that local replica as leader/replicas/ISR - - If no replica in `clientAZ`: return full replica set (standard Kafka behavior — client talks to leader) -3. If replicas are orphaned (none exist in cluster): **fall back to legacy hash-based selection** - -**Why the orphaned fallback is needed:** +### Filtering Logic -During cluster migration, there's a brief window between: -1. New cluster starts with restored KRaft metadata (contains old broker IDs) -2. Controller completes auto-migration to new broker IDs - -In this window, legacy topics have `Replicas=[3]` where broker 3 no longer exists. Without the fallback: -- New transformer would return empty/invalid metadata -- Partition would be effectively offline for clients - -The fallback ensures partitions remain available during this migration window (typically seconds to minutes). Once auto-migration completes, the fallback is no longer triggered. - -**Note:** This is consistent with current diskless behavior. Today, the transformer returns one replica based on hash as a "fake" leader. With managed RF, we have one real replica per AZ and return the local one. The orphaned fallback preserves availability only during the migration window. - -**Detection:** -- New binary always uses KRaft-placement-based projection for diskless topics -- Check `diskless.enable=true` topic config to identify diskless topics - -### Unclean Leader Election +``` +FOR each diskless partition: + assigned_replicas = KRaft replica set + alive_replicas = assigned_replicas ∩ alive_brokers + + IF alive_replicas is not empty: + # Normal case: use KRaft placement + IF any alive_replica in clientAZ: + RETURN local replica (AZ-aware routing) + ELSE: + RETURN all alive_replicas (cross-AZ fallback) + ELSE: + # All assigned replicas offline: fall back to hash + RETURN hash-based selection from all alive brokers in clientAZ +``` -For diskless topics, `unclean.leader.election.enable=true` can be safely enabled — there is no data loss risk since all data is in object storage. +### Key Properties -However, **auto-reassignment of offline replicas** (not unclean election) is our primary availability mechanism. See [Rejected Alternative E: Unclean Leader Election for Availability](#rejected-alternative-e-unclean-leader-election-for-availability) for the full analysis. +1. **Instant availability**: No waiting for controller +2. **AZ-aware when possible**: Uses KRaft placement if alive +3. **Graceful degradation**: Falls back to hash if needed +4. **No metadata dependency for availability**: Stale metadata doesn't cause downtime --- @@ -686,392 +398,238 @@ However, **auto-reassignment of offline replicas** (not unclean election) is our ### Metrics -**Controller metrics (all prefixed `kafka.controller.diskless.`):** -- `effective_rf{topic}` - RF assigned at creation -- `rack_aware{topic,partition}` - 1 if one-per-rack, 0 if not -- `rack_aware_partitions_total{topic}` - Count of rack-aware partitions -- `non_rack_aware_partitions_total{topic}` - Count of non-rack-aware partitions -- `rf1_topics_total` - Count of diskless topics with RF=1 -- `replicas_reassigned_total` - Count of replicas auto-reassigned due to offline broker -- `offline_replicas_total` - Count of replicas pending reassignment +**Controller metrics:** +- `kafka.controller.diskless.effective_rf{topic}` - RF assigned at creation +- `kafka.controller.diskless.rack_aware{topic,partition}` - 1 if one-per-rack, 0 if not +- `kafka.controller.diskless.rf1_topics_total` - Legacy topics not yet modernized -**Standard Kafka metrics (already exist):** -- `kafka.server:type=ReplicaManager,name=UnderReplicatedPartitions` -- `kafka.controller:type=KafkaController,name=OfflinePartitionsCount` +**Transformer metrics:** +- `kafka.server.diskless.transformer.fallback_total` - Count of hash fallbacks +- `kafka.server.diskless.transformer.offline_replicas_routed_around` - Routing decisions -### Admin Surfaces +**Standard Kafka metrics:** +- `UnderReplicatedPartitions` - Will show diskless partitions with offline brokers +- Note: For diskless, "under-replicated" is informational, not critical -**DescribeTopic shows actual replica assignments:** -```bash -kafka-topics.sh --bootstrap-server localhost:9092 --describe --topic foo +### Logs -# Output: -# Topic: foo TopicId: abc123 PartitionCount: 3 ReplicationFactor: 3 -# Partition: 0 Leader: 1 Replicas: 1,3,5 Isr: 1,3,5 -# Partition: 1 Leader: 3 Replicas: 3,5,1 Isr: 3,5,1 -# Partition: 2 Leader: 5 Replicas: 5,1,3 Isr: 5,1,3 +``` +INFO [Transformer] Diskless partition 'foo-0' has offline replica (broker 101), + routing to alive replica (broker 103) + +WARN [Transformer] Diskless partition 'foo-0' has ALL replicas offline, + using hash-based fallback to broker 105 ``` -### Logs +### Operational Notes -**Warning when rack-awareness is broken:** +**`kafka-topics.sh --describe` may show stale info:** ``` -WARN [Controller] Diskless topic 'foo' partition 0 has non-rack-aware placement: - replicas [1,2,5] map to racks [rack-A,rack-A,rack-C]. Expected one replica per rack. +Topic: foo PartitionCount: 3 ReplicationFactor: 3 + Partition: 0 Leader: 101 Replicas: 101,103,105 Isr: 101,103,105 + ↑ + Broker 101 may be offline, but transformer routes to 103 ``` -This warning is logged: -- When reassignment results in non-rack-aware placement -- Periodically if non-rack-aware placement persists (configurable interval) +This is acceptable because: +1. Availability is not affected +2. Operator can see actual state via broker metrics +3. Eventually consistent (when broker returns or is reassigned) --- ## Implementation Path -**Total Estimate: 9 weeks with 1 engineer, or 5-6 weeks with 2 engineers** - -### Phase 0: Research and Validation - -Verify assumptions before implementation: +**Total Estimate: 6 weeks with 1 engineer, or 4 weeks with 2 engineers** -1. Leader election works correctly without broker `Partition` objects (test with existing diskless topics) -2. ISR updates in KRaft don't require broker-side `Partition` state -3. `DescribeTopics` / `ListOffsets` work correctly for diskless topics with RF > 1 +### Phase 0: Research and Validation (1 week) -**Estimate:** 1 week +1. Leader election works without broker `Partition` objects +2. ISR updates don't require broker-side `Partition` state +3. `DescribeTopics` / `ListOffsets` work with RF > 1 -### Phase 1: Topic Creation with Rack-Aware Placement +### Phase 1: Topic Creation with Rack-Aware Placement (2 weeks) -1. Modify `ReplicationControlManager` to detect `diskless.enable=true` topics -2. Compute RF = rack count at creation time +1. Modify `ReplicationControlManager` to detect diskless topics +2. Compute RF = rack count at creation 3. Implement one-per-rack broker selection -4. Reject `replicationFactor > 1` and non-empty `replicaAssignments` +4. Reject `replicationFactor > 1` and manual assignments -**Estimate:** 2 weeks - -### Phase 2: Transformer Changes +### Phase 2: Transformer Changes (2 weeks) 1. Update `InklessTopicMetadataTransformer` to read KRaft placement 2. Implement AZ filtering logic -3. **Add orphaned replica fallback** — if KRaft replicas don't exist, fall back to legacy hash-based selection -4. Remove synthetic hashing calculation (but keep as fallback path) - -**Estimate:** 2 weeks +3. Add offline replica routing — if KRaft replica offline, route to alive replica +4. Add hash fallback — if all replicas offline, use legacy hash +5. Add metrics for fallback tracking -### Phase 3: Add Partitions Support +### Phase 3: Add Partitions Support (1 week) 1. Apply same one-per-rack logic when adding partitions 2. Handle case where rack count changed since topic creation -**Estimate:** 1 week - -### Phase 4: Offline Replica Auto-Reassignment - -1. Implement offline replica detection (any diskless partition with offline broker) -2. Add auto-reassignment logic in controller: - - Broker ID not registered in cluster → expand to RF=rack_count (legacy topic modernization) - - Broker ID registered but offline → keep current RF, replace offline replica (same rack if possible) -3. Add pacing controls (batch size, interval) -4. Add metrics: `replicas_reassigned_total`, `offline_replicas_total`, `rf1_topics_total` - -**Estimate:** 2 weeks (increased due to broader scope) - -### Phase 5: Observability - -1. Add `rack_aware`, `rack_aware_partitions_total`, `non_rack_aware_partitions_total` metrics -2. Add warning logs for non-rack-aware placement and RF=1 topics -3. Documentation - -**Estimate:** 1 week - ### Summary -| Phase | Scope | Estimate | -|-------|-------|----------| -| 0 | Research and Validation | 1 week | -| 1 | Topic Creation with Rack-Aware Placement | 2 weeks | -| 2 | Transformer Changes | 2 weeks | -| 3 | Add Partitions Support | 1 week | -| 4 | Offline Replica Auto-Reassignment | 2 weeks | -| 5 | Observability | 1 week | -| **Total** | | **9 weeks (1 eng) / 5-6 weeks (2 eng)** | - -This includes the offline replica auto-reassignment which preserves "always available" semantics. - -### Testing Strategy - -**Unit Tests:** -- `ReplicationControlManager`: rack-aware placement logic, RF computation, validation -- `InklessTopicMetadataTransformer`: AZ filtering, cross-AZ fallback -- Migration detection: orphaned replica identification - -**Integration Tests:** -- Topic creation with `diskless.enable=true` results in RF=rack_count -- Add partitions uses rack-aware placement -- Existing topics with orphaned replicas are auto-migrated -- Reassignment works and logs warnings for non-rack-aware placement -- No replica fetcher threads started for diskless topics with RF > 1 -- **Orphaned replica fallback**: Transformer returns valid broker when KRaft replicas don't exist -- **Offline replica auto-reassignment**: Broker goes offline → replica reassigned to online broker -- **Legacy topic modernization**: Broker ID not in cluster → RF expanded to rack_count - -**System Tests:** -- Multi-AZ cluster with diskless topics -- Client AZ awareness: verify clients talk to local replica -- Broker failure: verify leader election and continued availability -- Rolling restart: verify no disruption - -**Existing Test Coverage:** -See [DISKLESS_INTEGRATION_TEST_COVERAGE.md](../../DISKLESS_INTEGRATION_TEST_COVERAGE.md) for current Inkless test coverage. Managed RF tests should extend this framework. - -### Future Work: RLM Integration - -This design enables RLM integration but doesn't implement it. Key considerations for future RLM work: - -**Why RLM needs managed RF:** -- RLM's `onLeadershipChange()` is the entry point for tiering tasks -- Requires KRaft-managed partition leadership (not faked) -- Managed RF provides this foundation - -**What RLM integration will require:** -1. **Partition objects on leader:** RLM uses `Partition` and `UnifiedLog` APIs. The leader broker may need to create lightweight `Partition` objects for diskless topics. -2. **Segment boundary tracking:** RLM needs to know which offsets are local vs. tiered. For diskless, "local" means Control Plane (PostgreSQL), "tiered" means S3 segments. -3. **Expiration coordination:** Leader runs RLM expiration tasks to clean up tiered segments. - -**Design hints for RLM integration:** -- Create `Partition` objects only on the leader, not followers -- `UnifiedLog` can be a thin wrapper that delegates to Inkless handlers -- Alternatively, implement RLM hooks directly without full `Partition` — needs investigation - -**This is out of scope for managed RF** but the decisions here (KRaft-managed replicas, stable leader election) provide the foundation. See [DESIGN.md](DESIGN.md) Stream 7 for tiering pipeline details. +| Phase | Scope | Estimate | +|-----------|------------------------------------------|----------| +| 0 | Research and Validation | 1 week | +| 1 | Topic Creation with Rack-Aware Placement | 2 weeks | +| 2 | Transformer Changes (with fallback) | 2 weeks | +| 3 | Add Partitions Support | 1 week | +| **Total** | | **6 weeks (1 eng) / 4 weeks (2 eng)** | --- ## Rejected Alternatives -### Why KRaft-Managed Replicas Are Required +### Alternative A: Shrink RF During Migration -All RF=1 alternatives fail because **RLM requires KRaft-managed partition leadership**: +**Concept:** When migrating Classic → Diskless, shrink RF=3 to RF=1, then use current diskless model. -``` -Tiered storage integration (for PG scalability) - │ - ▼ -RLM expiration tasks (to clean up tiered segments) - │ - ▼ -onLeadershipChange() must fire (RLM's entry point) - │ - ▼ -KRaft-managed partition leadership (not faked/virtual) - │ - ▼ -RF > 1 with KRaft-managed replica assignments -``` +**Why considered:** +- Maintains consistency with current RF=1 diskless model +- No need to change existing diskless implementation -The tiering pipeline's merge phase requires a partition leader to coordinate segment creation and RLM registration. There is no path to production-grade diskless topics without embracing KRaft-based replica placement. +**Why rejected:** +1. Timing complexity (when does RF shrink happen?) +2. Loses rack-awareness — must recreate for reverse migration +3. **Reverse migration requires RF expansion** (two operations) +4. RLM integration still blocked +5. Racing job assignment continues -See [DESIGN.md](DESIGN.md) Stream 7 for tiering pipeline details. +See [Approach Comparison](#approach-comparison) for details. -### Rejected Alternative: Feature Flag Activation +--- -**Concept:** Follow upstream Kafka patterns with `DisklessVersion` feature enum, server config, and KRaft records. +### Alternative B: Keep Classic RF, Continue Faking Metadata -**Why we initially considered it:** -- Upstream Kafka uses feature flags for backward-compatible feature rollout (e.g., `EligibleLeaderReplicasVersion`) -- Allows gradual activation: deploy new binary first, then enable feature cluster-wide -- Provides explicit opt-in, reducing risk of unexpected behavior changes -- Supports mixed-version clusters where some brokers have the feature and others don't +**Concept:** Keep existing RF (e.g., RF=3) in KRaft but continue using transformer to fake metadata. -**What it would have involved:** -- Policy flag (`disklessManagedRF`) as new KRaft record -- Feature enum (`DisklessVersion`) following `EligibleLeaderReplicasVersion` pattern -- Server config (`diskless.managed.rf.enable`) requiring cluster-wide coordination -- Manual activation via `kafka-features.sh --upgrade --feature diskless.version=1` +**Why considered:** +- Zero controller changes for migration +- Simplest migration path +- If users migrate back, they keep same partition assignments -**Why we dropped it:** -- **Inkless deployment model is different**: We deploy new binaries on new VMs, not rolling upgrades on existing clusters. There's no mixed-version period to manage. -- **`diskless.enable` already gates behavior**: Topics must explicitly opt into diskless mode. The feature flag would be redundant. -- **Operational overhead without benefit**: Adds a manual activation step that doesn't provide additional safety in our deployment model. -- **Simpler rollback**: Binary rollback is sufficient; no need to coordinate feature flag state. +**Why rejected:** +1. **Ghost replicas** — KRaft shows healthy replicas that aren't real +2. **RLM blocked** — `onLeadershipChange()` needs `Partition` objects. Adding them would be similar complexity to proposed approach. +3. **Reverse migration requires "un-ghosting"** replicas +4. Misleading tooling output +5. Racing job assignment continues -**When to reconsider:** -- If upstreaming to Apache Kafka (strict backward compatibility requirements) -- If needing per-topic opt-in/opt-out for managed RF specifically -- If supporting long-lived mixed-version clusters (not our deployment model) +See [Ghost Replicas Problem](#ghost-replicas-problem) for details. -### Rejected Alternative: Dynamic Reconciliation +--- -**Concept:** Controller continuously reconciles replica placement to maintain one-per-rack as topology changes. +### Alternative C: Controller Auto-Reassignment -**Why we initially considered it:** -- Ensures rack-awareness is always maintained, even after topology changes -- Automatically adapts RF when racks are added (more availability) or removed (avoid under-replication) -- Reduces operational burden — no manual reassignment needed -- Provides "self-healing" behavior similar to Kubernetes operators +**Concept:** Controller proactively detects offline replicas and reassigns to online brokers immediately. -**What it would have involved:** -- Reconciliation loop monitoring rack changes -- Automatic RF adjustment when racks added/removed -- Drift detection and auto-fix when operator breaks rack-awareness -- Rack liveness state machine (HEALTHY → DEGRADED → UNAVAILABLE) to distinguish transient failures from permanent rack loss +**Why considered:** +- KRaft metadata always accurate +- Standard tooling shows correct state -**Why we dropped it:** -- **Significant complexity**: Adds new controller component with its own state machine, edge cases, and failure modes -- **Divergent from Kafka norms**: Standard Kafka doesn't auto-adjust RF or auto-reassign. Operators expect explicit control. -- **Rare scenarios**: Topology changes (adding/removing racks) are infrequent in practice. Optimizing for rare cases adds constant complexity. -- **Existing tools work**: `kafka-reassign-partitions.sh` handles all cases. Operators already know this workflow. -- **Harder to reason about**: Auto-reconciliation can surprise operators. "Why did my partition move?" is a common complaint with auto-rebalancing systems. +**Why rejected:** +- **3 weeks additional complexity** for something transformer already handles +- **Instant vs eventually consistent** — transformer is faster +- **Consistent with today** — diskless already uses transformer for availability +- **Fewer failure modes** — no controller state machine to debug **When to reconsider:** -- If customers frequently add/remove racks and manual reassignment becomes burdensome -- If we build a broader "Kafka operator" that manages cluster topology holistically +- If KRaft metadata accuracy becomes critical requirement +- If operators require `describe topic` to always reflect reality -### Rejected Alternative A: Virtual Leader for RLM Tasks - -**Concept:** Keep RF=1 (current faked metadata), but designate a deterministic "virtual leader" per partition for RLM tasks. - -**Why we considered it:** -- Avoids changing the current RF=1 model that works for produce/consume -- RLM only needs *some* broker to run expiration tasks — could be a "virtual" designation -- Simpler than managing real replicas if we can make RLM work with virtual leadership - -**Why it fails:** -- **Two sources of leadership**: KRaft has one leader (faked), RLM needs another (virtual). Confusing for operators and tooling. -- **Failover complexity**: Virtual leader failover requires new coordination mechanism outside KRaft. What happens when the virtual leader dies? -- **RLM assumptions**: RLM code assumes real `Partition` objects with `UnifiedLog`. Significant refactoring needed to work with virtual concept. -- **Tiering pipeline needs `ReplicaManager`**: The merge phase that creates tiered segments needs broker-side context that only exists with real partitions. +--- -### Rejected Alternative B: Control Plane Manages Tiered Expiration +### Alternative D: Custom Job Coordinator -**Concept:** Extend PostgreSQL (Control Plane) to track tiered segment metadata and run expiration directly, bypassing RLM. +**Concept:** Keep faking metadata, add custom coordinator (outside KRaft) for background jobs. -**Why we considered it:** -- Control Plane already tracks batch metadata — could extend to track tiered segments -- Avoids needing KRaft-managed replicas entirely -- Keeps all Inkless metadata in one place (PostgreSQL) +**Why considered:** +- Gets deterministic job ownership without changing replica model +- Could use PG-based leader election -**Why it fails:** -- **Contradicts PG scalability goal**: The whole point of tiering is to *reduce* PG load. Adding more metadata to PG defeats the purpose. -- **Duplicates RLM logic**: Retention policies (`retention.ms`, `retention.bytes`) are already implemented in RLM. Reimplementing in Control Plane doubles the code and bugs. -- **Breaks tooling**: Kafka admin tools expect RLM for tiered storage management. Custom Control Plane expiration wouldn't integrate. -- **Cross-system consistency**: Tiered segments in S3, metadata in PG, Kafka expecting RLM — three systems to keep consistent. Recipe for orphaned data. +**Why rejected:** +1. **Two leadership systems** — KRaft vs. custom coordinator +2. **RLM still blocked** — RLM uses KRaft leadership +3. **More complexity** — three systems (KRaft + transformer + coordinator) +4. **Defeats purpose** — if adding coordinator, why not use KRaft's? -### Rejected Alternative C: Direct Tiered Read via FetchHandler +--- -**Concept:** Extend `FetchHandler` to read tiered storage (S3 segments) directly, bypassing RLM read path. +## Appendix -**Why we considered it:** -- `FetchHandler` already serves diskless reads from object storage — could extend to tiered segments -- Avoids RLM dependency for reads -- Potentially simpler than full RLM integration +### Ghost Replicas Problem -**Why it fails:** -- **Only solves reads**: Expiration (the main RLM value) still needs a solution. Must combine with Alternative A or B, inheriting their problems. -- **Duplicates index handling**: RLM maintains indexes for tiered segments. `FetchHandler` would need to duplicate this or depend on RLM indexes anyway. -- **Partial solution**: Doesn't address the core problem (needing KRaft leadership for RLM). Just moves complexity around. +If we keep RF=3 in KRaft but transformer ignores it, Kafka creates "ghost replicas": -### Rejected Alternative D: Treat Tiered Data as Read-Only Archival +**Current diskless behavior in `ReplicaManager.scala`:** -**Concept:** Freeze tiered portion as read-only archive, use S3 lifecycle policies for expiration instead of RLM. +```scala +// applyLocalLeadersDelta - when broker becomes leader +localLeaders.foreachEntry { (tp, info) => + if (!_inklessMetadataView.isDisklessTopic(tp.topic())) + getOrCreatePartition(tp, delta, info.topicId) // SKIPPED for diskless! +} +``` -**Why we considered it:** -- S3 lifecycle policies are simple and battle-tested -- Avoids RLM complexity entirely for expiration -- "Archival" use case doesn't need sophisticated retention +**What this means:** -**Why it fails:** -- **No programmatic retention**: S3 lifecycle can't implement `retention.ms` or `retention.bytes` based on Kafka semantics. Can only do "delete after N days" globally. -- **Topic deletion broken**: Deleting a Kafka topic should clean up tiered data. S3 lifecycle doesn't know about Kafka topics. -- **Doesn't solve PG scalability**: The goal is to tier *new* diskless data to reduce PG load. This alternative only addresses old/archived data. -- **User expectations**: Users expect Kafka retention semantics to work. "Your retention.ms doesn't apply to tiered data" is a poor user experience. +| Aspect | Classic Topic | Diskless Topic (ghost) | +|-------------------------|----------------------------|------------------------------| +| `Partition` objects | Created on each replica | **Never created** | +| Replica fetcher threads | Started for followers | **Never started** | +| ISR tracking | Leader tracks follower lag | **Skipped** (no lag) | -### Rejected Alternative E: Unclean Leader Election for Availability +**The problem:** -**Concept:** Enable `unclean.leader.election.enable=true` for diskless topics to ensure availability when replicas go offline. +```bash +$ kafka-topics.sh --describe --topic diskless-foo -**Why we considered it:** -- For diskless topics, there is no data loss risk from unclean election (all data is in object storage) -- ISR membership is a metadata concept, not a data consistency concept -- Simple configuration change, no new code required -- Standard Kafka mechanism +Topic: diskless-foo Replicas: 1,2,3 ISR: 1,2,3 + ↑ + Looks healthy! + But brokers 2,3 don't know they're replicas +``` -**Why it's not the primary solution:** -- **Doesn't work for RF=1**: Unclean election needs multiple replicas to elect from. With RF=1, there's no alternative replica. -- **Reactive, not proactive**: Waits for election to happen rather than proactively ensuring availability -- **Doesn't preserve rack-awareness**: Elects from existing (possibly degraded) replica set +**Operational risks:** -**What we do instead:** -Auto-reassignment of offline replicas is more powerful: -- Works for any RF (including RF=1) -- Proactively moves replica to online broker -- Preserves rack-awareness when possible -- Metadata-only operation (instant) +| Risk | Impact | +|------------------------------|-------------------------------------------| +| False sense of redundancy | Operator thinks RF=3 provides safety | +| Confusing metrics | Under-replicated = 0, but replicas unused | +| Tooling mismatch | `describe` shows healthy, reality differs | +| Debugging difficulty | "Why isn't replication working?" | -**Note:** Unclean leader election *can* be enabled for diskless topics (no downside), but auto-reassignment is the primary availability mechanism. +This creates technical debt and operational confusion that compounds over time — essentially throwing problems at operators. --- -## Appendix: Topic Migration Interactions - -This section describes how the managed RF design interacts with **topic migration (Tiered Classic → Diskless)**. For full migration mechanics, see [DESIGN.md](DESIGN.md). - -### ISR Semantics During Topic Migration - -When migrating a topic from Tiered Classic to Diskless, ISR semantics transition through phases: - -**Phase 1: Pre-migration (Tiered Classic)** -- Standard Kafka ISR semantics apply -- Replicas must fetch from leader and stay within `replica.lag.time.max.ms` -- Leader manages replication to followers +### Operational Benefits Summary -**Phase 2: During Topic Migration (Sealing)** -- Active segment is rolled and copied to tiered storage -- ISR is maintained normally -- No new writes accepted until migration completes -- All replicas must be in-sync before proceeding +The proposed design (Approach C) provides clear operational benefits compared to alternatives: -**Phase 3: Post-migration (Diskless)** -- New writes go to diskless storage (object storage) -- ISR for diskless portion is "all replicas" (trivially in-sync) -- Reads from tiered portion use existing RLM machinery -- Topic uses diskless ISR semantics going forward - -### Replica Set Changes During Topic Migration - -Topic migration (Tiered Classic → Diskless) may trigger replica set changes: - -**Before migration:** RF=3 with classic placement (leader + 2 followers) - -**After migration:** RF=rack_count with managed placement (one per rack) - -If rack_count differs from original RF: -- Controller adjusts to target placement as part of the migration process -- Uses standard add-then-remove approach -- Paced to avoid disruption - -**Note:** This is a one-time adjustment during topic migration, not ongoing reconciliation. - -### Interaction with Tiering Pipeline - -After topic migration to Diskless: -1. New writes accumulate in diskless storage -2. Aged batches become eligible for tiering (per `local.retention.ms`) -3. Tiering pipeline converts batches to tiered segments -4. Partition leader coordinates segment creation via RLM -5. Batch metadata deleted from PostgreSQL - -The managed RF design ensures a KRaft-managed partition leader exists for step 4. +| Aspect | Approaches A/B (faking) | Approach C (proposed) | +|---------------------------|--------------------------------|--------------------------------| +| Tooling accuracy | `describe topic` misleading | `describe topic` accurate | +| Under-replicated alerts | False positives | Meaningful (informational) | +| Job debugging | "Which broker ran this?" | "Leader of partition X" | +| Incident response | Check all broker logs | Check leader broker | +| Capacity planning | Unpredictable load | Proportional to leadership | +| Standard Kafka ops | May not work as expected | Work normally | --- -## Open Items +### Leadership Benefits for Operations + +Current diskless uses **racing/randomized** job assignment. Leadership provides: -**Resolved in this design:** -- ~~Transformer fallback behavior~~ → Return full replica set (standard Kafka behavior) -- ~~ISR storage~~ → Stored in KRaft as all replicas -- ~~`broker.rack` config changes~~ → Use existing reassignment tooling to fix placement if needed (rare edge case) -- ~~Re-rack-aware tooling~~ → Use existing `kafka-reassign-partitions.sh` (no new tooling needed) +| Benefit | Racing Model | Leader Model | +|-------------------------|----------------------------|-------------------------------| +| Accountability | "Which broker ran this?" | "Leader of partition X" | +| Load distribution | Random, unpredictable | Proportional to leadership | +| Capacity planning | Add brokers, hope it helps | Rebalance partition leaders | +| Incident response | Check all broker logs | Check leader broker | +| Tooling | Custom | Standard Kafka tooling | -**Research (Phase 0):** -- See [Phase 0: Research and Validation](#phase-0-research-and-validation) +Leadership isn't just about RLM — it's about having a **deterministic owner** for partition-level operations. diff --git a/docs/inkless/ts-unification/DISKLESS_MANAGED_RF_FINAL.md b/docs/inkless/ts-unification/DISKLESS_MANAGED_RF_FINAL.md deleted file mode 100644 index 8f65d14f20..0000000000 --- a/docs/inkless/ts-unification/DISKLESS_MANAGED_RF_FINAL.md +++ /dev/null @@ -1,635 +0,0 @@ -# Diskless-Managed Replication Factor - -## Table of Contents - -1. [Purpose](#purpose) -2. [Objectives](#objectives) -3. [Design: Transformer-First Availability](#design-transformer-first-availability) -4. [Activation Model](#activation-model-binary-version) -5. [Placement Model](#placement-model) -6. [Controller Behavior](#controller-behavior) -7. [Broker Behavior](#broker-behavior) -8. [Metadata Transformation](#metadata-transformation) -9. [Observability](#observability) -10. [Implementation Path](#implementation-path) *(6 weeks / 1 eng, 4 weeks / 2 eng)* -11. [Rejected Alternatives](#rejected-alternatives) -12. [Appendix](#appendix) - - [Ghost Replicas Problem](#ghost-replicas-problem) - - [Operational Benefits Summary](#operational-benefits-summary) - - [Leadership Benefits for Operations](#leadership-benefits-for-operations) - ---- - -## Purpose - -### Current State: RF=1 with Transformer Override - -Current diskless topics use **RF=1 in KRaft** with the transformer routing requests to any alive broker. The single replica may point to an **offline or non-existent broker** — the transformer ignores KRaft metadata entirely and hashes to any alive broker. - -``` -Current diskless: - KRaft: Partition 0 → Replicas=[1], Leader=1 ← may be offline/stale - Transformer: Ignores KRaft, hashes to any alive broker -``` - -This was a deliberate decision that enabled: -- Fast iteration without controller changes -- Simple implementation -- "Always available" semantics (transformer routes around any failure) - -**This has worked well for pure diskless topics.** - -### The Question: What Does Topic Migration Require? - -We need to support **bidirectional topic migration**: -1. **Classic → Diskless** (immediate need) -2. **Diskless → Classic** (future need) - -The key question: **how does each approach handle both directions?** - -We evaluate three approaches (A, B, C) below, starting with reverse migration requirements. The proposed design (Approach C) is then detailed in the rest of this document. - -### Approach Comparison - -**Terminology:** -- **Offline replica**: KRaft shows replica on broker X, but broker X doesn't exist or is down. No `Partition` object. -- **Ghost replica**: KRaft shows replica on broker X, broker X is alive, but `ReplicaManager` skips creating `Partition` object for diskless. Broker doesn't know it's a replica. -- **Real (metadata-only) replica**: KRaft shows replica on broker X, broker X recognizes it as a replica, but no local log (data in object storage). - -#### Summary Table - -| Aspect | A: Shrink to RF=1 | B: Keep RF=3, fake | C: RF=rack_count (proposed) | -|-------------------------|--------------------------|--------------------------|------------------------------| -| KRaft RF after migrate | 1 | 3 (unchanged) | 3 (managed, rack-aware) | -| Transformer behavior | Hash to any broker | Hash to any broker | Filter by AZ (KRaft-sourced) | -| Replica objects | None (offline in KRaft) | Ghost (no `Partition`) | Real (metadata-only) | -| Migration coordinator | Custom (who?) | Custom (who?) | **Leader** | -| RLM integration | Custom hooks (~11 wks) | Custom hooks (~11 wks) | **Standard (~0 wks)** | -| Job assignment | Racing | Racing | **Leader-based** | -| Reverse migration ready | No (must expand RF) | No (must fix ghosts) | **Yes** | - -#### Approach A: Shrink to RF=1 During Migration - -``` -Classic: Replicas=[1,2,3], Leader=1, data on all brokers - ↓ migration starts -Shrink: Replicas=[1], Leader=1 ← When? Who coordinates? - ↓ -Diskless: Replicas=[1], data in object storage -``` - -**Problems:** -- Timing of RF shrink is complex (before seal? during? after?) -- Single point of failure during migration -- Loses rack-awareness — original placement lost, must recreate for reverse migration -- **Reverse migration requires RF expansion first** (two operations) - -#### Approach B: Keep Classic RF=3, Continue Faking - -``` -Classic: Replicas=[1,2,3], Leader=1, data on all brokers - ↓ migration starts (no RF change!) -Diskless: Replicas=[1,2,3], Leader=1, data in object storage - ↑ KRaft unchanged, transformer ignores and hashes -``` - -What "ghost replica" means: -- KRaft says broker is a replica with ISR membership -- But `ReplicaManager.isDisklessTopic()` skips `getOrCreatePartition()` -- No `Partition` object, no fetcher, no local log -- Broker doesn't know it's a "replica" — just serves requests via Inkless handlers -- ISR never shrinks (no lag tracking for diskless) - -**Benefit:** If users migrate back to classic, they keep the same partition assignments. - -**Problems:** -- **RLM still blocked** — `onLeadershipChange()` needs `Partition` objects to run tiering tasks. Ghost replicas don't have them. Adding conditional `Partition` creation for diskless would be similar complexity to the proposed approach. -- **Reverse migration requires "un-ghosting"** — must create real `Partition` objects before migrating back -- Misleading tooling output (`describe topic` shows healthy replicas that aren't real) — operational burden on operators -- See [Ghost Replicas Problem](#ghost-replicas-problem) - -#### Approach C: RF=rack_count with Real Replicas (Proposed) - -``` -Classic: Replicas=[1,2,3], Leader=1, data on all brokers - ↓ migration starts -Diskless: Replicas=[1,3,5], Leader=1, data in object storage - ↑ rack-aware placement, real metadata-only replicas - Leader coordinates migration, RLM tasks, background jobs -``` - -What "real (metadata-only) replica" means: -- KRaft says broker is a replica with ISR membership -- Broker recognizes it's a replica for this partition -- No local log (data in object storage) — but could have `Partition` object for coordination -- Leader can run RLM tasks, coordinate migration, own background jobs - -**Benefits:** -- **Leader coordinates migration** — deterministic owner -- **Standard RLM integration** — `onLeadershipChange()` works -- **Deterministic job assignment** — leader owns partition operations -- **Accurate tooling output** — replicas are real -- **Reverse migration ready** — already has RF=3 with real replicas - -### Diskless → Classic Migration Readiness - -| Aspect | A: RF=1 | B: RF=3 faked | C: RF=rack_count | -|-------------------|-----------------------|-------------------------|-------------------------| -| Starting RF | 1 | 3 (ghost replicas) | 3 (real replicas) | -| Before migration | Must expand RF=1 → RF=3| Must "un-ghost" replicas| **Ready as-is** | -| Replica state | Create from scratch | Create `Partition` objects| Already correct | -| Complexity | High (2 operations) | Medium (fix ghosts) | **Low (just migrate)** | - -**Key insight:** Only Approach C is ready for reverse migration without additional work. - -### Cost Analysis - -#### Cost of Keeping RF=1/Faked (Approaches A and B) - -| Aspect | Custom Solution Required | Effort | -|---------------------------|-----------------------------------|-----------| -| Tiering entry point | Bypass `onLeadershipChange()` | ~2 weeks | -| Leader coordination | Custom leader election for tiering| ~2 weeks | -| Segment merging | Custom coordination | ~3 weeks | -| RLM API compatibility | Fake `Partition` objects or fork | ~2 weeks | -| Expiration tasks | Custom expiration outside RLM | ~2 weeks | -| **Total custom work** | | **~11 weeks** | - -#### Cost of Proposed Design (Approach C: RF=rack_count) - -| Aspect | Standard Solution | Effort | -|---------------------------|-----------------------------------|-----------| -| Rack-aware placement | Modify `ReplicationControlManager`| ~2 weeks | -| Transformer filtering | Update `InklessTopicMetadataTransformer` | ~2 weeks | -| Add partitions | Same logic as creation | ~1 week | -| RLM integration | Standard `onLeadershipChange()` | ~0 weeks* | -| **Total work** | | **~6 weeks** | - -*RLM integration comes "for free" once we have real leadership. - -#### Decision Framework - -| Factor | A: Shrink to RF=1 | B: Keep RF=3, fake | C: RF=rack_count (proposed) | -|--------------------------|--------------------------|--------------------------|------------------------------| -| Short-term cost | Lower | Lower | ~6 weeks | -| Classic → Diskless | Custom coordination | Custom coordination | Standard replica handoff | -| RLM integration | Custom hooks (~11 wks) | Custom hooks (~11 wks) | Standard (~0 wks) | -| Diskless → Classic | RF expansion + migration | Un-ghost + migration | Migration only | -| Long-term maintenance | Two code paths | Two code paths | One code path | -| Kafka alignment | Divergent | Divergent | Aligned | - -#### Summary - -``` -Approach A/B: 0 weeks now + ~11 weeks later = 11+ weeks total -Approach C: 6 weeks now + 0 weeks later = 6 weeks total - ───────────── - Savings: 5+ weeks -``` - -Plus, deferring creates technical debt: -- Two code paths for leadership (KRaft vs. custom) -- Custom tiering pipeline that diverges from Kafka -- Every new feature asks "does this work with faked RF?" - -### Recommendation - -**Implement the proposed design (Approach C: RF=rack_count)** because: -1. Topic migration (both directions) benefits from real replicas -2. RLM integration becomes standard rather than custom -3. One-time 6-week investment vs. 11+ weeks of custom work -4. Avoids accumulating technical debt -5. Aligns diskless with Kafka's replica model - -### Related Documents - -- [DESIGN.md](DESIGN.md) — Overall tiered storage unification design - ---- - -## Objectives - -Enable **rack-aware, stable KRaft-managed replicas** for diskless topics: - -- **KRaft-managed replicas**: Diskless topics will have actual KRaft-managed replicas (RF = rack count) -- **Rack-aware at creation**: Topics are created with one replica per rack -- **Standard operations after creation**: Once created, standard Kafka replica management applies -- **Controller-managed RF**: Users don't specify RF; controller computes from rack topology -- **Leader-agnostic produce and consume**: Any replica can accept writes and serve reads -- **Always available via transformer**: Transformer ensures availability by routing to alive brokers - ---- - -## Design: Transformer-First Availability - -### The Insight - -For diskless topics, **availability** and **metadata accuracy** can be decoupled: - -| Concern | Priority | Who Handles | Speed | -|-------------------|--------------|-------------------|------------| -| Availability | Critical | Transformer | Instant | -| Metadata accuracy | Nice-to-have | Controller (lazy) | Eventually | - -### How It Works - -**Current diskless behavior (RF=1, faked):** -- Transformer hashes to any alive broker -- Partition is always available -- KRaft metadata is ignored for routing - -**Proposed behavior (RF=rack_count, real replicas):** -- KRaft stores RF=rack_count with real broker IDs -- Transformer filters by client AZ from KRaft replicas -- **If assigned replica is offline → fall back to any alive broker in AZ** -- Partition is always available (same as today!) -- KRaft metadata may be temporarily stale (shows offline broker) - -### Why This Works for Diskless - -``` -Classic topic: - Data on broker X → client MUST talk to broker X - Broker X offline → partition unavailable (until reassigned) - -Diskless topic: - Data in object storage → ANY broker can serve - Broker X offline → transformer routes to broker Y - Partition stays available (no reassignment needed!) -``` - -The transformer already has all the information it needs: -1. KRaft replica assignments (preferred brokers) -2. Alive brokers in cluster -3. Client AZ - -It can make instant routing decisions without waiting for controller. - ---- - -## Activation Model (Binary Version) - -Managed RF is activated by **deploying a new binary version**. No feature flags, no topic attributes, no server configs. - -### Behavior Summary - -**Old binary:** -- Diskless topics use legacy "RF=1 / faked metadata" behavior -- Transformer calculates synthetic placement via hashing - -**New binary:** -- Diskless topics use KRaft-managed placement (one replica per rack at creation) -- Transformer filters KRaft placement by client AZ -- **Falls back to alive brokers if assigned replicas are offline** - -### Existing Diskless Topics - -Existing diskless topics (RF=1) continue to work: - -**Immediate availability:** -- Transformer sees RF=1 with potentially offline broker -- Falls back to hash-based selection (same as today) -- No downtime - -**Eventual modernization (optional):** -- Controller can lazily detect RF=1 diskless topics -- Background task expands to RF=rack_count when convenient -- Not urgent — availability is already handled by transformer - -**Manual modernization (alternative):** -- Operator uses `kafka-reassign-partitions.sh` to expand RF -- Same tooling as classic topics - ---- - -## Placement Model - -### Rack-Aware Placement at Creation - -When a diskless topic is created: -- Controller determines current rack count from registered brokers -- RF is set to rack count (e.g., 3 racks → RF=3) -- One broker is selected per rack for each partition -- Broker selection within a rack uses load balancing (least loaded broker) - -### Placement Is Static After Creation - -Once a topic is created: -- RF does not automatically change if racks are added/removed -- Replica assignments don't automatically adjust -- Standard Kafka replica management applies - ---- - -## Controller Behavior - -### Topic Creation - -When creating diskless topics (`diskless.enable=true`): -- Controller counts distinct racks from registered brokers -- RF = rack count -- One replica assigned per rack -- Reject `replicationFactor > 1` and manual assignments - -### Add Partitions - -New partitions use same one-per-rack logic as creation. - -### Standard Operations (After Creation) - -**No auto-reassignment on broker failure.** Transformer handles availability. - -**Leader Election:** -- Standard Kafka leader election from ISR -- For diskless topics, all replicas are always in ISR - -**Broker Failure:** -- Replicas on failed broker become offline in KRaft metadata -- **Transformer routes around offline broker** (instant) -- Partition remains available -- KRaft shows under-replicated (informational only for diskless) -- When broker returns, it's back in ISR immediately - -**Reassignment:** -- `kafka-reassign-partitions.sh` works normally -- Operator can reassign replicas if desired -- **Not required for availability** — transformer handles routing - ---- - -## Broker Behavior - -- No replica fetcher threads for diskless topics -- No local `Partition` objects (for now — may add for RLM coordination later) -- Inkless handlers (`AppendHandler`, `FetchHandler`) serve requests - ---- - -## Metadata Transformation - -### Filtering Logic - -``` -FOR each diskless partition: - assigned_replicas = KRaft replica set - alive_replicas = assigned_replicas ∩ alive_brokers - - IF alive_replicas is not empty: - # Normal case: use KRaft placement - IF any alive_replica in clientAZ: - RETURN local replica (AZ-aware routing) - ELSE: - RETURN all alive_replicas (cross-AZ fallback) - ELSE: - # All assigned replicas offline: fall back to hash - RETURN hash-based selection from all alive brokers in clientAZ -``` - -### Key Properties - -1. **Instant availability**: No waiting for controller -2. **AZ-aware when possible**: Uses KRaft placement if alive -3. **Graceful degradation**: Falls back to hash if needed -4. **No metadata dependency for availability**: Stale metadata doesn't cause downtime - ---- - -## Observability - -### Metrics - -**Controller metrics:** -- `kafka.controller.diskless.effective_rf{topic}` - RF assigned at creation -- `kafka.controller.diskless.rack_aware{topic,partition}` - 1 if one-per-rack, 0 if not -- `kafka.controller.diskless.rf1_topics_total` - Legacy topics not yet modernized - -**Transformer metrics:** -- `kafka.server.diskless.transformer.fallback_total` - Count of hash fallbacks -- `kafka.server.diskless.transformer.offline_replicas_routed_around` - Routing decisions - -**Standard Kafka metrics:** -- `UnderReplicatedPartitions` - Will show diskless partitions with offline brokers -- Note: For diskless, "under-replicated" is informational, not critical - -### Logs - -``` -INFO [Transformer] Diskless partition 'foo-0' has offline replica (broker 101), - routing to alive replica (broker 103) - -WARN [Transformer] Diskless partition 'foo-0' has ALL replicas offline, - using hash-based fallback to broker 105 -``` - -### Operational Notes - -**`kafka-topics.sh --describe` may show stale info:** -``` -Topic: foo PartitionCount: 3 ReplicationFactor: 3 - Partition: 0 Leader: 101 Replicas: 101,103,105 Isr: 101,103,105 - ↑ - Broker 101 may be offline, but transformer routes to 103 -``` - -This is acceptable because: -1. Availability is not affected -2. Operator can see actual state via broker metrics -3. Eventually consistent (when broker returns or is reassigned) - ---- - -## Implementation Path - -**Total Estimate: 6 weeks with 1 engineer, or 4 weeks with 2 engineers** - -### Phase 0: Research and Validation (1 week) - -1. Leader election works without broker `Partition` objects -2. ISR updates don't require broker-side `Partition` state -3. `DescribeTopics` / `ListOffsets` work with RF > 1 - -### Phase 1: Topic Creation with Rack-Aware Placement (2 weeks) - -1. Modify `ReplicationControlManager` to detect diskless topics -2. Compute RF = rack count at creation -3. Implement one-per-rack broker selection -4. Reject `replicationFactor > 1` and manual assignments - -### Phase 2: Transformer Changes (2 weeks) - -1. Update `InklessTopicMetadataTransformer` to read KRaft placement -2. Implement AZ filtering logic -3. Add offline replica routing — if KRaft replica offline, route to alive replica -4. Add hash fallback — if all replicas offline, use legacy hash -5. Add metrics for fallback tracking - -### Phase 3: Add Partitions Support (1 week) - -1. Apply same one-per-rack logic when adding partitions -2. Handle case where rack count changed since topic creation - -### Summary - -| Phase | Scope | Estimate | -|-----------|------------------------------------------|----------| -| 0 | Research and Validation | 1 week | -| 1 | Topic Creation with Rack-Aware Placement | 2 weeks | -| 2 | Transformer Changes (with fallback) | 2 weeks | -| 3 | Add Partitions Support | 1 week | -| **Total** | | **6 weeks (1 eng) / 4 weeks (2 eng)** | - ---- - -## Rejected Alternatives - -### Alternative A: Shrink RF During Migration - -**Concept:** When migrating Classic → Diskless, shrink RF=3 to RF=1, then use current diskless model. - -**Why considered:** -- Maintains consistency with current RF=1 diskless model -- No need to change existing diskless implementation - -**Why rejected:** -1. Timing complexity (when does RF shrink happen?) -2. Loses rack-awareness — must recreate for reverse migration -3. **Reverse migration requires RF expansion** (two operations) -4. RLM integration still blocked -5. Racing job assignment continues - -See [Approach Comparison](#approach-comparison) for details. - ---- - -### Alternative B: Keep Classic RF, Continue Faking Metadata - -**Concept:** Keep existing RF (e.g., RF=3) in KRaft but continue using transformer to fake metadata. - -**Why considered:** -- Zero controller changes for migration -- Simplest migration path -- If users migrate back, they keep same partition assignments - -**Why rejected:** -1. **Ghost replicas** — KRaft shows healthy replicas that aren't real -2. **RLM blocked** — `onLeadershipChange()` needs `Partition` objects. Adding them would be similar complexity to proposed approach. -3. **Reverse migration requires "un-ghosting"** replicas -4. Misleading tooling output -5. Racing job assignment continues - -See [Ghost Replicas Problem](#ghost-replicas-problem) for details. - ---- - -### Alternative C: Controller Auto-Reassignment - -**Concept:** Controller proactively detects offline replicas and reassigns to online brokers immediately. - -**Why considered:** -- KRaft metadata always accurate -- Standard tooling shows correct state - -**Why rejected:** -- **3 weeks additional complexity** for something transformer already handles -- **Instant vs eventually consistent** — transformer is faster -- **Consistent with today** — diskless already uses transformer for availability -- **Fewer failure modes** — no controller state machine to debug - -**When to reconsider:** -- If KRaft metadata accuracy becomes critical requirement -- If operators require `describe topic` to always reflect reality - ---- - -### Alternative D: Custom Job Coordinator - -**Concept:** Keep faking metadata, add custom coordinator (outside KRaft) for background jobs. - -**Why considered:** -- Gets deterministic job ownership without changing replica model -- Could use PG-based leader election - -**Why rejected:** -1. **Two leadership systems** — KRaft vs. custom coordinator -2. **RLM still blocked** — RLM uses KRaft leadership -3. **More complexity** — three systems (KRaft + transformer + coordinator) -4. **Defeats purpose** — if adding coordinator, why not use KRaft's? - ---- - -## Appendix - -### Ghost Replicas Problem - -If we keep RF=3 in KRaft but transformer ignores it, Kafka creates "ghost replicas": - -**Current diskless behavior in `ReplicaManager.scala`:** - -```scala -// applyLocalLeadersDelta - when broker becomes leader -localLeaders.foreachEntry { (tp, info) => - if (!_inklessMetadataView.isDisklessTopic(tp.topic())) - getOrCreatePartition(tp, delta, info.topicId) // SKIPPED for diskless! -} -``` - -**What this means:** - -| Aspect | Classic Topic | Diskless Topic (ghost) | -|-------------------------|----------------------------|------------------------------| -| `Partition` objects | Created on each replica | **Never created** | -| Replica fetcher threads | Started for followers | **Never started** | -| ISR tracking | Leader tracks follower lag | **Skipped** (no lag) | - -**The problem:** - -```bash -$ kafka-topics.sh --describe --topic diskless-foo - -Topic: diskless-foo Replicas: 1,2,3 ISR: 1,2,3 - ↑ - Looks healthy! - But brokers 2,3 don't know they're replicas -``` - -**Operational risks:** - -| Risk | Impact | -|------------------------------|-------------------------------------------| -| False sense of redundancy | Operator thinks RF=3 provides safety | -| Confusing metrics | Under-replicated = 0, but replicas unused | -| Tooling mismatch | `describe` shows healthy, reality differs | -| Debugging difficulty | "Why isn't replication working?" | - -This creates technical debt and operational confusion that compounds over time — essentially throwing problems at operators. - ---- - -### Operational Benefits Summary - -The proposed design (Approach C) provides clear operational benefits compared to alternatives: - -| Aspect | Approaches A/B (faking) | Approach C (proposed) | -|---------------------------|--------------------------------|--------------------------------| -| Tooling accuracy | `describe topic` misleading | `describe topic` accurate | -| Under-replicated alerts | False positives | Meaningful (informational) | -| Job debugging | "Which broker ran this?" | "Leader of partition X" | -| Incident response | Check all broker logs | Check leader broker | -| Capacity planning | Unpredictable load | Proportional to leadership | -| Standard Kafka ops | May not work as expected | Work normally | - ---- - -### Leadership Benefits for Operations - -Current diskless uses **racing/randomized** job assignment. Leadership provides: - -| Benefit | Racing Model | Leader Model | -|-------------------------|----------------------------|-------------------------------| -| Accountability | "Which broker ran this?" | "Leader of partition X" | -| Load distribution | Random, unpredictable | Proportional to leadership | -| Capacity planning | Add brokers, hope it helps | Rebalance partition leaders | -| Incident response | Check all broker logs | Check leader broker | -| Tooling | Custom | Standard Kafka tooling | - -Leadership isn't just about RLM — it's about having a **deterministic owner** for partition-level operations. diff --git a/docs/inkless/ts-unification/DISKLESS_MANAGED_RF_SIMPLIFIED.md b/docs/inkless/ts-unification/DISKLESS_MANAGED_RF_SIMPLIFIED.md deleted file mode 100644 index 9c273bf1bd..0000000000 --- a/docs/inkless/ts-unification/DISKLESS_MANAGED_RF_SIMPLIFIED.md +++ /dev/null @@ -1,1109 +0,0 @@ -# Diskless-Managed Replication Factor (Simplified Design) - -> **This is a draft alternative to [DISKLESS_MANAGED_RF.md](DISKLESS_MANAGED_RF.md)** that reduces complexity by using transformer-first availability instead of controller auto-reassignment. - -## Table of Contents - -1. [Purpose](#purpose) -2. [Objectives](#objectives) -3. [Key Simplification: Transformer-First Availability](#key-simplification-transformer-first-availability) -4. [Activation Model](#activation-model-binary-version) - - [Existing Diskless Topics](#existing-diskless-topics) -5. [Placement Model](#placement-model) -6. [Controller Behavior](#controller-behavior) - - [Topic Creation](#topic-creation) - - [Add Partitions](#add-partitions) - - [Standard Operations](#standard-operations-after-creation) -7. [Broker Behavior](#broker-behavior) -8. [Metadata Transformation](#metadata-transformation) -9. [Observability](#observability) -10. [Implementation Path](#implementation-path) *(6 weeks / 1 eng, 4 weeks / 2 eng)* -11. [Trade-offs vs Original Design](#trade-offs-vs-original-design) -12. [Rejected Alternatives](#rejected-alternatives) - - [F: Keep Classic RF, Continue Faking](#rejected-alternative-f-keep-classic-rf-continue-faking-metadata) - - [G: Controller Auto-Reassignment](#rejected-alternative-g-controller-auto-reassignment-from-original-design) - - [H: Shrink RF During Migration](#rejected-alternative-h-shrink-rf-during-migration) - - [I: Custom Job Coordinator](#rejected-alternative-i-custom-job-coordinator-without-kraft-leadership) - ---- - -## Purpose - -### Current State: RF=1 with Transformer Override - -Current diskless topics use **RF=1 in KRaft** with the transformer routing requests to any alive broker. The single replica may point to an **offline or non-existent broker** — the transformer ignores KRaft metadata entirely and hashes to any alive broker. - -``` -Current diskless: - KRaft: Partition 0 → Replicas=[1], Leader=1 ← may be offline/stale - Transformer: Ignores KRaft, hashes to any alive broker -``` - -This was a deliberate decision that enabled: -- Fast iteration without controller changes -- Simple implementation -- "Always available" semantics (transformer routes around any failure) - -**This has worked well for pure diskless topics.** - -### The Question: What Does Topic Migration Require? - -We need to support **bidirectional topic migration**: -1. **Classic → Diskless** (immediate need) -2. **Diskless → Classic** (future need) - -The key question: **how does each approach handle reverse migration?** - -#### Diskless → Classic Migration (Future) - -Before comparing approaches for Classic → Diskless, consider what's needed for the reverse: - -| Aspect | A: RF=1 | B: RF=3 faked | C: RF=rack_count | -|-------------------|-----------------------|-------------------------|-------------------------| -| Starting RF | 1 | 3 (ghost replicas) | 3 (real replicas) | -| Before migration | Must expand RF=1 → RF=3| Must "un-ghost" replicas| **Ready as-is** | -| Replica state | Create from scratch | Create `Partition` objects| Already correct | -| Complexity | High (2 operations) | Medium (fix ghosts) | **Low (just migrate)** | - -**Key insight:** Only approach C is ready for reverse migration without additional work. - -#### How Critical Is Replica-Awareness for Topic Migration? - -Now compare three approaches for Classic → Diskless migration: - -| Aspect | A: Shrink to RF=1 | B: Keep RF=3, fake | C: RF=rack_count (this) | -|-------------------------|--------------------------|--------------------------|--------------------------| -| KRaft RF after migrate | 1 | 3 (unchanged) | 3 (managed, rack-aware) | -| Transformer behavior | Hash to any broker | Hash to any broker | Filter by AZ (KRaft-sourced) | -| Replica objects | None (offline in KRaft) | Ghost (no `Partition`) | Real (metadata-only) | -| Migration coordinator | Custom (who?) | Custom (who?) | **Leader** | -| RLM integration | Custom hooks | Custom hooks | **Standard** | -| Job assignment | Racing | Racing | **Leader-based** | -| Reverse migration ready | No (must expand RF) | No (must fix ghosts) | **Yes** | - -**Terminology:** -- **Offline replica**: KRaft shows replica on broker X, but broker X doesn't exist or is down. No `Partition` object. -- **Ghost replica**: KRaft shows replica on broker X, broker X is alive, but `ReplicaManager` skips creating `Partition` object for diskless. Broker doesn't know it's a replica. -- **Real (metadata-only) replica**: KRaft shows replica on broker X, broker X recognizes it as a replica, but no local log (data in object storage). - -**Approach A: Shrink to RF=1 during migration** - -``` -Classic: Replicas=[1,2,3], Leader=1, data on all brokers - ↓ migration starts -Shrink: Replicas=[1], Leader=1 ← When does this happen? - ↓ Who removes replicas 2,3? - ↓ What if broker 1 fails mid-shrink? -Diskless: Replicas=[1], data in object storage - Transformer hashes to any alive broker (ignores KRaft) -``` - -Problems: -- Timing of RF shrink is complex (before seal? during? after?) -- Single point of failure during migration -- Loses rack-awareness from classic topic -- **Reverse migration requires RF expansion first** - -**Approach B: Keep classic RF=3, continue faking metadata** - -``` -Classic: Replicas=[1,2,3], Leader=1, data on all brokers - ↓ migration starts (no RF change!) -Diskless: Replicas=[1,2,3], Leader=1, data in object storage - ↑ KRaft unchanged, transformer ignores and hashes - Brokers 1,2,3 are "ghost replicas" -``` - -What "ghost replica" means: -- KRaft says broker is a replica with ISR membership -- But `ReplicaManager.isDisklessTopic()` skips `getOrCreatePartition()` -- No `Partition` object, no fetcher, no local log -- Broker doesn't know it's a "replica" — just serves requests via Inkless handlers -- ISR never shrinks (no lag tracking for diskless) - -Problems: -- **RLM still blocked** — `onLeadershipChange()` needs `Partition` objects to run tiering tasks. Ghost replicas don't have them. -- **Reverse migration requires "un-ghosting"** — must create real `Partition` objects before migrating back -- Side effects: misleading tooling output (`describe topic` shows healthy replicas that aren't real) -- See [What Happens to "Ignored" Replicas?](#what-happens-to-ignored-replicas) - -**Approach C: RF=rack_count with real replicas (this design)** - -``` -Classic: Replicas=[1,2,3], Leader=1, data on all brokers - ↓ migration starts -Diskless: Replicas=[1,3,5], Leader=1, data in object storage - ↑ rack-aware placement, real metadata-only replicas - Leader coordinates migration, RLM tasks, background jobs -``` - -What "real (metadata-only) replica" means: -- KRaft says broker is a replica with ISR membership -- Broker recognizes it's a replica for this partition -- No local log (data in object storage) — but could have `Partition` object for coordination -- Leader can run RLM tasks, coordinate migration, own background jobs - -Benefits: -- **Leader coordinates migration** — deterministic owner -- **Standard RLM integration** — `onLeadershipChange()` works -- **Deterministic job assignment** — leader owns partition operations -- **Accurate tooling output** — replicas are real -- **Reverse migration ready** — already has RF=3 with real replicas - -### Cost of Continuing with RF=1 - -If we keep RF=1 and use transformer override for topic migration: - -**Challenges:** - -1. **When does RF shrink happen?** - - Classic topic has RF=3 with replicas [1,2,3] - - Control Plane marks migration point, seals the topic - - But when do we remove replicas 2 and 3? - - Before seal? During? After data is in object storage? - - What if broker 1 fails mid-transition? - -2. **Migration coordination without real leadership:** - - With RF=1, the single broker "owns" the migration - - If it fails, how do we pick a new coordinator? - - Custom leader election outside KRaft? Or use KRaft but ignore it elsewhere? - -3. **RLM integration blocked:** - - `onLeadershipChange()` won't fire without real KRaft leadership - - Tiering pipeline needs a leader to coordinate segment merging - - Custom hooks needed to bypass RLM's standard entry points - - See [RLM Integration: Critical Decision](#rlm-integration-critical-decision) below - -4. **Diskless → Classic complexity:** - - Must expand RF=1 → RF=3 as part of migration - - Two complex operations combined: RF expansion + data migration - - More failure modes, harder to reason about - -5. **Alternative: Keep existing placement, keep faking?** - - Classic RF=3 becomes diskless RF=3, but transformer still fakes - - Zero metadata changes, simplest migration - - But: RLM still blocked, tooling still confused - - Defers the problem, doesn't solve it - - See [What Happens to "Ignored" Replicas?](#what-happens-to-ignored-replicas) below - -**What RF=1 continuation requires:** -- Custom migration coordinator (not partition leader) -- Custom RLM integration hooks (bypass `onLeadershipChange()`) -- RF expansion logic embedded in Diskless → Classic migration -- Ongoing maintenance of parallel code paths - -### Cost of Moving to RF=rack_count - -**One-time investment:** -- Controller changes for rack-aware placement (~2 weeks) -- Transformer changes for AZ filtering (~2 weeks) -- Testing and validation (~2 weeks) - -**What we get:** -- Topic migration uses standard replica handoff patterns -- RLM integration uses standard `onLeadershipChange()` -- Diskless → Classic migration starts from correct RF -- Single code path for replica management - -### Decision Framework - -| Factor | Keep RF=1 | Move to RF=rack_count | -|--------------------------|--------------------------|----------------------------| -| Short-term cost | Lower | ~6 weeks | -| Classic → Diskless | Custom coordination | Standard replica handoff | -| RLM integration | Custom hooks | Standard leadership hook | -| Diskless → Classic | RF expansion + migration | Migration only | -| Long-term maintenance | Two code paths | One code path | -| Kafka alignment | Divergent | Aligned | - -### What Happens to "Ignored" Replicas? - -If we keep RF=3 in KRaft but the transformer ignores it, what does Kafka do with those replicas? - -**Current diskless behavior in `ReplicaManager.scala`:** - -```scala -// applyLocalLeadersDelta - when broker becomes leader -localLeaders.foreachEntry { (tp, info) => - if (!_inklessMetadataView.isDisklessTopic(tp.topic())) - getOrCreatePartition(tp, delta, info.topicId) // SKIPPED for diskless! -} - -// applyLocalFollowersDelta - when broker becomes follower -localFollowers.foreachEntry { (tp, info) => - if (!_inklessMetadataView.isDisklessTopic(tp.topic())) - getOrCreatePartition(tp, delta, info.topicId) // SKIPPED for diskless! -} -``` - -**What this means:** - -| Aspect | Classic Topic | Diskless Topic (current) | -|-------------------------|----------------------------|------------------------------| -| `Partition` objects | Created on each replica | **Never created** | -| Local log directories | Created on each replica | **Never created** | -| Replica fetcher threads | Started for followers | **Never started** | -| ISR tracking | Leader tracks follower lag | **Skipped** (no lag to track)| -| Leader election | Uses `Partition` state | KRaft metadata only | - -**The "ghost replica" problem:** - -If KRaft says `Replicas=[1,2,3]` but diskless skips `Partition` creation: - -``` -KRaft metadata: - Partition 0: Replicas=[1,2,3], Leader=1, ISR=[1,2,3] - -Broker 1 (leader): - - No Partition object for this topic - - No local log - - Serves requests via Inkless handlers - -Broker 2 (follower): - - No Partition object - - No fetcher thread (nothing to fetch) - - Doesn't know it's a "follower" - -Broker 3 (follower): - - Same as broker 2 -``` - -**Will they become offline?** - -| Scenario | What Happens | -|------------------------------|-------------------------------------------| -| Broker 2 restarts | KRaft still shows it as replica, ISR=all | -| Broker 2 decommissioned | KRaft shows offline, but no data lost | -| Leader election triggered | KRaft picks new leader, but no `Partition`| -| ISR shrink check | **Skipped** for diskless (no lag to check)| - -**Key insight:** Replicas won't become "offline" in the Kafka sense because: -1. ISR shrink is skipped for diskless topics -2. No fetcher means no lag detection -3. KRaft ISR stays as "all replicas" forever - -**But this creates confusion:** - -```bash -$ kafka-topics.sh --describe --topic diskless-foo - -Topic: diskless-foo Replicas: 1,2,3 ISR: 1,2,3 - ↑ - Looks healthy! - But brokers 2,3 aren't really "in sync" - They don't even know they're replicas -``` - -**The operational risk:** - -| Risk | Impact | -|------------------------------|-------------------------------------------| -| False sense of redundancy | Operator thinks RF=3 provides safety | -| Confusing metrics | Under-replicated = 0, but replicas unused | -| Tooling mismatch | `describe` shows healthy, reality differs | -| Decommission confusion | Remove broker 2, ISR still shows it | -| Debugging difficulty | "Why isn't replication working?" | - -**Summary:** - -Keeping replicas but ignoring them creates **ghost replicas**: -- KRaft thinks they exist and are healthy -- Brokers don't know they're replicas -- ISR never shrinks (no lag tracking) -- Operators get misleading information -- No actual redundancy benefit - -This is a form of **technical debt** that compounds over time. - ---- - -### Current Diskless Job Assignment: Racing Brokers - -**How diskless background jobs work today:** - -Diskless has background jobs (merger, cleaner, etc.) that need to run periodically. Currently, these use a **racing/randomized** approach: - -``` -Current model (no leader): - Broker 0: "I'll try to clean partition X" ──┐ - Broker 1: "I'll try to clean partition X" ──┼── Race! First one wins - Broker 2: "I'll try to clean partition X" ──┘ - - Coordination via: PostgreSQL locks, randomized delays, distributed claims -``` - -**Why this was chosen:** -- Simpler implementation (no leader election needed) -- Works without real KRaft replicas -- Good enough for initial diskless MVP - -**The operational complexity:** - -| Aspect | Racing Model | -|---------------------------|---------------------------------------| -| Job assignment | Random/first-come-first-serve | -| Duplicate work | Possible (mitigated by PG locks) | -| Debugging | "Which broker ran this job?" | -| Monitoring | Jobs scattered across brokers | -| Failure handling | Another broker eventually picks up | -| Load distribution | Probabilistic, not guaranteed | - -**What leadership enables:** - -With real KRaft-managed leaders, background jobs have a **home**: - -``` -Leader-based model: - Partition X leader = Broker 1 - - Broker 0: "Not leader of X, skip" - Broker 1: "I'm leader of X, I run jobs for X" ← Deterministic! - Broker 2: "Not leader of X, skip" - - Coordination via: KRaft leadership (standard Kafka) -``` - -| Aspect | Leader Model | -|---------------------------|---------------------------------------| -| Job assignment | Deterministic (leader owns it) | -| Duplicate work | Impossible (single leader) | -| Debugging | "Leader of partition X ran this" | -| Monitoring | Jobs grouped by leader | -| Failure handling | Leader election, new leader takes over| -| Load distribution | Controlled by partition assignment | - -**Jobs that benefit from leadership:** - -| Job | Current (racing) | With Leader | -|---------------------------|----------------------------|---------------------------------| -| Batch merging | Any broker can try | Leader merges its partitions | -| Segment cleanup | Race to delete | Leader cleans its partitions | -| Tiering (diskless→TS) | Complex coordination | Leader uploads via RLM | -| Expiration | Distributed claims | Leader runs RLM expiration | -| Offset tracking | Scattered | Leader maintains offsets | - -**The TS migration benefit:** - -Tiered Storage migration requires uploading segments in TS format. With leadership: - -``` -Without leader (current): - Who uploads segment for partition X? - → Need custom coordinator - → Race conditions possible - → Custom logic to track progress - -With leader: - Partition X leader uploads segments - → Reuse RLM upload logic directly - → Standard Kafka pattern - → Progress tracked via RLM APIs -``` - -**Key insight:** - -Leadership isn't just about RLM integration — it's about having a **deterministic owner** for partition-level operations. This simplifies: -- Implementation (reuse Kafka patterns) -- Operations (clear ownership) -- Debugging (know who did what) -- Future features (upload, compaction, etc.) - -### Operational Wins from Leadership - -**1. Clear accountability:** -``` -Racing model: - Alert: "Partition X merger failed" - Operator: "Which broker? Let me check all broker logs..." - -Leader model: - Alert: "Partition X merger failed on broker 2 (leader)" - Operator: "Check broker 2 logs for partition X" -``` - -**2. Predictable load distribution:** -``` -Racing model: - Broker 0: 45% of merger jobs (got lucky) - Broker 1: 35% of merger jobs - Broker 2: 20% of merger jobs (got unlucky) - -Leader model: - Broker 0: Jobs for partitions it leads (balanced by assignment) - Broker 1: Jobs for partitions it leads - Broker 2: Jobs for partitions it leads -``` - -**3. Simpler capacity planning:** - -| Aspect | Racing | Leader | -|---------------------|--------------------------------|----------------------------------| -| Job count per broker| Unpredictable | = partitions led | -| Resource usage | Spiky, varies by luck | Proportional to leadership | -| Scaling decision | "Add brokers, hope it helps" | "Rebalance partition leaders" | - -**4. Easier incident response:** - -| Scenario | Racing | Leader | -|---------------------------|----------------------------------|----------------------------------| -| Job stuck | Which broker has it? | Leader of partition X | -| Job running slow | Check all brokers | Check leader broker | -| Want to pause jobs | Disable on all brokers | Move leadership away | -| Debug job history | Correlate logs across brokers | Single broker's logs | - -**5. Standard Kafka operations:** - -```bash -# Racing model: No good way to move jobs -# "Jobs run wherever, hope for the best" - -# Leader model: Standard Kafka tooling -kafka-leader-election.sh --topic foo --partition 0 --election-type preferred -# Jobs for partition 0 now run on the new leader -``` - -**6. Monitoring alignment with Kafka:** - -| Metric | Racing | Leader | -|-------------------------------|-------------------------|-------------------------------| -| Jobs per broker | Custom metric | Partition leadership count | -| Job failures | Custom alerting | Under-replicated partitions | -| Load imbalance | Custom detection | Leader skew metrics | - -**Summary: Operational benefits of leadership** - -| Benefit | Impact | -|----------------------------|------------------------------------------------| -| Single point of truth | Leader owns partition, no ambiguity | -| Standard tooling | `kafka-leader-election`, `kafka-reassign` | -| Predictable load | Jobs proportional to leadership | -| Simpler debugging | One broker to check per partition | -| Kafka-native monitoring | Reuse existing metrics and alerts | -| Incident response | Move leadership to isolate/debug | - ---- - -### RLM Integration: Critical Decision - -**Why RLM matters for diskless:** - -The tiering pipeline (diskless batches → tiered segments) is critical for PostgreSQL scalability. Without it, all batch metadata stays in PG forever, which becomes the bottleneck. - -``` -Current diskless data flow: - Produce → Object Storage (WAL) → PostgreSQL (batch metadata) - ↑ - This grows unbounded! - -Target diskless data flow: - Produce → Object Storage (WAL) → PostgreSQL (recent batches) - ↓ tiering pipeline - Tiered Storage (aged batches) - ↓ RLM expiration - Deleted (per retention policy) -``` - -**How RLM works:** - -```java -// RemoteLogManager.java - entry point -public void onLeadershipChange(Set partitions, ...) { - for (Partition partition : partitions) { - // Start tiering tasks for this partition - // Only runs on the LEADER - scheduleTieringTasks(partition); - } -} -``` - -RLM requires: -1. **Real KRaft leadership** — `onLeadershipChange()` must fire -2. **`Partition` objects** — RLM uses `Partition` and `UnifiedLog` APIs -3. **Leader coordination** — Only leader writes tiered segments - -**Cost of deferring RLM integration (keeping RF=1/faked):** - -| Aspect | Custom Solution Required | Effort | -|---------------------------|-----------------------------------|-----------| -| Tiering entry point | Bypass `onLeadershipChange()` | ~2 weeks | -| Leader coordination | Custom leader election for tiering| ~2 weeks | -| Segment merging | Who merges? How to coordinate? | ~3 weeks | -| RLM API compatibility | Fake `Partition` objects or fork | ~2 weeks | -| Expiration tasks | Custom expiration outside RLM | ~2 weeks | -| **Total custom work** | | **~11 weeks** | - -**Cost of doing RF=rack_count now:** - -| Aspect | Standard Solution | Effort | -|---------------------------|-----------------------------------|-----------| -| Rack-aware placement | Modify `ReplicationControlManager`| ~2 weeks | -| Transformer filtering | Update `InklessTopicMetadataTransformer` | ~2 weeks | -| Add partitions | Same logic as creation | ~1 week | -| RLM integration | Standard `onLeadershipChange()` | ~0 weeks* | -| **Total work** | | **~6 weeks** | - -*RLM integration comes "for free" once we have real leadership. - -**The math:** - -``` -Option A: Keep RF=1, defer RLM = 0 weeks now + 11 weeks later = 11 weeks total -Option B: Do RF=rack_count now = 6 weeks now + 0 weeks later = 6 weeks total - ───────────────────────────── - Savings: 5 weeks -``` - -**But the real cost is worse:** - -If we defer, we carry technical debt: -- Two code paths for leadership (KRaft vs. custom) -- Custom tiering pipeline that diverges from Kafka -- Harder to maintain, harder to reason about -- Every new feature asks "does this work with faked RF?" - -### Recommendation - -**Move to RF=rack_count now** because: -1. Topic migration (both directions) benefits from real replicas -2. RLM integration becomes standard rather than custom -3. One-time 6-week investment vs. 11+ weeks of custom work -4. Avoids accumulating technical debt -5. Aligns diskless with Kafka's replica model - -### Related Documents - -- [DESIGN.md](DESIGN.md) — Overall tiered storage unification design (includes topic migration, tiering pipeline details) - ---- - -## Objectives - -Enable **rack-aware, stable KRaft-managed replicas** for diskless topics: - -- **KRaft-managed replicas**: Diskless topics will have actual KRaft-managed replicas (RF = rack count) -- **Rack-aware at creation**: Topics are created with one replica per rack -- **Standard operations after creation**: Once created, standard Kafka replica management applies -- **Controller-managed RF**: Users don't specify RF; controller computes from rack topology -- **Leader-agnostic produce and consume**: Any replica can accept writes and serve reads -- **Always available via transformer**: Transformer ensures availability by routing to alive brokers - -**Key difference from original design:** Availability is handled by the transformer (instant), not controller auto-reassignment. Metadata accuracy is eventual. - ---- - -## Key Simplification: Transformer-First Availability - -### The Insight - -For diskless topics, **availability** and **metadata accuracy** can be decoupled: - -| Concern | Priority | Who Handles | Speed | -|-------------------|--------------|-------------------|------------| -| Availability | Critical | Transformer | Instant | -| Metadata accuracy | Nice-to-have | Controller (lazy) | Eventually | - -### How It Works - -**Current diskless behavior (RF=1, faked):** -- Transformer hashes to any alive broker -- Partition is always available -- KRaft metadata is ignored for routing - -**Simplified managed RF behavior:** -- KRaft stores RF=rack_count with real broker IDs -- Transformer filters by client AZ from KRaft replicas -- **If assigned replica is offline → fall back to any alive broker in AZ** -- Partition is always available (same as today!) -- KRaft metadata may be temporarily stale (shows offline broker) - -### Why This Works for Diskless - -``` -Classic topic: - Data on broker X → client MUST talk to broker X - Broker X offline → partition unavailable (until reassigned) - -Diskless topic: - Data in object storage → ANY broker can serve - Broker X offline → transformer routes to broker Y - Partition stays available (no reassignment needed!) -``` - -The transformer already has all the information it needs: -1. KRaft replica assignments (preferred brokers) -2. Alive brokers in cluster -3. Client AZ - -It can make instant routing decisions without waiting for controller. - ---- - -## Activation Model (Binary Version) - -*Unchanged from original design.* - -Managed RF is activated by **deploying a new binary version**. No feature flags, no topic attributes, no server configs. - -### Behavior Summary - -**Old binary:** -- Diskless topics use legacy "RF=1 / faked metadata" behavior -- Transformer calculates synthetic placement via hashing - -**New binary:** -- Diskless topics use KRaft-managed placement (one replica per rack at creation) -- Transformer filters KRaft placement by client AZ -- **Falls back to alive brokers if assigned replicas are offline** - -### Existing Diskless Topics - -Existing diskless topics (RF=1) continue to work: - -**Immediate availability:** -- Transformer sees RF=1 with potentially offline broker -- Falls back to hash-based selection (same as today) -- No downtime - -**Eventual modernization (optional):** -- Controller can lazily detect RF=1 diskless topics -- Background task expands to RF=rack_count when convenient -- Not urgent — availability is already handled by transformer - -**Manual modernization (alternative):** -- Operator uses `kafka-reassign-partitions.sh` to expand RF -- Same tooling as classic topics - -```bash -# Optional: manually expand legacy topic to rack-aware -kafka-reassign-partitions.sh --bootstrap-server localhost:9092 \ - --reassignment-json-file reassignment.json \ - --execute -``` - -**Key difference from original:** No urgent auto-migration needed. Transformer handles availability immediately. - ---- - -## Placement Model - -*Unchanged from original design.* - -### Rack-Aware Placement at Creation - -When a diskless topic is created: -- Controller determines current rack count from registered brokers -- RF is set to rack count (e.g., 3 racks → RF=3) -- One broker is selected per rack for each partition -- Broker selection within a rack uses load balancing (least loaded broker) - -### Placement Is Static After Creation - -Once a topic is created: -- RF does not automatically change if racks are added/removed -- Replica assignments don't automatically adjust -- Standard Kafka replica management applies - ---- - -## Controller Behavior - -### Topic Creation - -*Unchanged from original design.* - -When creating diskless topics (`diskless.enable=true`): -- Controller counts distinct racks from registered brokers -- RF = rack count -- One replica assigned per rack -- Reject `replicationFactor > 1` and manual assignments - -### Add Partitions - -*Unchanged from original design.* - -New partitions use same one-per-rack logic as creation. - -### Standard Operations (After Creation) - -**Key difference: No auto-reassignment on broker failure.** - -**Leader Election:** -- Standard Kafka leader election from ISR -- For diskless topics, all replicas are always in ISR - -**Broker Failure:** -- Replicas on failed broker become offline in KRaft metadata -- **Transformer routes around offline broker** (instant) -- Partition remains available -- KRaft shows under-replicated (informational only for diskless) -- When broker returns, it's back in ISR immediately - -**Reassignment:** -- `kafka-reassign-partitions.sh` works normally -- Operator can reassign replicas if desired -- **Not required for availability** — transformer handles routing - -### What We Remove - -No controller logic for: -- ❌ Proactive offline replica detection -- ❌ Immediate auto-reassignment -- ❌ "Not registered vs offline" distinction -- ❌ Pacing controls for reassignment -- ❌ Complex state machine - ---- - -## Broker Behavior - -*Unchanged from original design.* - -- No replica fetcher threads for diskless topics -- No local `Partition` objects -- Inkless handlers (`AppendHandler`, `FetchHandler`) serve requests - ---- - -## Metadata Transformation - -### Updated Filtering Logic - -``` -FOR each diskless partition: - assigned_replicas = KRaft replica set - alive_replicas = assigned_replicas ∩ alive_brokers - - IF alive_replicas is not empty: - # Normal case: use KRaft placement - IF any alive_replica in clientAZ: - RETURN local replica (AZ-aware routing) - ELSE: - RETURN all alive_replicas (cross-AZ fallback) - ELSE: - # All assigned replicas offline: fall back to hash - RETURN hash-based selection from all alive brokers in clientAZ -``` - -### Key Properties - -1. **Instant availability**: No waiting for controller -2. **AZ-aware when possible**: Uses KRaft placement if alive -3. **Graceful degradation**: Falls back to hash if needed -4. **No metadata dependency for availability**: Stale metadata doesn't cause downtime - -### Comparison - -| Scenario | Original Design | Simplified Design | -|----------------------|----------------------------|------------------------------| -| Broker fails | Wait for controller reassign| Instant transformer fallback | -| All replicas offline | Wait for controller | Instant hash fallback | -| Client routing | After KRaft updated | Immediate | - ---- - -## Observability - -### Metrics - -**Controller metrics:** -- `kafka.controller.diskless.effective_rf{topic}` - RF assigned at creation -- `kafka.controller.diskless.rack_aware{topic,partition}` - 1 if one-per-rack, 0 if not -- `kafka.controller.diskless.rf1_topics_total` - Legacy topics not yet modernized - -**Transformer metrics (new):** -- `kafka.server.diskless.transformer.fallback_total` - Count of hash fallbacks -- `kafka.server.diskless.transformer.offline_replicas_routed_around` - Routing decisions - -**Standard Kafka metrics:** -- `UnderReplicatedPartitions` - Will show diskless partitions with offline brokers -- Note: For diskless, "under-replicated" is informational, not critical - -### Logs - -``` -INFO [Transformer] Diskless partition 'foo-0' has offline replica (broker 101), - routing to alive replica (broker 103) - -WARN [Transformer] Diskless partition 'foo-0' has ALL replicas offline, - using hash-based fallback to broker 105 -``` - -### Admin Surfaces - -**`kafka-topics.sh --describe` may show stale info:** -``` -Topic: foo PartitionCount: 3 ReplicationFactor: 3 - Partition: 0 Leader: 101 Replicas: 101,103,105 Isr: 101,103,105 - ↑ - Broker 101 may be offline, but transformer routes to 103 -``` - -This is acceptable because: -1. Availability is not affected -2. Operator can see actual state via broker metrics -3. Eventually consistent (when broker returns or is reassigned) - ---- - -## Implementation Path - -**Total Estimate: 6 weeks with 1 engineer, or 4 weeks with 2 engineers** - -### Phase 0: Research and Validation (1 week) - -*Same as original.* - -1. Leader election works without broker `Partition` objects -2. ISR updates don't require broker-side `Partition` state -3. `DescribeTopics` / `ListOffsets` work with RF > 1 - -### Phase 1: Topic Creation with Rack-Aware Placement (2 weeks) - -*Same as original.* - -1. Modify `ReplicationControlManager` to detect diskless topics -2. Compute RF = rack count at creation -3. Implement one-per-rack broker selection -4. Reject `replicationFactor > 1` and manual assignments - -### Phase 2: Transformer Changes (2 weeks) - -Updated scope: - -1. Update `InklessTopicMetadataTransformer` to read KRaft placement -2. Implement AZ filtering logic -3. **Add offline replica routing** — if KRaft replica offline, route to alive replica -4. **Add hash fallback** — if all replicas offline, use legacy hash -5. Add metrics for fallback tracking - -### Phase 3: Add Partitions Support (1 week) - -*Same as original.* - -1. Apply same one-per-rack logic when adding partitions -2. Handle case where rack count changed since topic creation - -### ~~Phase 4: Offline Replica Auto-Reassignment~~ (REMOVED) - -**Not needed.** Transformer handles availability. - -### ~~Phase 5: Observability~~ → Phase 4: Observability (included in Phase 2) - -Simplified — mostly transformer metrics. - -### Summary - -| Phase | Scope | Estimate | -|-----------|------------------------------------------|--------------------------------| -| 0 | Research and Validation | 1 week | -| 1 | Topic Creation with Rack-Aware Placement | 2 weeks | -| 2 | Transformer Changes (with fallback) | 2 weeks | -| 3 | Add Partitions Support | 1 week | -| **Total** | | **6 weeks (1 eng) / 4 weeks (2 eng)** | - -**Savings: 3 weeks** (removed Phase 4: Auto-Reassignment) - ---- - -## Trade-offs vs Original Design - -### What We Gain - -| Benefit | Impact | -|-------------------------------|---------------------------------------------| -| Simpler implementation | 3 weeks saved, less code to maintain | -| No controller complexity | No state machine, pacing, edge cases | -| Instant availability | Transformer decides immediately | -| Consistent with current | Transformer fallback = what diskless does | -| Fewer moving parts | Less to debug, less to break | - -### What We Accept - -| Trade-off | Impact | Mitigation | -|---------------------------|---------------------------------|----------------------------------| -| Stale KRaft metadata | `describe` shows offline broker | Transformer metrics show reality | -| Under-replicated alerts | Metrics fire for diskless | Document as informational | -| Manual legacy modernization| RF=1 stays until reassigned | Provide runbook | -| No auto rack restoration | Broken rack-awareness persists | Same as original (operator fixes)| - -### When Original Design Is Better - -Choose original design if: -- KRaft metadata accuracy is critical for tooling/automation -- You want `describe topic` to always show current state -- Operators expect under-replicated alerts to always require action - -Choose simplified design if: -- You prioritize simplicity and faster delivery -- You accept "eventually consistent" metadata -- Current diskless transformer behavior is acceptable baseline - ---- - -## Rejected Alternatives - -*Includes alternatives from original design, plus alternatives specific to the "keep faking" approach.* - -### Rejected Alternative F: Keep Classic RF, Continue Faking Metadata - -**Concept:** When migrating Classic → Diskless, keep the existing RF (e.g., RF=3) in KRaft but continue using the transformer to fake metadata. No replica management changes. - -**Variations considered:** - -1. **Keep RF=3, transformer ignores:** Classic RF=3 becomes diskless RF=3, transformer hashes to any alive broker -2. **Shrink to RF=1 during migration:** Classic RF=3 shrinks to RF=1, then uses current diskless model -3. **Keep placement, add custom coordination:** Keep RF=3, add custom job coordinator outside KRaft - -**Why we considered it:** -- Zero controller changes for migration -- Simplest migration path (just flip `diskless.enable`) -- Works with existing diskless implementation -- Faster time to initial migration - -**Why we rejected it:** - -1. **Ghost replicas problem:** - - KRaft shows `Replicas=[1,2,3], ISR=[1,2,3]` - - But brokers 2,3 have no `Partition` objects, no fetchers - - ISR never shrinks (no lag tracking for diskless) - - Operators get misleading information - - See [What Happens to "Ignored" Replicas?](#what-happens-to-ignored-replicas) - -2. **RLM integration blocked:** - - `onLeadershipChange()` won't fire without real leadership - - Custom hooks needed (~11 weeks vs ~6 weeks for real replicas) - - See [RLM Integration: Critical Decision](#rlm-integration-critical-decision) - -3. **Racing job assignment continues:** - - Background jobs (merger, cleaner) remain randomized - - No deterministic owner for partition operations - - See [Current Diskless Job Assignment: Racing Brokers](#current-diskless-job-assignment-racing-brokers) - -4. **RF shrink complexity:** - - If shrinking RF=3 → RF=1, when does this happen? - - Who coordinates the shrink? What if coordinator fails? - - See [Cost of Continuing with RF=1](#cost-of-continuing-with-rf1) - -5. **Diskless → Classic migration harder:** - - Must expand RF=1 → RF=3 as part of reverse migration - - Two complex operations combined - -6. **Technical debt compounds:** - - Two systems: KRaft replicas (classic) vs transformer magic (diskless) - - Every new feature asks "does this work with faked RF?" - - Harder to maintain, harder to onboard - -**Cost comparison:** - -| Approach | Now | Later | Total | -|-------------------------|---------|----------|-----------| -| Keep faking + custom RLM| 0 weeks | ~11 weeks| 11+ weeks | -| RF=rack_count now | 6 weeks | 0 weeks | 6 weeks | - -**When this might make sense:** -- If RLM integration is not needed (PG can scale indefinitely) -- If bidirectional migration is not needed -- If operational confusion from ghost replicas is acceptable -- None of these apply to our use case - ---- - -### Rejected Alternative G: Controller Auto-Reassignment (from original design) - -**Concept:** Controller proactively detects offline replicas and reassigns to online brokers immediately. - -**Why we considered it:** -- KRaft metadata always accurate -- Standard tooling shows correct state -- Consistent with future direction (KRaft as source of truth) - -**Why we chose transformer-first instead:** -- **3 weeks additional complexity** for something transformer already handles -- **Instant vs eventually consistent** — transformer is faster -- **Consistent with today** — diskless already uses transformer for availability -- **Fewer failure modes** — no controller state machine to debug - -**When to reconsider:** -- If KRaft metadata accuracy becomes critical requirement -- If RLM integration needs accurate replica state for leader selection -- If operators require `describe topic` to always reflect reality - ---- - -### Rejected Alternative H: Shrink RF During Migration - -**Concept:** When migrating Classic → Diskless, shrink RF=3 to RF=1 as part of the migration process, then use current diskless model. - -**Why we considered it:** -- Maintains consistency with current RF=1 diskless model -- No need to change existing diskless implementation -- Single replica = simpler mental model - -**Why we rejected it:** - -1. **Timing complexity:** - - When does RF shrink happen? Before seal? During? After? - - Who removes replicas 2 and 3? - - What if coordinator fails mid-shrink? - -2. **Reverse migration harder:** - - Diskless → Classic requires RF=1 → RF=3 expansion - - Two operations: expand RF + migrate data - -3. **Loses rack-awareness:** - - Classic topics are often rack-aware (RF=3 across 3 racks) - - Shrinking to RF=1 loses this property - - Must re-establish rack-awareness later - -4. **All problems of RF=1:** - - Ghost replicas if we keep KRaft RF=3 - - RLM integration blocked - - Racing job assignment - ---- - -### Rejected Alternative I: Custom Job Coordinator (Without KRaft Leadership) - -**Concept:** Keep faking metadata, but add a custom coordinator (outside KRaft) to assign background jobs deterministically. - -**Why we considered it:** -- Gets deterministic job ownership without changing replica model -- Could use PG-based leader election -- Avoids KRaft changes - -**Why we rejected it:** - -1. **Two leadership systems:** - - KRaft has its own leader concept - - Custom coordinator has different leader - - Which one is authoritative? - -2. **RLM still blocked:** - - RLM uses KRaft leadership, not custom coordinator - - Still need custom hooks for tiering - -3. **More complexity, not less:** - - Now have KRaft + transformer + custom coordinator - - Three systems to keep consistent - -4. **Defeats the purpose:** - - If we're adding a coordinator, why not use KRaft's? - - KRaft leadership is battle-tested, custom is not - ---- - -## Migration Path: Original → Simplified - -If we start with simplified and later need original: - -1. Simplified design is a **subset** of original -2. Can add controller auto-reassignment later without breaking changes -3. Transformer fallback remains as safety net - -If we start with original and want to simplify: - -1. Remove controller auto-reassignment code -2. Transformer already has fallback logic -3. Accept stale metadata - -**Recommendation:** Start with simplified. Add controller complexity only if proven necessary. From 655a305ba9c974c8cfa493d47d848233b61dd3b7 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Wed, 14 Jan 2026 11:30:04 +0200 Subject: [PATCH 08/21] fixup! add diskless managed rf feature design --- .../ts-unification/DISKLESS_MANAGED_RF.md | 186 +++++++++++++++--- 1 file changed, 154 insertions(+), 32 deletions(-) diff --git a/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md b/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md index 8f65d14f20..25935ca363 100644 --- a/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md +++ b/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md @@ -2,7 +2,7 @@ ## Table of Contents -1. [Purpose](#purpose) +1. [Motivation](#motivation) 2. [Objectives](#objectives) 3. [Design: Transformer-First Availability](#design-transformer-first-availability) 4. [Activation Model](#activation-model-binary-version) @@ -11,7 +11,7 @@ 7. [Broker Behavior](#broker-behavior) 8. [Metadata Transformation](#metadata-transformation) 9. [Observability](#observability) -10. [Implementation Path](#implementation-path) *(6 weeks / 1 eng, 4 weeks / 2 eng)* +10. [Implementation Path](#implementation-path) *(6-8 weeks / 1 eng)* 11. [Rejected Alternatives](#rejected-alternatives) 12. [Appendix](#appendix) - [Ghost Replicas Problem](#ghost-replicas-problem) @@ -20,7 +20,7 @@ --- -## Purpose +## Motivation ### Current State: RF=1 with Transformer Override @@ -51,6 +51,13 @@ We evaluate three approaches (A, B, C) below, starting with reverse migration re ### Approach Comparison +We evaluate three approaches: +- **Approach A: Shrink to RF=1** — Reduce RF during migration, use current diskless model +- **Approach B: Keep RF=3, fake metadata** — Keep classic RF, but transformer ignores it +- **Approach C: RF=rack_count with real replicas** — Proposed design with KRaft-managed replicas + +The following terminology helps distinguish the key differences: + **Terminology:** - **Offline replica**: KRaft shows replica on broker X, but broker X doesn't exist or is down. No `Partition` object. - **Ghost replica**: KRaft shows replica on broker X, broker X is alive, but `ReplicaManager` skips creating `Partition` object for diskless. Broker doesn't know it's a replica. @@ -63,6 +70,7 @@ We evaluate three approaches (A, B, C) below, starting with reverse migration re | KRaft RF after migrate | 1 | 3 (unchanged) | 3 (managed, rack-aware) | | Transformer behavior | Hash to any broker | Hash to any broker | Filter by AZ (KRaft-sourced) | | Replica objects | None (offline in KRaft) | Ghost (no `Partition`) | Real (metadata-only) | +| Local state after migr. | **Must delete** (risk) | **Must delete** (risk) | Standard handoff | | Migration coordinator | Custom (who?) | Custom (who?) | **Leader** | | RLM integration | Custom hooks (~11 wks) | Custom hooks (~11 wks) | **Standard (~0 wks)** | | Job assignment | Racing | Racing | **Leader-based** | @@ -133,12 +141,12 @@ What "real (metadata-only) replica" means: ### Diskless → Classic Migration Readiness -| Aspect | A: RF=1 | B: RF=3 faked | C: RF=rack_count | -|-------------------|-----------------------|-------------------------|-------------------------| -| Starting RF | 1 | 3 (ghost replicas) | 3 (real replicas) | -| Before migration | Must expand RF=1 → RF=3| Must "un-ghost" replicas| **Ready as-is** | -| Replica state | Create from scratch | Create `Partition` objects| Already correct | -| Complexity | High (2 operations) | Medium (fix ghosts) | **Low (just migrate)** | +| Aspect | A: RF=1 | B: RF=3 faked | C: RF=rack_count | +|-------------------|-------------------------|---------------------------|---------------------------| +| Starting RF | 1 | 3 (ghost replicas) | 3 (real replicas) | +| Before migration | Must expand RF=1 → RF=3 | Must "un-ghost" replicas | **Ready as-is** | +| Replica state | Create from scratch | Create `Partition` objects| Already correct | +| Complexity | High (2 operations) | Medium (fix ghosts) | **Low (just migrate)** | **Key insight:** Only Approach C is ready for reverse migration without additional work. @@ -146,6 +154,44 @@ What "real (metadata-only) replica" means: #### Cost of Keeping RF=1/Faked (Approaches A and B) +Approaches A and B appear simpler but have **hidden upfront costs** related to local state management: + +**The local state problem:** + +When migrating Classic → Diskless, brokers still have local log directories with segment files. Diskless handlers ignore local state, but: + +| Problem | Risk | Severity | +|-----------------------------|-----------------------------------------------|--------------| +| Stale local metadata | Partition metadata files (LEO, etc.) diverge from object storage | **Critical** | +| Orphaned segment files | Requires tiering to object storage + retention cleanup | Medium | +| Log cleaner interaction | May attempt to clean diskless partitions | High | +| Broker restart recovery | Which is source of truth? Local metadata or object storage? | **Critical** | +| Replica fetcher state | Followers may have stale fetch positions | High | + +**Note:** Some of these issues (metadata sync, fetcher state) must be solved for *all* approaches. However, Approach C addresses them via standard replica handoff patterns, while A/B require custom one-off fixes. + +**Required work for A/B to be safe:** + +| Task | Effort | +|-------------------------------------|-----------| +| Sync local metadata → object storage| ~1-2 weeks| +| Tier local segments to object storage| ~1 week | +| Delete local logs after tiering | ~0.5 week | +| Prevent log cleaner on diskless | ~0.5 week | +| Block replica fetcher startup | ~0.5 week | +| Handle broker restart (source of truth) | ~1 week | +| Custom TS fetch path (no `UnifiedLog`) | ~2-3 weeks | +| **Subtotal (local state + TS)** | **~6-8 weeks** | + +This is **not** near-zero — it's ~6-8 weeks of custom work just to make migration safe, with ongoing risk if any edge case is missed. + +**Additional deferred cost** (for future features): +- Diskless → Classic migration (reverse direction) +- RLM integration for tiering pipeline +- KIP-aligned features + +The table below shows the deferred cost for RLM integration: + | Aspect | Custom Solution Required | Effort | |---------------------------|-----------------------------------|-----------| | Tiering entry point | Bypass `onLeadershipChange()` | ~2 weeks | @@ -167,39 +213,64 @@ What "real (metadata-only) replica" means: *RLM integration comes "for free" once we have real leadership. +**Buffer for unknowns:** Add ~1-2 weeks for edge cases, testing, and unexpected issues. + +**Realistic total: ~7-8 weeks** + +**Why local state is simpler with Approach C:** + +With real replicas, the migration can use standard Kafka replica handoff patterns: +- **Leader coordinates** data movement to object storage (deterministic owner) +- **Metadata sync** is part of standard replica protocol (LEO, offsets tracked by leader) +- **Local logs cleanup** follows standard tiering (RLM-style) — not custom deletion +- **Broker restart** has clear source of truth: leader + object storage +- **Fetcher state** managed via standard replica protocol (followers follow leader) + +The same problems exist, but Approach C solves them via **standard Kafka patterns** rather than custom one-off fixes. + #### Decision Framework | Factor | A: Shrink to RF=1 | B: Keep RF=3, fake | C: RF=rack_count (proposed) | |--------------------------|--------------------------|--------------------------|------------------------------| -| Short-term cost | Lower | Lower | ~6 weeks | +| Short-term cost | ~7 weeks (local + TS) | ~7 weeks (local + TS) | ~6-8 weeks (incl. buffer) | +| Local state risk | **High** (must cleanup) | **High** (must cleanup) | Low (standard handoff) | | Classic → Diskless | Custom coordination | Custom coordination | Standard replica handoff | | RLM integration | Custom hooks (~11 wks) | Custom hooks (~11 wks) | Standard (~0 wks) | | Diskless → Classic | RF expansion + migration | Un-ghost + migration | Migration only | | Long-term maintenance | Two code paths | Two code paths | One code path | -| Kafka alignment | Divergent | Divergent | Aligned | +| Kafka/KIP alignment | Divergent | Divergent | Aligned | #### Summary ``` -Approach A/B: 0 weeks now + ~11 weeks later = 11+ weeks total -Approach C: 6 weeks now + 0 weeks later = 6 weeks total - ───────────── - Savings: 5+ weeks +Approach A/B: ~7 weeks now (local state + TS) + ~11 weeks later (RLM) = 18+ weeks total +Approach C: ~6 weeks now + ~2 weeks buffer + 0 weeks later = 8 weeks total + ───────────── + Savings: 10+ weeks ``` +**Plus ongoing risk with A/B:** Custom local state cleanup and TS fetch has edge cases — if any are missed, data corruption, metadata drift, or fetch failures can occur during/after migration. + Plus, deferring creates technical debt: - Two code paths for leadership (KRaft vs. custom) - Custom tiering pipeline that diverges from Kafka - Every new feature asks "does this work with faked RF?" +**Operational cost of A/B:** +- Misleading tooling (`describe topic` shows ghost replicas) +- Debugging complexity ("which broker ran this job?") +- False alerts (under-replicated metrics for healthy topics) +- See [Operational Benefits Summary](#operational-benefits-summary) + ### Recommendation **Implement the proposed design (Approach C: RF=rack_count)** because: 1. Topic migration (both directions) benefits from real replicas 2. RLM integration becomes standard rather than custom -3. One-time 6-week investment vs. 11+ weeks of custom work -4. Avoids accumulating technical debt -5. Aligns diskless with Kafka's replica model +3. One-time 6-8 week investment vs. 15+ weeks with A/B +4. Avoids local state cleanup risks (data corruption, offset drift) +5. Avoids accumulating technical debt +6. Aligns diskless with Kafka's replica model ### Related Documents @@ -207,6 +278,8 @@ Plus, deferring creates technical debt: --- +*The following sections detail the proposed design (Approach C: RF=rack_count with real replicas).* + ## Objectives Enable **rack-aware, stable KRaft-managed replicas** for diskless topics: @@ -224,7 +297,11 @@ Enable **rack-aware, stable KRaft-managed replicas** for diskless topics: ### The Insight -For diskless topics, **availability** and **metadata accuracy** can be decoupled: +For diskless topics, **availability** and **metadata accuracy** can be decoupled. + +Unlike classic topics where data lives on specific brokers (making KRaft metadata critical for routing), diskless data lives in object storage — any broker can serve any partition. This means: +- **Availability** can be handled instantly by the transformer (route to any alive broker) +- **Metadata accuracy** can be eventually consistent (controller updates KRaft when convenient) | Concern | Priority | Who Handles | Speed | |-------------------|--------------|-------------------|------------| @@ -291,6 +368,46 @@ Existing diskless topics (RF=1) continue to work: - Falls back to hash-based selection (same as today) - No downtime +**Note on RF < rack_count:** + +The same fallback logic applies to any RF < rack_count (including RF=1). This is intentional: + +| Aspect | Design Decision | Rationale | +|--------|-----------------|-----------| +| RF=1 valid? | **Yes** | Fewer replicas = fewer brokers with jobs, but still available | +| Target RF | rack_count at creation | Default, not enforced after creation | +| RF < racks after ops | **Allowed** | Operator can reassign; transformer handles availability | +| Future configurability | Possible | Could allow `min.insync.replicas` style config later | + +**Tiered Storage (TS) reading — Important Consideration:** + +TS fetching via `RemoteLogManager.read()` requires local state that current diskless doesn't have: + +| Requirement | What RLM needs | Current diskless | Approach C | +|-------------|----------------|------------------|------------| +| `UnifiedLog` | `fetchLog.apply(tp)` must return log | **Missing** (no `Partition`) | Has it (real replicas) | +| `LeaderEpochFileCache` | For segment lookup | **Missing** | Has it | +| Segment metadata | To locate tiered data | **Missing** | Has it | + +**Impact on topic migration:** + +During Classic → Diskless migration, the read path must handle tiered data: +``` +Offset < disklessStartOffset → Read from Tiered Storage (via RLM) +Offset >= disklessStartOffset → Read from Diskless (object storage) +``` + +With Approaches A/B: +- Broker has no `UnifiedLog` for diskless partitions +- `RLM.read()` fails with `OffsetOutOfRangeException` +- **Custom TS fetch path required** (~2-3 weeks additional work) + +With Approach C: +- Real replicas have `Partition` → `UnifiedLog` → works with RLM +- Standard TS fetch path works + +**Note:** Pure diskless topics (never migrated) use Inkless `FetchHandler` which reads from object storage directly — no RLM dependency. The issue is specifically for **migrated topics** that have tiered data. + **Eventual modernization (optional):** - Controller can lazily detect RF=1 diskless topics - Background task expands to RF=rack_count when convenient @@ -329,7 +446,9 @@ When creating diskless topics (`diskless.enable=true`): - Controller counts distinct racks from registered brokers - RF = rack count - One replica assigned per rack -- Reject `replicationFactor > 1` and manual assignments +- Accept `replicationFactor=-1` (recommended) or `replicationFactor=1` (for compatibility) +- Reject `replicationFactor > 1` (RF is system-managed) +- Reject manual replica assignments ### Add Partitions @@ -440,7 +559,7 @@ This is acceptable because: ## Implementation Path -**Total Estimate: 6 weeks with 1 engineer, or 4 weeks with 2 engineers** +**Total Estimate: 6-8 weeks with 1 engineer** (includes ~1-2 weeks buffer for unknowns) ### Phase 0: Research and Validation (1 week) @@ -476,7 +595,8 @@ This is acceptable because: | 1 | Topic Creation with Rack-Aware Placement | 2 weeks | | 2 | Transformer Changes (with fallback) | 2 weeks | | 3 | Add Partitions Support | 1 week | -| **Total** | | **6 weeks (1 eng) / 4 weeks (2 eng)** | +| Buffer | Edge cases, testing, unknowns | 1-2 weeks| +| **Total** | | **6-8 weeks** | --- @@ -491,11 +611,12 @@ This is acceptable because: - No need to change existing diskless implementation **Why rejected:** -1. Timing complexity (when does RF shrink happen?) -2. Loses rack-awareness — must recreate for reverse migration -3. **Reverse migration requires RF expansion** (two operations) -4. RLM integration still blocked -5. Racing job assignment continues +1. **Local state cleanup required** — Must delete local logs on all replicas to avoid drift/corruption (~3-4 weeks) +2. Timing complexity (when does RF shrink happen?) +3. Loses rack-awareness — must recreate for reverse migration +4. **Reverse migration requires RF expansion** (two operations) +5. RLM integration still blocked +6. Racing job assignment continues See [Approach Comparison](#approach-comparison) for details. @@ -511,11 +632,12 @@ See [Approach Comparison](#approach-comparison) for details. - If users migrate back, they keep same partition assignments **Why rejected:** -1. **Ghost replicas** — KRaft shows healthy replicas that aren't real -2. **RLM blocked** — `onLeadershipChange()` needs `Partition` objects. Adding them would be similar complexity to proposed approach. -3. **Reverse migration requires "un-ghosting"** replicas -4. Misleading tooling output -5. Racing job assignment continues +1. **Local state cleanup required** — Must delete local logs on all replicas to avoid drift/corruption (~3-4 weeks) +2. **Ghost replicas** — KRaft shows healthy replicas that aren't real +3. **RLM blocked** — `onLeadershipChange()` needs `Partition` objects. Adding them would be similar complexity to proposed approach. +4. **Reverse migration requires "un-ghosting"** replicas +5. Misleading tooling output +6. Racing job assignment continues See [Ghost Replicas Problem](#ghost-replicas-problem) for details. From 946acdc72d00a6330f2312f7db71ea7774f223ad Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Wed, 14 Jan 2026 12:41:13 +0200 Subject: [PATCH 09/21] fixup! add diskless managed rf feature design --- .../ts-unification/DISKLESS_MANAGED_RF.md | 47 ++++++++++++++----- 1 file changed, 36 insertions(+), 11 deletions(-) diff --git a/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md b/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md index 25935ca363..573e0d1607 100644 --- a/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md +++ b/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md @@ -303,6 +303,8 @@ Unlike classic topics where data lives on specific brokers (making KRaft metadat - **Availability** can be handled instantly by the transformer (route to any alive broker) - **Metadata accuracy** can be eventually consistent (controller updates KRaft when convenient) +**Scope note:** This decoupling applies to partitions in a **diskless-only** state. During tiered→diskless migration (hybrid/tiered state), reads may require a broker with `UnifiedLog`/RLM state, so routing must stay within the KRaft replica set (Kafka semantics). + | Concern | Priority | Who Handles | Speed | |-------------------|--------------|-------------------|------------| | Availability | Critical | Transformer | Instant | @@ -406,7 +408,17 @@ With Approach C: - Real replicas have `Partition` → `UnifiedLog` → works with RLM - Standard TS fetch path works -**Note:** Pure diskless topics (never migrated) use Inkless `FetchHandler` which reads from object storage directly — no RLM dependency. The issue is specifically for **migrated topics** that have tiered data. +**Implication for metadata transformation (offset-unaware):** + +Kafka **Metadata** responses do not include fetch offsets, so the transformer cannot know whether a client will read: +- Tiered/local range (must be served by a broker with `UnifiedLog`/RLM state), or +- Diskless range (can be served by any broker). + +Therefore the transformer must be conservative **per partition state**: +- **If tiered reads are possible** (migration state is `TIERED_ONLY` / `HYBRID` per `DESIGN.md`), the transformer must return only brokers in the **KRaft replica set** (so TS reads work). +- **Only when the partition is `DISKLESS_ONLY`** (no tiered reads), the transformer may route to any alive broker for availability. + +**Note:** Pure diskless topics (never migrated) use Inkless `FetchHandler` which reads from object storage directly — no RLM dependency. The TS limitation matters during/around migration when a partition still has tiered/local data. **Eventual modernization (optional):** - Controller can lazily detect RF=1 diskless topics @@ -490,26 +502,39 @@ New partitions use same one-per-rack logic as creation. ``` FOR each diskless partition: + # Important: Metadata responses are offset-unaware, so routing must be + # conservative when tiered reads are possible. + mode = migrationState(tp) # DISKLESS_ONLY vs HYBRID/TIERED_ONLY + assigned_replicas = KRaft replica set alive_replicas = assigned_replicas ∩ alive_brokers - IF alive_replicas is not empty: - # Normal case: use KRaft placement - IF any alive_replica in clientAZ: - RETURN local replica (AZ-aware routing) + IF mode != DISKLESS_ONLY: + # Tiered reads may be required -> must stay on replicas that have UnifiedLog/RLM state + IF alive_replicas is not empty: + RETURN alive_replicas (prefer clientAZ, else cross-AZ) ELSE: - RETURN all alive_replicas (cross-AZ fallback) + RETURN assigned_replicas (Kafka semantics: partition unavailable until replica returns) ELSE: - # All assigned replicas offline: fall back to hash - RETURN hash-based selection from all alive brokers in clientAZ + # Diskless-only: any broker can serve, so we can preserve "always available" + IF alive_replicas is not empty: + # Normal case: use KRaft placement + IF any alive_replica in clientAZ: + RETURN local replica (AZ-aware routing) + ELSE: + RETURN all alive_replicas (cross-AZ fallback) + ELSE: + # All assigned replicas offline: fall back to hash + RETURN hash-based selection from all alive brokers in clientAZ ``` ### Key Properties -1. **Instant availability**: No waiting for controller +1. **Instant availability (diskless-only)**: No waiting for controller when `DISKLESS_ONLY` 2. **AZ-aware when possible**: Uses KRaft placement if alive -3. **Graceful degradation**: Falls back to hash if needed -4. **No metadata dependency for availability**: Stale metadata doesn't cause downtime +3. **Graceful degradation (diskless-only)**: Falls back to hash if needed +4. **Tiered-safe routing (hybrid/tiered)**: When tiered reads are possible, only replica brokers are returned +5. **Availability semantics are state-dependent**: Hybrid/tiered behaves like Kafka; diskless-only is always available --- From 6fc71de18101d8959b57d90237ad9aac9f7b1054 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Wed, 14 Jan 2026 12:56:41 +0200 Subject: [PATCH 10/21] fixup! add diskless managed rf feature design --- .../ts-unification/DISKLESS_MANAGED_RF.md | 101 ++++++++++++++++++ 1 file changed, 101 insertions(+) diff --git a/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md b/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md index 573e0d1607..960cc1ebaf 100644 --- a/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md +++ b/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md @@ -56,6 +56,21 @@ We evaluate three approaches: - **Approach B: Keep RF=3, fake metadata** — Keep classic RF, but transformer ignores it - **Approach C: RF=rack_count with real replicas** — Proposed design with KRaft-managed replicas +**Key decision (affects this whole design): do we require tiered reads to be served only by replica brokers?** + +Kafka **Metadata** responses are offset-unaware, but Tiered Storage reads via `RemoteLogManager` require local `UnifiedLog` +state on the serving broker. This creates a hard constraint: +- When a partition can serve **tiered/local reads** (migration state `TIERED_ONLY` / `HYBRID`), metadata transformation must + return only brokers in the **KRaft replica set** (Kafka semantics). +- Only when a partition is **`DISKLESS_ONLY`** can we safely decouple “availability routing” from KRaft placement and route to + any alive broker (transformer-first availability). + +This is a deliberate trade-off: it keeps the “always available” property for diskless-only partitions, while preserving +correctness for tiered/hybrid reads. + +If we later decide to make **tiered reads serveable from any broker**, then we can remove this constraint and strengthen the +decoupling (see “Decision Option: Tiered reads from any broker” below). + The following terminology helps distinguish the key differences: **Terminology:** @@ -69,6 +84,7 @@ The following terminology helps distinguish the key differences: |-------------------------|--------------------------|--------------------------|------------------------------| | KRaft RF after migrate | 1 | 3 (unchanged) | 3 (managed, rack-aware) | | Transformer behavior | Hash to any broker | Hash to any broker | Filter by AZ (KRaft-sourced) | +| Hybrid/tiered reads | **Custom / risky** | **Custom / risky** | **Replicas only (safe)** | | Replica objects | None (offline in KRaft) | Ghost (no `Partition`) | Real (metadata-only) | | Local state after migr. | **Must delete** (risk) | **Must delete** (risk) | Standard handoff | | Migration coordinator | Custom (who?) | Custom (who?) | **Leader** | @@ -76,6 +92,32 @@ The following terminology helps distinguish the key differences: | Job assignment | Racing | Racing | **Leader-based** | | Reverse migration ready | No (must expand RF) | No (must fix ghosts) | **Yes** | +#### Migration RF Normalization (Classic → Diskless) + +Another key decision: **if a classic topic’s RF is not aligned with `rack_count`, when do we “normalize” it?** + +This matters because diskless-managed RF assumes replicas are meaningful for **leadership/job ownership** and for +**tiered/hybrid correctness** (replica brokers host `UnifiedLog` state used by RLM). + +We have three options: + +| Option | What happens | Pros | Cons | +|--------|--------------|------|------| +| Pre-migration (operator) | Operator increases RF / fixes placement before switching to diskless | Simplifies migration implementation | Often **expensive and slow** (classic replication copies data we are about to move to object storage anyway); higher ops burden | +| Pre-migration (automatic) | Migration pipeline expands RF while still classic | Fully automated | Same cost as operator path; adds risk/complexity during classic operation | +| Post-switch (recommended) | Switch partition to diskless storage first, then expand RF to `rack_count` as **metadata-only replicas** (eventually) | Avoids wasting time/resources replicating classic data; keeps a single migration pipeline | Rack-awareness/job distribution may be suboptimal until normalization completes | + +**Recommendation:** Prefer **post-switch normalization**, but treat it as **eventual / best-effort**, not a forced prerequisite. + +After the switch to `DISKLESS_ONLY`, correctness is not dependent on having `RF=rack_count` immediately: +- Metadata (and thus preferred routing) may temporarily return **fewer replicas**, and they may not be rack-aware. +- This can reduce locality and concentrate “leader/job ownership” on fewer brokers. +- However, diskless serving remains correct (data is in object storage), and availability can still be preserved via the + transformer’s diskless-only fallback behavior. + +Normalization is still **worth doing** for consistency and long-term operability (leadership distribution, rack-awareness, +future RLM-aligned work), but it can happen asynchronously (controller background task or operator-triggered reassignment). + #### Approach A: Shrink to RF=1 During Migration ``` @@ -425,6 +467,16 @@ Therefore the transformer must be conservative **per partition state**: - Background task expands to RF=rack_count when convenient - Not urgent — availability is already handled by transformer +**But important for future tiering / migration work:** + +If we later want Tiered Storage reads (RLM) to work for these partitions (for example during tiered→diskless migration, or +any design that reads via `UnifiedLog`/RLM), then RF=1 “orphan/synthetic placement” is **not sufficient**. We must first +modernize placement so that at least one **real replica broker** has the `UnifiedLog` state required by RLM. + +In practice this means the “eventual modernization” step becomes a **prerequisite** for tiering-related work: +- migrate/replace an orphan single replica (broker id not present) to an alive broker +- and typically expand to RF=rack_count to align with Approach C and avoid re-introducing one-off tiered-read fixes + **Manual modernization (alternative):** - Operator uses `kafka-reassign-partitions.sh` to expand RF - Same tooling as classic topics @@ -704,6 +756,55 @@ See [Ghost Replicas Problem](#ghost-replicas-problem) for details. --- +### Decision Option (keep alive): Allow Tiered Storage (RLM) reads from any broker + +**Concept:** Make Tiered Storage fetching work even when the broker serving the request does *not* host the partition’s +`UnifiedLog` (i.e., TS reads become “stateless” or “proxy-able”). This would preserve “any broker can serve” semantics even +for hybrid/tiered partitions. + +**Why it matters to this design:** It directly affects whether metadata transformation must be migration-state-aware +(`HYBRID`/`TIERED_ONLY` → replicas-only) or can always return “best available” brokers without risking tiered-read failures. + +#### Integration path if we choose this option (high level) + +One plausible incremental path is **proxy-based TS reads** (least invasive to Kafka internals): + +1. **Detect TS-needed reads at request time** (offset-aware): + - In the fetch handling path, for partitions where `offset < disklessStartOffset`, mark them as “tiered/local path”. +2. **Route/proxy TS-needed reads to a replica broker**: + - If the receiving broker is not in the replica set (no `UnifiedLog`), forward that partition’s fetch request to a chosen + replica broker (prefer same AZ, fall back cross-AZ). +3. **Replica broker executes standard `UnifiedLog.read()`**: + - This naturally covers local→tiered fallback via `RemoteLogManager` and returns data. +4. **Aggregate and respond**: + - The front broker merges proxied results with diskless-range results and returns a single Fetch response. +5. **Hardening**: + - Add timeouts/backpressure, metrics, and failure semantics (e.g., if no replica broker reachable, behave like Kafka: fail + the tiered portion). + +An alternative path is **making RLM independent of local `UnifiedLog`** (larger change / closer to KIP scope): move leader +epoch lineage + segment mapping into an external store so any broker can call RLM directly. This is likely larger than the +proxy approach. + +#### How this would change A/B/C (weighting) + +| Impact area | A/B (faking) | C (proposed) | +|------------|--------------|--------------| +| Remove “custom TS fetch path” work | **Yes** (saves ~2-3 weeks) | Not needed either way | +| Fix ghost replicas / lack of `Partition` objects | **No** | Already addressed by C | +| Enable standard RLM integration for tiering pipeline (`onLeadershipChange`) | **No** | **Yes** | +| Reduce need to modernize RF=1/orphans before tiered reads | **Partially** | Still recommended for ops + leadership | +| New complexity introduced | **High** (proxying, cross-AZ, quotas) | Medium (optional future) | + +**Bottom line:** This option can reduce *one* major A/B gap (tiered reads during migration) and could strengthen the +“availability routing” story, but it does **not** remove the main reasons we prefer C (real replicas, leadership, avoiding +ghost replicas, standard RLM integration). + +**Status:** Keep as an explicit team decision. Not required for Approach C, but compatible as a future enhancement if “any +broker can serve even hybrid/tiered” becomes a hard requirement. + +--- + ## Appendix ### Ghost Replicas Problem From 9c35394d3e35fc552b4cc8c3579dc8c6016ac5d4 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Wed, 14 Jan 2026 13:30:45 +0200 Subject: [PATCH 11/21] fixup! add diskless managed rf feature design --- .../ts-unification/DISKLESS_MANAGED_RF.md | 87 +++++++++++++++++-- 1 file changed, 79 insertions(+), 8 deletions(-) diff --git a/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md b/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md index 960cc1ebaf..4439ca4e80 100644 --- a/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md +++ b/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md @@ -13,10 +13,12 @@ 9. [Observability](#observability) 10. [Implementation Path](#implementation-path) *(6-8 weeks / 1 eng)* 11. [Rejected Alternatives](#rejected-alternatives) + - [Decision Option: Tiered reads from any broker](#decision-option-keep-alive-allow-tiered-storage-rlm-reads-from-any-broker) 12. [Appendix](#appendix) - [Ghost Replicas Problem](#ghost-replicas-problem) - - [Operational Benefits Summary](#operational-benefits-summary) - - [Leadership Benefits for Operations](#leadership-benefits-for-operations) + - [Operational Benefits (Day-2)](#operational-benefits-day-2) + - [Operational Benefits Summary](#operational-benefits-summary) + - [Leadership Benefits for Operations](#leadership-benefits-for-operations) --- @@ -69,7 +71,13 @@ This is a deliberate trade-off: it keeps the “always available” property for correctness for tiered/hybrid reads. If we later decide to make **tiered reads serveable from any broker**, then we can remove this constraint and strengthen the -decoupling (see “Decision Option: Tiered reads from any broker” below). +decoupling (see [Decision Option: Tiered reads from any broker](#decision-option-keep-alive-allow-tiered-storage-rlm-reads-from-any-broker)). + +**Scope/impact clarification:** +- **Approach C does not require this option** to reach a consistent architecture. Without it, we still get correctness by + keeping hybrid/tiered reads on replica brokers and “always available” behavior for diskless-only partitions. +- **Approaches A/B benefit partially** if this option is implemented (it can remove the “custom TS fetch path” gap), but it + does **not** fix the core A/B problems (ghost replicas, lack of real leadership/job ownership, RLM/tiering pipeline hooks). The following terminology helps distinguish the key differences: @@ -107,6 +115,9 @@ We have three options: | Pre-migration (automatic) | Migration pipeline expands RF while still classic | Fully automated | Same cost as operator path; adds risk/complexity during classic operation | | Post-switch (recommended) | Switch partition to diskless storage first, then expand RF to `rack_count` as **metadata-only replicas** (eventually) | Avoids wasting time/resources replicating classic data; keeps a single migration pipeline | Rack-awareness/job distribution may be suboptimal until normalization completes | +**Encoding in the proposed algorithm:** After switching a partition to `DISKLESS_ONLY`, the controller may *eventually* +normalize RF/placement (background task) but does not block availability or correctness on completing that step. + **Recommendation:** Prefer **post-switch normalization**, but treat it as **eventual / best-effort**, not a forced prerequisite. After the switch to `DISKLESS_ONLY`, correctness is not dependent on having `RF=rack_count` immediately: @@ -423,6 +434,15 @@ The same fallback logic applies to any RF < rack_count (including RF=1). This is | RF < racks after ops | **Allowed** | Operator can reassign; transformer handles availability | | Future configurability | Possible | Could allow `min.insync.replicas` style config later | +**Decision summary (routing vs tiered correctness):** + +- **If a partition may require Tiered Storage reads** (`TIERED_ONLY` / `HYBRID`): metadata must return **replica brokers only** + (brokers with `UnifiedLog`/RLM state). Availability follows Kafka semantics. +- **If a partition is `DISKLESS_ONLY`**: metadata can prioritize KRaft placement but may fall back to **any alive broker** + (transformer-first availability). +- If we later choose [Tiered reads from any broker](#decision-option-keep-alive-allow-tiered-storage-rlm-reads-from-any-broker), + we can remove the “replicas-only for tiered/hybrid” constraint and strengthen decoupling. + **Tiered Storage (TS) reading — Important Consideration:** TS fetching via `RemoteLogManager.read()` requires local state that current diskless doesn't have: @@ -433,6 +453,9 @@ TS fetching via `RemoteLogManager.read()` requires local state that current disk | `LeaderEpochFileCache` | For segment lookup | **Missing** | Has it | | Segment metadata | To locate tiered data | **Missing** | Has it | +**Therefore:** for migrated topics, “any broker can serve” is only safe once the partition is `DISKLESS_ONLY`, unless we +implement the “tiered reads from any broker” decision option. + **Impact on topic migration:** During Classic → Diskless migration, the read path must handle tiered data: @@ -522,6 +545,18 @@ New partitions use same one-per-rack logic as creation. **No auto-reassignment on broker failure.** Transformer handles availability. +**Why we do not auto-reassign (even within the same rack):** +- **Diskless-only partitions:** availability does not require reassignment; the transformer can route to any alive broker when + all assigned replicas are offline. Auto-reassignment adds churn and new failure modes for marginal benefit. +- **Hybrid/tiered partitions:** tiered reads must be served by replica brokers (needs `UnifiedLog`/RLM state). If all replicas + are offline, we prefer **Kafka semantics** (unavailable until a replica returns) over silently routing to non-replica + brokers that cannot serve tiered reads. +- **Avoid thrash:** reassigning on every transient broker flap can cause repeated metadata churn and operational instability. + +**Trade-off:** we prefer *temporary cross-AZ serving* (diskless-only fallback) over controller-driven reassignment, because +it preserves the “always available” property with fewer moving parts. If an operator wants rack-locality restored, standard +reassignment tooling remains available. + **Leader Election:** - Standard Kafka leader election from ISR - For diskless topics, all replicas are always in ISR @@ -542,16 +577,23 @@ New partitions use same one-per-rack logic as creation. ## Broker Behavior -- No replica fetcher threads for diskless topics +- No replica fetcher threads for diskless topics (invariant: must not regress) - No local `Partition` objects (for now — may add for RLM coordination later) - Inkless handlers (`AppendHandler`, `FetchHandler`) serve requests +**Regression guardrail:** diskless topics must not start follower fetchers or attempt log replication. Add tests/metrics to +ensure diskless partitions never create fetcher state. + --- ## Metadata Transformation ### Filtering Logic +This is the concrete encoding of the “offset-unaware metadata” constraint described in +[Tiered Storage (TS) reading — Important Consideration](#tiered-storage-ts-reading--important-consideration) and the “key +decision” in [Approach Comparison](#approach-comparison). + ``` FOR each diskless partition: # Important: Metadata responses are offset-unaware, so routing must be @@ -724,6 +766,21 @@ See [Ghost Replicas Problem](#ghost-replicas-problem) for details. **Concept:** Controller proactively detects offline replicas and reassigns to online brokers immediately. +**Example:** + +``` +diskless topic foo-0 (RF=3, rack-aware) + Replicas: [101(rack=a), 202(rack=b), 303(rack=c)] + +Broker 101 goes offline. + +Auto-reassign approach: + Controller immediately replaces 101 with 111 (also rack=a): + Replicas become [111, 202, 303] + +If 101 flaps (comes back quickly), we may churn again unless we add damping/heuristics. +``` + **Why considered:** - KRaft metadata always accurate - Standard tooling shows correct state @@ -752,6 +809,8 @@ See [Ghost Replicas Problem](#ghost-replicas-problem) for details. 1. **Two leadership systems** — KRaft vs. custom coordinator 2. **RLM still blocked** — RLM uses KRaft leadership 3. **More complexity** — three systems (KRaft + transformer + coordinator) + - Hard to battle-proof: split-brain risk, conflicting ownership, retries/timeouts, and unclear source of truth during + partial outages 4. **Defeats purpose** — if adding coordinator, why not use KRaft's? --- @@ -851,9 +910,18 @@ Topic: diskless-foo Replicas: 1,2,3 ISR: 1,2,3 This creates technical debt and operational confusion that compounds over time — essentially throwing problems at operators. +It also increases the risk of **unknown bugs**: tests/alerts may report “healthy ISR” while the data-plane is effectively +running a different system, masking real issues. + --- -### Operational Benefits Summary +### Operational Benefits (Day-2) + +The next two sections are related but distinct: +- **Operational Benefits Summary**: impact on tooling/alerts and operator mental model. +- **Leadership Benefits for Operations**: impact on partition-scoped job ownership and debugging. + +#### Operational Benefits Summary The proposed design (Approach C) provides clear operational benefits compared to alternatives: @@ -866,9 +934,7 @@ The proposed design (Approach C) provides clear operational benefits compared to | Capacity planning | Unpredictable load | Proportional to leadership | | Standard Kafka ops | May not work as expected | Work normally | ---- - -### Leadership Benefits for Operations +#### Leadership Benefits for Operations Current diskless uses **racing/randomized** job assignment. Leadership provides: @@ -880,4 +946,9 @@ Current diskless uses **racing/randomized** job assignment. Leadership provides: | Incident response | Check all broker logs | Check leader broker | | Tooling | Custom | Standard Kafka tooling | +**Observing progress / correctness:** +- **Leadership distribution**: standard `LeaderCount` / leader imbalance metrics per broker. +- **Job ownership**: leader-scoped logs/metrics (e.g., “job started/completed for partition X on leader Y”). +- **Rebalancing effectiveness**: changes in leader distribution after preferred leader election / reassignment. + Leadership isn't just about RLM — it's about having a **deterministic owner** for partition-level operations. From cd2d01c47f524d900a89738beb562ce34073a560 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Wed, 14 Jan 2026 13:37:02 +0200 Subject: [PATCH 12/21] add cross reference --- docs/inkless/ts-unification/DESIGN.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docs/inkless/ts-unification/DESIGN.md b/docs/inkless/ts-unification/DESIGN.md index 7caf2a3f4c..855033e0e0 100644 --- a/docs/inkless/ts-unification/DESIGN.md +++ b/docs/inkless/ts-unification/DESIGN.md @@ -555,6 +555,9 @@ public class DisklessMigrationHandler { ### 4.4 Stream 4: Multi-Replica Model for Diskless Topics +> **📄 Detailed Design:** See [DISKLESS_MANAGED_RF.md](DISKLESS_MANAGED_RF.md) for the comprehensive design of rack-aware, +> KRaft-managed replicas for diskless topics, including approach comparison, cost analysis, and implementation path. + **Objective:** Enable diskless topics to use 3 actual Kafka replicas while preserving write-to-any semantics. #### 4.4.1 Current vs. Proposed Model @@ -1972,6 +1975,7 @@ Once the tiering pipeline is complete, enabling reverse migration becomes straig - [KIP-405: Tiered Storage](https://cwiki.apache.org/confluence/display/KAFKA/KIP-405%3A+Kafka+Tiered+Storage) - [Inkless Architecture Documentation](./architecture.md) +- [Diskless-Managed Replication Factor](DISKLESS_MANAGED_RF.md) — Detailed design for rack-aware, KRaft-managed replicas - [RemoteLogManager Implementation](../../core/src/main/java/kafka/log/remote/RemoteLogManager.java) - [ControlPlane Interface](../../storage/inkless/src/main/java/io/aiven/inkless/control_plane/ControlPlane.java) From 1df130aa3bb3f2bacd9e4f80ad05ad372aabf235 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Wed, 14 Jan 2026 13:41:49 +0200 Subject: [PATCH 13/21] fixup! add diskless managed rf feature design --- .../ts-unification/DISKLESS_MANAGED_RF.md | 53 +++++++++++++++++++ 1 file changed, 53 insertions(+) diff --git a/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md b/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md index 4439ca4e80..60c6b99955 100644 --- a/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md +++ b/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md @@ -1,7 +1,60 @@ # Diskless-Managed Replication Factor +## Overview + +### Problem + +Current diskless topics use **RF=1 with faked metadata** — the transformer routes to any broker, ignoring KRaft placement. +This works for pure diskless, but **blocks bidirectional topic migration** (Classic ↔ Diskless) and **RLM integration** for +the tiering pipeline. + +### Key Factors Considered + +| Factor | Constraint/Consideration | +|--------|--------------------------| +| **Topic migration** | Must support Classic → Diskless (immediate) and Diskless → Classic (future) | +| **Tiered Storage reads** | RLM requires `UnifiedLog` state — only replica brokers can serve tiered reads | +| **Always-available semantics** | Diskless topics must remain available even when KRaft replicas are offline | +| **RLM/tiering pipeline** | Standard `onLeadershipChange()` integration requires real KRaft leadership | +| **Operational clarity** | Tooling (`describe topic`) should reflect reality; deterministic job ownership | +| **Implementation cost** | Minimize weeks-to-value while avoiding technical debt | + +### Proposed Approach: RF=rack_count with Transformer-First Availability + +**At topic creation:** +- Controller assigns **RF = rack_count** (one replica per rack/AZ) +- Real KRaft-managed replicas with rack-aware placement + +**At runtime (routing):** +- **Hybrid/tiered partitions**: metadata returns replica brokers only (tiered reads require `UnifiedLog`) +- **Diskless-only partitions**: metadata prefers replicas but falls back to any alive broker if all are offline + +**Key simplification:** No controller auto-reassignment. Availability is handled by the transformer at routing time, not by +moving replicas in KRaft. This keeps the design simple while preserving the "always available" property. + +### Trade-offs Accepted + +| Trade-off | Accepted | Rationale | +|-----------|----------|-----------| +| KRaft metadata may show offline brokers | Yes | Availability is not blocked; eventual consistency is acceptable | +| `describe topic` may be temporarily stale | Yes | Operator can see actual state via metrics; availability is real | +| Tiered reads require replica brokers | Yes | Necessary for RLM correctness; can revisit with "tiered reads from any broker" option | + +### Estimate + +**6-8 weeks** (1 engineer) — vs **18+ weeks** for alternatives that defer real replicas. + +### Key Decisions for Review + +1. **Tiered reads: replicas-only vs any broker** — Currently replicas-only; "any broker" kept as explicit decision option +2. **Post-switch RF normalization** — Eventual/best-effort, not forced prerequisite +3. **No controller auto-reassignment** — Transformer handles availability; simpler design + +--- + ## Table of Contents +0. [Overview](#overview) 1. [Motivation](#motivation) 2. [Objectives](#objectives) 3. [Design: Transformer-First Availability](#design-transformer-first-availability) From 295e5d4095896a92e6b2f69a395cb00e9982677a Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Wed, 14 Jan 2026 14:01:00 +0200 Subject: [PATCH 14/21] fixup! add diskless managed rf feature design --- .../ts-unification/DISKLESS_MANAGED_RF.md | 162 +++++++++++------- 1 file changed, 104 insertions(+), 58 deletions(-) diff --git a/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md b/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md index 60c6b99955..ff8e4c429a 100644 --- a/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md +++ b/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md @@ -13,7 +13,7 @@ the tiering pipeline. | Factor | Constraint/Consideration | |--------|--------------------------| | **Topic migration** | Must support Classic → Diskless (immediate) and Diskless → Classic (future) | -| **Tiered Storage reads** | RLM requires `UnifiedLog` state — only replica brokers can serve tiered reads | +| **Tiered Storage reads** | RLM requires `UnifiedLog` state — only replica brokers can serve tiered reads (or requires RLM extension; see [Decision Option](#decision-option-keep-alive-allow-tiered-storage-rlm-reads-from-any-broker)) | | **Always-available semantics** | Diskless topics must remain available even when KRaft replicas are offline | | **RLM/tiering pipeline** | Standard `onLeadershipChange()` integration requires real KRaft leadership | | **Operational clarity** | Tooling (`describe topic`) should reflect reality; deterministic job ownership | @@ -115,14 +115,17 @@ We evaluate three approaches: Kafka **Metadata** responses are offset-unaware, but Tiered Storage reads via `RemoteLogManager` require local `UnifiedLog` state on the serving broker. This creates a hard constraint: -- When a partition can serve **tiered/local reads** (migration state `TIERED_ONLY` / `HYBRID`), metadata transformation must - return only brokers in the **KRaft replica set** (Kafka semantics). +- When a partition can serve **tiered/local reads** (`DISKLESS_TIERED`), metadata transformation must return only brokers in + the **KRaft replica set** (Kafka semantics). - Only when a partition is **`DISKLESS_ONLY`** can we safely decouple “availability routing” from KRaft placement and route to any alive broker (transformer-first availability). This is a deliberate trade-off: it keeps the “always available” property for diskless-only partitions, while preserving correctness for tiered/hybrid reads. +> **Terminology**: `DISKLESS_TIERED` encompasses migration states where tiered data exists (`TIERED_ONLY`, `HYBRID` per +> `DESIGN.md`). Routing logic treats these identically: replicas-only. + If we later decide to make **tiered reads serveable from any broker**, then we can remove this constraint and strengthen the decoupling (see [Decision Option: Tiered reads from any broker](#decision-option-keep-alive-allow-tiered-storage-rlm-reads-from-any-broker)). @@ -141,17 +144,19 @@ The following terminology helps distinguish the key differences: #### Summary Table -| Aspect | A: Shrink to RF=1 | B: Keep RF=3, fake | C: RF=rack_count (proposed) | -|-------------------------|--------------------------|--------------------------|------------------------------| -| KRaft RF after migrate | 1 | 3 (unchanged) | 3 (managed, rack-aware) | -| Transformer behavior | Hash to any broker | Hash to any broker | Filter by AZ (KRaft-sourced) | -| Hybrid/tiered reads | **Custom / risky** | **Custom / risky** | **Replicas only (safe)** | -| Replica objects | None (offline in KRaft) | Ghost (no `Partition`) | Real (metadata-only) | -| Local state after migr. | **Must delete** (risk) | **Must delete** (risk) | Standard handoff | -| Migration coordinator | Custom (who?) | Custom (who?) | **Leader** | -| RLM integration | Custom hooks (~11 wks) | Custom hooks (~11 wks) | **Standard (~0 wks)** | -| Job assignment | Racing | Racing | **Leader-based** | -| Reverse migration ready | No (must expand RF) | No (must fix ghosts) | **Yes** | +``` +Aspect | A: Shrink RF=1 | B: Keep RF=3 fake | C: RF=rack_count ✓ +-----------------------|---------------------|---------------------|--------------------- +KRaft RF after migrate | 1 | 3 (unchanged) | 3 (rack-aware) +Transformer behavior | Hash to any broker | Hash to any broker | Filter by AZ +Hybrid/tiered reads | Custom / risky | Custom / risky | Replicas only (safe) +Replica objects | None (offline) | Ghost (no Partition)| Real (metadata-only) +Local state after migr | Must delete (risk) | Must delete (risk) | Standard handoff +Migration coordinator | Custom (who?) | Custom (who?) | Leader +RLM integration | Custom (~11 wks) | Custom (~11 wks) | Standard (~0 wks) +Job assignment | Racing | Racing | Leader-based +Reverse migration | No (must expand RF) | No (must fix ghosts)| Yes +``` #### Migration RF Normalization (Classic → Diskless) @@ -162,11 +167,19 @@ This matters because diskless-managed RF assumes replicas are meaningful for **l We have three options: -| Option | What happens | Pros | Cons | -|--------|--------------|------|------| -| Pre-migration (operator) | Operator increases RF / fixes placement before switching to diskless | Simplifies migration implementation | Often **expensive and slow** (classic replication copies data we are about to move to object storage anyway); higher ops burden | -| Pre-migration (automatic) | Migration pipeline expands RF while still classic | Fully automated | Same cost as operator path; adds risk/complexity during classic operation | -| Post-switch (recommended) | Switch partition to diskless storage first, then expand RF to `rack_count` as **metadata-only replicas** (eventually) | Avoids wasting time/resources replicating classic data; keeps a single migration pipeline | Rack-awareness/job distribution may be suboptimal until normalization completes | +``` +Option | What happens +---------------------------|---------------------------------------------------------- +Pre-migration (operator) | Operator increases RF before switching to diskless +Pre-migration (automatic) | Migration pipeline expands RF while still classic +Post-switch (recommended) | Switch to diskless first, expand RF eventually +``` + +| Option | Pros | Cons | +|--------|------|------| +| Pre-migration (operator) | Simplifies migration impl | Expensive/slow (replicates data we'll move anyway); ops burden | +| Pre-migration (automatic) | Fully automated | Same cost; adds risk during classic operation | +| Post-switch ✓ | Avoids wasting resources; single pipeline | Suboptimal placement until normalization | **Encoding in the proposed algorithm:** After switching a partition to `DISKLESS_ONLY`, the controller may *eventually* normalize RF/placement (background task) but does not block availability or correctness on completing that step. @@ -247,12 +260,14 @@ What "real (metadata-only) replica" means: ### Diskless → Classic Migration Readiness -| Aspect | A: RF=1 | B: RF=3 faked | C: RF=rack_count | -|-------------------|-------------------------|---------------------------|---------------------------| -| Starting RF | 1 | 3 (ghost replicas) | 3 (real replicas) | -| Before migration | Must expand RF=1 → RF=3 | Must "un-ghost" replicas | **Ready as-is** | -| Replica state | Create from scratch | Create `Partition` objects| Already correct | -| Complexity | High (2 operations) | Medium (fix ghosts) | **Low (just migrate)** | +``` +Aspect | A: RF=1 | B: RF=3 faked | C: RF=rack_count ✓ +-----------------|---------------------|------------------------|-------------------- +Starting RF | 1 | 3 (ghost replicas) | 3 (real replicas) +Before migration | Must expand RF=1→3 | Must "un-ghost" | Ready as-is +Replica state | Create from scratch | Create Partition objs | Already correct +Complexity | High (2 operations) | Medium (fix ghosts) | Low (just migrate) +``` **Key insight:** Only Approach C is ready for reverse migration without additional work. @@ -336,15 +351,17 @@ The same problems exist, but Approach C solves them via **standard Kafka pattern #### Decision Framework -| Factor | A: Shrink to RF=1 | B: Keep RF=3, fake | C: RF=rack_count (proposed) | -|--------------------------|--------------------------|--------------------------|------------------------------| -| Short-term cost | ~7 weeks (local + TS) | ~7 weeks (local + TS) | ~6-8 weeks (incl. buffer) | -| Local state risk | **High** (must cleanup) | **High** (must cleanup) | Low (standard handoff) | -| Classic → Diskless | Custom coordination | Custom coordination | Standard replica handoff | -| RLM integration | Custom hooks (~11 wks) | Custom hooks (~11 wks) | Standard (~0 wks) | -| Diskless → Classic | RF expansion + migration | Un-ghost + migration | Migration only | -| Long-term maintenance | Two code paths | Two code paths | One code path | -| Kafka/KIP alignment | Divergent | Divergent | Aligned | +``` +Factor | A: Shrink RF=1 | B: Keep RF=3 fake | C: RF=rack_count ✓ +----------------------|----------------------|----------------------|--------------------- +Short-term cost | ~7 wks (local + TS) | ~7 wks (local + TS) | ~6-8 wks (w/ buffer) +Local state risk | High (must cleanup) | High (must cleanup) | Low (std handoff) +Classic → Diskless | Custom coordination | Custom coordination | Std replica handoff +RLM integration | Custom (~11 wks) | Custom (~11 wks) | Standard (~0 wks) +Diskless → Classic | RF expand + migrate | Un-ghost + migrate | Migration only +Long-term maintenance | Two code paths | Two code paths | One code path +Kafka/KIP alignment | Divergent | Divergent | Aligned +``` #### Summary @@ -378,6 +395,26 @@ Plus, deferring creates technical debt: 5. Avoids accumulating technical debt 6. Aligns diskless with Kafka's replica model +### Open Questions for Review + +1. **Tiered reads: replicas-only vs any broker** + - Current decision: replicas-only (RLM requires `UnifiedLog`) + - Alternative kept alive as [Decision Option](#decision-option-keep-alive-allow-tiered-storage-rlm-reads-from-any-broker) + - **Question:** Is this the right default? Should we invest in "tiered reads from any broker" sooner? + +2. **Post-switch RF normalization: automatic vs operator-triggered** + - Current decision: eventual/best-effort, controller may normalize in background + - **Question:** Should normalization be fully automatic, or should operators trigger it explicitly? + +3. **`Partition` objects for diskless** + - Current: skipped (`ReplicaManager.isDisklessTopic()` check) + - Future: may add for RLM coordination + - **Question:** Should we add minimal `Partition` objects now (for RLM hooks), or defer? + +4. **Existing RF=1 diskless topics modernization** + - Current: eventual background task or manual `kafka-reassign-partitions.sh` + - **Question:** Should we proactively modernize existing RF=1 topics, or leave as opt-in? + ### Related Documents - [DESIGN.md](DESIGN.md) — Overall tiered storage unification design @@ -489,22 +526,24 @@ The same fallback logic applies to any RF < rack_count (including RF=1). This is **Decision summary (routing vs tiered correctness):** -- **If a partition may require Tiered Storage reads** (`TIERED_ONLY` / `HYBRID`): metadata must return **replica brokers only** +- **If a partition may require Tiered Storage reads** (`DISKLESS_TIERED`): metadata must return **replica brokers only** (brokers with `UnifiedLog`/RLM state). Availability follows Kafka semantics. - **If a partition is `DISKLESS_ONLY`**: metadata can prioritize KRaft placement but may fall back to **any alive broker** (transformer-first availability). - If we later choose [Tiered reads from any broker](#decision-option-keep-alive-allow-tiered-storage-rlm-reads-from-any-broker), we can remove the “replicas-only for tiered/hybrid” constraint and strengthen decoupling. -**Tiered Storage (TS) reading — Important Consideration:** +#### Tiered Storage (TS) reading — Important Consideration TS fetching via `RemoteLogManager.read()` requires local state that current diskless doesn't have: -| Requirement | What RLM needs | Current diskless | Approach C | -|-------------|----------------|------------------|------------| -| `UnifiedLog` | `fetchLog.apply(tp)` must return log | **Missing** (no `Partition`) | Has it (real replicas) | -| `LeaderEpochFileCache` | For segment lookup | **Missing** | Has it | -| Segment metadata | To locate tiered data | **Missing** | Has it | +``` +Requirement | What RLM needs | Current diskless | Approach C +---------------------|--------------------------------|---------------------|------------------- +UnifiedLog | fetchLog.apply(tp) returns log | Missing | Has it (real) +LeaderEpochFileCache | For segment lookup | Missing | Has it +Segment metadata | To locate tiered data | Missing | Has it +``` **Therefore:** for migrated topics, “any broker can serve” is only safe once the partition is `DISKLESS_ONLY`, unless we implement the “tiered reads from any broker” decision option. @@ -533,7 +572,7 @@ Kafka **Metadata** responses do not include fetch offsets, so the transformer ca - Diskless range (can be served by any broker). Therefore the transformer must be conservative **per partition state**: -- **If tiered reads are possible** (migration state is `TIERED_ONLY` / `HYBRID` per `DESIGN.md`), the transformer must return only brokers in the **KRaft replica set** (so TS reads work). +- **If tiered reads are possible** (`DISKLESS_TIERED`), the transformer must return only brokers in the **KRaft replica set** (so TS reads work). - **Only when the partition is `DISKLESS_ONLY`** (no tiered reads), the transformer may route to any alive broker for availability. **Note:** Pure diskless topics (never migrated) use Inkless `FetchHandler` which reads from object storage directly — no RLM dependency. The TS limitation matters during/around migration when a partition still has tiered/local data. @@ -651,18 +690,22 @@ decision” in [Approach Comparison](#approach-comparison). FOR each diskless partition: # Important: Metadata responses are offset-unaware, so routing must be # conservative when tiered reads are possible. - mode = migrationState(tp) # DISKLESS_ONLY vs HYBRID/TIERED_ONLY + mode = migrationState(tp) # DISKLESS_ONLY or DISKLESS_TIERED (has tiered data) assigned_replicas = KRaft replica set alive_replicas = assigned_replicas ∩ alive_brokers - IF mode != DISKLESS_ONLY: + IF mode == DISKLESS_TIERED: # Tiered reads may be required -> must stay on replicas that have UnifiedLog/RLM state IF alive_replicas is not empty: RETURN alive_replicas (prefer clientAZ, else cross-AZ) ELSE: - RETURN assigned_replicas (Kafka semantics: partition unavailable until replica returns) - ELSE: + # Yes, this loses availability — but tiered reads REQUIRE replica brokers. + # This is standard Kafka semantics: partition unavailable until replica returns. + # Note: This scenario is rare (all replicas down) and temporary. + # Rolling restarts or upgrades replacing nodes won't hit this path — at least one replica is alive. + RETURN assigned_replicas (partition unavailable) + ELSE: # DISKLESS_ONLY # Diskless-only: any broker can serve, so we can preserve "always available" IF alive_replicas is not empty: # Normal case: use KRaft placement @@ -677,11 +720,12 @@ FOR each diskless partition: ### Key Properties -1. **Instant availability (diskless-only)**: No waiting for controller when `DISKLESS_ONLY` +1. **Instant availability (`DISKLESS_ONLY`)**: No waiting for controller; falls back to hash if all replicas offline 2. **AZ-aware when possible**: Uses KRaft placement if alive -3. **Graceful degradation (diskless-only)**: Falls back to hash if needed -4. **Tiered-safe routing (hybrid/tiered)**: When tiered reads are possible, only replica brokers are returned -5. **Availability semantics are state-dependent**: Hybrid/tiered behaves like Kafka; diskless-only is always available +3. **Graceful degradation (`DISKLESS_ONLY`)**: Falls back to hash-based selection if needed +4. **Tiered-safe routing (`DISKLESS_TIERED`)**: Only replica brokers are returned (RLM requires `UnifiedLog`) +5. **Availability trade-off (`DISKLESS_TIERED`)**: If all replicas offline, partition is unavailable (standard Kafka semantics) +6. **State-dependent semantics**: Hybrid/tiered behaves like Kafka; diskless-only is always available --- @@ -875,7 +919,7 @@ If 101 flaps (comes back quickly), we may churn again unless we add damping/heur for hybrid/tiered partitions. **Why it matters to this design:** It directly affects whether metadata transformation must be migration-state-aware -(`HYBRID`/`TIERED_ONLY` → replicas-only) or can always return “best available” brokers without risking tiered-read failures. +(`DISKLESS_TIERED` → replicas-only) or can always return “best available” brokers without risking tiered-read failures. #### Integration path if we choose this option (high level) @@ -900,13 +944,15 @@ proxy approach. #### How this would change A/B/C (weighting) -| Impact area | A/B (faking) | C (proposed) | -|------------|--------------|--------------| -| Remove “custom TS fetch path” work | **Yes** (saves ~2-3 weeks) | Not needed either way | -| Fix ghost replicas / lack of `Partition` objects | **No** | Already addressed by C | -| Enable standard RLM integration for tiering pipeline (`onLeadershipChange`) | **No** | **Yes** | -| Reduce need to modernize RF=1/orphans before tiered reads | **Partially** | Still recommended for ops + leadership | -| New complexity introduced | **High** (proxying, cross-AZ, quotas) | Medium (optional future) | +``` +Impact area | A/B (faking) | C (proposed) +---------------------------------------|------------------------|------------------------- +Remove "custom TS fetch path" work | Yes (saves ~2-3 wks) | Not needed either way +Fix ghost replicas / no Partition | No | Already addressed +Enable std RLM integration | No | Yes +Reduce need to modernize RF=1/orphans | Partially | Still recommended +New complexity introduced | High (proxy, cross-AZ) | Medium (optional future) +``` **Bottom line:** This option can reduce *one* major A/B gap (tiered reads during migration) and could strengthen the “availability routing” story, but it does **not** remove the main reasons we prefer C (real replicas, leadership, avoiding From 1a6a5855e4aaff3f0c233982bd211c63e9e96979 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Wed, 14 Jan 2026 15:57:04 +0200 Subject: [PATCH 15/21] fixup! add diskless managed rf feature design --- docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md b/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md index ff8e4c429a..b39961e7fa 100644 --- a/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md +++ b/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md @@ -390,7 +390,7 @@ Plus, deferring creates technical debt: **Implement the proposed design (Approach C: RF=rack_count)** because: 1. Topic migration (both directions) benefits from real replicas 2. RLM integration becomes standard rather than custom -3. One-time 6-8 week investment vs. 15+ weeks with A/B +3. One-time 6-8 week investment vs. 18+ weeks with A/B 4. Avoids local state cleanup risks (data corruption, offset drift) 5. Avoids accumulating technical debt 6. Aligns diskless with Kafka's replica model From a609b8d74959f3a0073deef6ef2bd71dbbaac27e Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Thu, 15 Jan 2026 15:01:32 +0200 Subject: [PATCH 16/21] updates from phase 0 --- .../ts-unification/DISKLESS_MANAGED_RF.md | 32 +++++++++++++++++-- 1 file changed, 29 insertions(+), 3 deletions(-) diff --git a/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md b/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md index b39961e7fa..8680e15ee9 100644 --- a/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md +++ b/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md @@ -438,6 +438,23 @@ Enable **rack-aware, stable KRaft-managed replicas** for diskless topics: ## Design: Transformer-First Availability +### Diskless ISR Semantics + +Diskless replicas never diverge because user data lives in remote object storage rather than broker-local logs. As a result, +ISR membership for `DISKLESS_ONLY` partitions is liveness-gated, not lag-gated: + +- The controller drops a replica from ISR when it is fenced, unregistered, or explicitly removed (rolling restart or broker replacement). +- When the replica re-registers (rolling restart/upgrade completes), the controller can add it back immediately—no catch-up fetch or +`Partition` object is required, because the replica is “in sync” by construction. +- `min.insync.replicas` semantics remain intact: as long as the required number of replicas are alive/unfenced, they are considered in-sync. + There is no risk of serving stale data since all reads/writes ultimately hit remote storage. +- Rolling maintenance therefore behaves predictably: ISR temporarily shrinks while a broker is fenced and expands as soon as it comes back, + while availability is preserved by the transformer’s routing logic. + +This weaker ISR definition is intentional for `DISKLESS_ONLY`. Hybrid/tiered states still rely on `UnifiedLog` state and classic +lag-based ISR semantics. If a future KIP introduces local logs as caches, that iteration can extend/restore traditional ISR behavior for +diskless replicas. + ### The Insight For diskless topics, **availability** and **metadata accuracy** can be decoupled. @@ -771,6 +788,15 @@ This is acceptable because: 2. Operator can see actual state via broker metrics 3. Eventually consistent (when broker returns or is reassigned) +**`DescribeTopicPartitions` path today:** brokers handle this RPC (controllers do not). `KafkaApis` responds from the local `MetadataCache`, +then applies the same diskless transformation as `MetadataResponse` (hashing/faking placement). Once KRaft metadata maintains diskless RF>1 +state, this post-processing must be updated (or removed) to simply reflect controller placement. + +**`ListOffsets` path today:** brokers call `ReplicaManager.fetchOffset`. For diskless topics this invokes +`ReplicaManager.disklessFetchOffset`, which fans out to the Inkless control plane to look up offsets in remote storage. This already bypasses +local logs, but once RF>1 diskless placement is real, the response still needs to prefer controller-provided replica IDs and no longer rely +on synthetic hashing. + --- ## Implementation Path @@ -779,9 +805,9 @@ This is acceptable because: ### Phase 0: Research and Validation (1 week) -1. Leader election works without broker `Partition` objects -2. ISR updates don't require broker-side `Partition` state -3. `DescribeTopics` / `ListOffsets` work with RF > 1 +1. Leader election works without broker `Partition` objects *(validated; controller depends only on KRaft `PartitionInfo` and broker liveness)* +2. ISR updates don't require broker-side `Partition` state *(validated; controller drops/adds replicas purely via metadata, see [Diskless ISR Semantics](#diskless-isr-semantics))* +3. `DescribeTopics` / `ListOffsets` work with RF > 1 *(sanity check once RF>1 diskless is enabled: broker responses must reflect KRaft placement for metadata and use control-plane offsets without synthetic hashing)* ### Phase 1: Topic Creation with Rack-Aware Placement (2 weeks) From 80ec6adb5980c99475c0ef3d93dad6d00c967811 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Fri, 16 Jan 2026 14:34:16 +0200 Subject: [PATCH 17/21] results from phase 0 --- .../ts-unification/DISKLESS_MANAGED_RF.md | 45 +- .../ReplicationControlManagerTest.java | 404 ++++++++++++++++++ 2 files changed, 446 insertions(+), 3 deletions(-) diff --git a/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md b/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md index 8680e15ee9..0cc1fde583 100644 --- a/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md +++ b/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md @@ -805,9 +805,48 @@ on synthetic hashing. ### Phase 0: Research and Validation (1 week) -1. Leader election works without broker `Partition` objects *(validated; controller depends only on KRaft `PartitionInfo` and broker liveness)* -2. ISR updates don't require broker-side `Partition` state *(validated; controller drops/adds replicas purely via metadata, see [Diskless ISR Semantics](#diskless-isr-semantics))* -3. `DescribeTopics` / `ListOffsets` work with RF > 1 *(sanity check once RF>1 diskless is enabled: broker responses must reflect KRaft placement for metadata and use control-plane offsets without synthetic hashing)* +**Objective:** Validate that controller operations are metadata-driven and do not require broker-side `Partition` objects. + +**Findings:** + +1. **Leader election** *(validated)* + - Controller (`ReplicationControlManager`) elects leaders purely from KRaft `PartitionInfo` + broker liveness/fencing + - No dependency on broker-side `Partition` objects + - `handleBrokerFenced()` removes broker from ISR and triggers re-election via `generateLeaderAndIsrUpdates()` + +2. **ISR maintenance** *(validated)* + - ISR updates are metadata-driven via `handleBrokerFenced()`, `handleBrokerUnregistered()`, `handleBrokerInControlledShutdown()` + - Controller shrinks/expands ISR without broker `Partition`/fetcher state + - For `DISKLESS_ONLY`, ISR is intentionally liveness-gated (no divergence risk; data is in remote storage) + - `min.insync.replicas` remains satisfied as long as enough replicas are alive + +3. **Broker heartbeats** *(validated)* + - `BrokerHeartbeatManager` is broker-scoped (epochs, metadata offsets, fencing) + - No per-partition payload, no dependency on `Partition` objects + +4. **DescribeTopicPartitions path** *(documented)* + - Served by brokers (controllers don't handle it) + - Brokers answer from `MetadataCache` and currently apply diskless "fake placement" transformation + - Once diskless RF>1 uses real KRaft placement, post-processing must be updated to reflect controller placement + +5. **ListOffsets path** *(documented)* + - Served by brokers via `ReplicaManager.fetchOffset` + - Diskless topics use `disklessFetchOffset` to query Inkless control plane/remote storage (no local logs) + - When RF>1 diskless lands, responses should rely on controller placement without synthetic hashing + +6. **Staleness/consistency** *(documented)* + - All broker-served metadata is as current as broker's `MetadataCache` + - Clients see eventually consistent view until brokers apply latest KRaft deltas (matches classic Kafka) + +**Remaining item:** +- Item 3 (`DescribeTopics`/`ListOffsets` with RF>1) cannot be exercised until diskless RF>1 is implemented +- It remains a future sanity check to ensure broker replies reflect KRaft placement plus control-plane offsets + +**Test coverage:** +- See `ReplicationControlManagerTest.DisklessRfPhase0Tests` in `metadata/src/test/java/` +- Classic topic tests validate controller behavior (pass today) +- Diskless RF>1 tests are `@DisabledIf` until Phase 1 is implemented +- Run disabled tests: `DISKLESS_RF_FUTURE_TESTS_ENABLED=true ./gradlew :metadata:test --tests "...DisklessRfPhase0Tests.*" --rerun` ### Phase 1: Topic Creation with Rack-Aware Placement (2 weeks) diff --git a/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java index a114bccffc..7ab24246f8 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java @@ -102,7 +102,9 @@ import org.apache.kafka.server.util.MockRandom; import org.apache.kafka.timeline.SnapshotRegistry; +import org.junit.jupiter.api.Nested; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.condition.DisabledIf; import org.junit.jupiter.api.Timeout; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.CsvSource; @@ -166,6 +168,408 @@ @Timeout(40) public class ReplicationControlManagerTest { + + /* + * Phase 0 tests for Diskless RF>1 + * See: docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md + * + * These tests validate that controller's leader election, ISR management, and heartbeat + * handling are metadata-driven and do not require broker-side Partition objects. + * + * =================================================================================== + * REPLICA TERMINOLOGY (from design doc) + * =================================================================================== + * + * OFFLINE REPLICA: + * - KRaft shows replica on broker X, but broker X is down or doesn't exist + * - Current diskless: RF=1 replica may be offline; transformer ignores and hashes to any alive broker + * - Proposed RF>1: Controller tracks in KRaft; transformer routes around to alive replicas + * + * GHOST REPLICA: + * - KRaft shows replica on broker X, broker X is alive, but no Partition object exists + * - Current diskless: isDisklessTopic() skips getOrCreatePartition() - replicas are "ghosts" + * - Problem: `describe topic` shows healthy replicas that aren't real + * - Proposed RF>1: Still no Partition objects, but replicas meaningful for routing/leadership + * + * REAL (METADATA-ONLY) REPLICA: + * - Broker recognizes it's a replica, but no local log (data in object storage) + * - Current diskless: N/A (RF=1, faked metadata) + * - Proposed RF>1: Target state for diskless replicas + * + * =================================================================================== + * CURRENT DISKLESS BEHAVIOR (RF=1, faked metadata) + * =================================================================================== + * + * In ReplicaManager.scala, diskless topics skip Partition object creation: + * + * if (!_inklessMetadataView.isDisklessTopic(tp.topic())) + * getOrCreatePartition(tp, delta, info.topicId) // SKIPPED for diskless! + * + * InklessTopicMetadataTransformer "fakes" the metadata response: + * - Ignores KRaft replica assignments entirely + * - Hashes topic+partition to select any alive broker as "leader" + * - Sets replicas/ISR to single-element list with the hashed broker + * - Sets offlineReplicas to empty (no "real" replicas to be offline) + * + * =================================================================================== + * PROPOSED BEHAVIOR (RF=rack_count, real metadata) + * =================================================================================== + * + * - Controller assigns real replicas (one per rack) in KRaft metadata + * - Brokers still skip Partition objects (no local logs) + * - Transformer filters KRaft placement by client AZ + * - Falls back to any alive broker if all replicas offline (DISKLESS_ONLY) + * - ISR is liveness-gated (replicas never diverge - data in object storage) + * - offlineReplicas reflects actual broker status from KRaft + * + * =================================================================================== + * PHASE 0 FINDINGS + * =================================================================================== + * + * - Leader election: Controller elects from KRaft + broker liveness; no Partition objects needed + * - ISR maintenance: Metadata-driven shrink/expand on fence/unfence + * - Broker heartbeats: Broker-scoped, no per-partition payload + * + * =================================================================================== + * TEST ORGANIZATION + * =================================================================================== + * + * - Classic topic tests: Verify controller behavior (pass today) + * - Diskless RF>1 tests: @DisabledIf until RF>1 topic creation implemented + * - Diskless ISR semantics: Liveness-gated expansion (new behavior, @DisabledIf) + * + * To force-run disabled tests: + * DISKLESS_RF_FUTURE_TESTS_ENABLED=true ./gradlew :metadata:test \ + * --tests "...DisklessRfPhase0Tests.*" --rerun + */ + @Nested + class DisklessRfPhase0Tests { + + // Returns true if future tests should be skipped (default behavior) + boolean disklessRfFutureTestsDisabled() { + return !"true".equals(System.getenv("DISKLESS_RF_FUTURE_TESTS_ENABLED")); + } + + // ================================================================================== + // CLASSIC TOPIC TESTS + // Verify controller behavior that applies to both classic and future diskless RF>1. + // These pass today and document the foundation. + // ================================================================================== + + // Leader election via controller metadata when leader is fenced. + // Finding: Controller elects from KRaft + broker liveness; no Partition objects needed. + @Test + void leaderElectionWhenLeaderFenced_classicTopic() { + ReplicationControlTestContext ctx = new ReplicationControlTestContext.Builder() + .setDisklessStorageSystemEnabled(true) + .build(); + ReplicationControlManager replication = ctx.replicationControl; + ctx.registerBrokers(1, 2, 3); + ctx.unfenceBrokers(1, 2, 3); + + CreatableTopicResult topic = ctx.createTestTopic( + "classic-rf3", new int[][] {new int[] {1, 2, 3}}); + + Uuid topicId = topic.topicId(); + List records = new ArrayList<>(); + replication.handleBrokerFenced(1, records); + ctx.replay(records); + + PartitionRegistration partition = replication.getPartition(topicId, 0); + assertNotNull(partition, "Partition should exist after leader fencing"); + assertArrayEquals(new int[] {2, 3}, partition.isr, + "ISR should exclude fenced leader"); + assertTrue(partition.leader == 2 || partition.leader == 3, + "Leader should be re-elected from remaining ISR members"); + } + + // Follower fencing shrinks ISR without affecting leader. + // Finding: ISR shrink is metadata-driven via handleBrokerFenced. + @Test + void isrShrinksWhenFollowerFenced_classicTopic() { + ReplicationControlTestContext ctx = new ReplicationControlTestContext.Builder() + .setDisklessStorageSystemEnabled(true) + .build(); + ReplicationControlManager replication = ctx.replicationControl; + ctx.registerBrokers(1, 2, 3); + ctx.unfenceBrokers(1, 2, 3); + + CreatableTopicResult topic = ctx.createTestTopic( + "classic-rf3", new int[][] {new int[] {1, 2, 3}}); + + Uuid topicId = topic.topicId(); + List records = new ArrayList<>(); + replication.handleBrokerFenced(2, records); + ctx.replay(records); + + PartitionRegistration partition = replication.getPartition(topicId, 0); + assertNotNull(partition); + assertArrayEquals(new int[] {1, 3}, partition.isr, + "ISR should exclude fenced follower"); + assertEquals(1, partition.leader, + "Leader should remain stable when follower is fenced"); + } + + // Controlled shutdown removes broker from ISR and re-elects leader if needed. + // Finding: handleBrokerInControlledShutdown is metadata-driven. + @Test + void controlledShutdownRemovesFromIsrAndReelectsLeader_classicTopic() { + ReplicationControlTestContext ctx = new ReplicationControlTestContext.Builder() + .setDisklessStorageSystemEnabled(true) + .build(); + ReplicationControlManager replication = ctx.replicationControl; + ctx.registerBrokers(1, 2, 3); + ctx.unfenceBrokers(1, 2, 3); + + CreatableTopicResult topic = ctx.createTestTopic( + "classic-rf3", new int[][] {new int[] {1, 2, 3}}); + + Uuid topicId = topic.topicId(); + List records = new ArrayList<>(); + // Simulate controlled shutdown of the leader + replication.handleBrokerInControlledShutdown(1, 101, records); + ctx.replay(records); + + PartitionRegistration partition = replication.getPartition(topicId, 0); + assertNotNull(partition); + assertArrayEquals(new int[] {2, 3}, partition.isr, + "ISR should exclude broker in controlled shutdown"); + assertTrue(partition.leader == 2 || partition.leader == 3, + "Leader should be re-elected from remaining ISR"); + } + + // ================================================================================== + // DISKLESS RF>1 TESTS + // Same controller behaviors but with diskless topics. + // @DisabledIf until diskless RF>1 topic creation is implemented (Phase 1). + // ================================================================================== + + // [FUTURE] Leader election for diskless topics. + // Blocked on: Diskless RF>1 topic creation. + @DisabledIf("disklessRfFutureTestsDisabled") + @Test + void leaderElectionWhenLeaderFenced_disklessTopic() { + ReplicationControlTestContext ctx = new ReplicationControlTestContext.Builder() + .setDefaultDisklessEnable(true) + .setDisklessStorageSystemEnabled(true) + .build(); + ReplicationControlManager replication = ctx.replicationControl; + ctx.registerBrokers(1, 2, 3); + ctx.unfenceBrokers(1, 2, 3); + + // TODO: Update expected error to NONE.code() when diskless RF>1 is implemented + CreatableTopicResult topic = ctx.createTestTopic( + "diskless-rf3", new int[][] {new int[] {1, 2, 3}}, + Map.of(DISKLESS_ENABLE_CONFIG, "true"), + NONE.code()); + + Uuid topicId = topic.topicId(); + List records = new ArrayList<>(); + replication.handleBrokerFenced(1, records); + ctx.replay(records); + + PartitionRegistration partition = replication.getPartition(topicId, 0); + assertNotNull(partition); + assertArrayEquals(new int[] {2, 3}, partition.isr); + assertTrue(partition.leader == 2 || partition.leader == 3, + "Leader should be re-elected from remaining ISR"); + } + + // [FUTURE] ISR shrink for diskless topics. + // Blocked on: Diskless RF>1 topic creation. + @DisabledIf("disklessRfFutureTestsDisabled") + @Test + void isrShrinksWhenFollowerFenced_disklessTopic() { + ReplicationControlTestContext ctx = new ReplicationControlTestContext.Builder() + .setDefaultDisklessEnable(true) + .setDisklessStorageSystemEnabled(true) + .build(); + ReplicationControlManager replication = ctx.replicationControl; + ctx.registerBrokers(1, 2, 3); + ctx.unfenceBrokers(1, 2, 3); + + CreatableTopicResult topic = ctx.createTestTopic( + "diskless-rf3", new int[][] {new int[] {1, 2, 3}}, + Map.of(DISKLESS_ENABLE_CONFIG, "true"), + NONE.code()); + + Uuid topicId = topic.topicId(); + List records = new ArrayList<>(); + replication.handleBrokerFenced(2, records); + ctx.replay(records); + + PartitionRegistration partition = replication.getPartition(topicId, 0); + assertNotNull(partition); + assertArrayEquals(new int[] {1, 3}, partition.isr); + assertEquals(1, partition.leader); + } + + // ================================================================================== + // DISKLESS-SPECIFIC ISR SEMANTICS: Liveness-gated ISR expansion + // + // For diskless, ISR should be liveness-gated, not lag-gated: + // - Replicas never diverge (data in remote storage) + // - Unfenced broker should immediately rejoin ISR (no catch-up needed) + // + // This is NEW behavior that differs from classic Kafka. + // ================================================================================== + + // [FUTURE - NEW BEHAVIOR] Diskless ISR expands on unfence without lag checks. + // + // Classic Kafka requires: follower catch-up via fetcher, then AlterPartition from leader. + // Diskless: liveness alone is sufficient (replicas can't diverge). + // + // Blocked on: + // 1. Diskless RF>1 topic creation (Phase 1) + // 2. Liveness-gated ISR expansion (new controller logic) + @DisabledIf("disklessRfFutureTestsDisabled") + @Test + void isrExpandsOnUnfence_disklessLivenessGated() { + ReplicationControlTestContext ctx = new ReplicationControlTestContext.Builder() + .setDefaultDisklessEnable(true) + .setDisklessStorageSystemEnabled(true) + .build(); + ReplicationControlManager replication = ctx.replicationControl; + ctx.registerBrokers(1, 2, 3); + ctx.unfenceBrokers(1, 2, 3); + + CreatableTopicResult topic = ctx.createTestTopic( + "diskless-rf3", new int[][] {new int[] {1, 2, 3}}, + Map.of(DISKLESS_ENABLE_CONFIG, "true"), + NONE.code()); + + Uuid topicId = topic.topicId(); + + // Fence broker 2 - ISR should shrink (this works today) + List records = new ArrayList<>(); + replication.handleBrokerFenced(2, records); + ctx.replay(records); + + PartitionRegistration partition = replication.getPartition(topicId, 0); + assertArrayEquals(new int[] {1, 3}, partition.isr, + "ISR should shrink when broker is fenced"); + + // Unfence broker 2 - for diskless, ISR should expand immediately (NEW BEHAVIOR) + // For classic Kafka, this would require AlterPartition from leader after catch-up + records.clear(); + replication.handleBrokerUnfenced(2, 102, records); + ctx.replay(records); + + partition = replication.getPartition(topicId, 0); + assertArrayEquals(new int[] {1, 2, 3}, partition.isr, + "Diskless ISR should expand on unfence (liveness-gated)"); + assertEquals(1, partition.leader, + "Leader should remain stable during ISR expansion"); + } + + // [FUTURE] Controlled shutdown for diskless topics. + // Blocked on: Diskless RF>1 topic creation. + @DisabledIf("disklessRfFutureTestsDisabled") + @Test + void controlledShutdownRemovesFromIsrAndReelectsLeader_disklessTopic() { + ReplicationControlTestContext ctx = new ReplicationControlTestContext.Builder() + .setDefaultDisklessEnable(true) + .setDisklessStorageSystemEnabled(true) + .build(); + ReplicationControlManager replication = ctx.replicationControl; + ctx.registerBrokers(1, 2, 3); + ctx.unfenceBrokers(1, 2, 3); + + CreatableTopicResult topic = ctx.createTestTopic( + "diskless-rf3", new int[][] {new int[] {1, 2, 3}}, + Map.of(DISKLESS_ENABLE_CONFIG, "true"), + NONE.code()); + + Uuid topicId = topic.topicId(); + List records = new ArrayList<>(); + replication.handleBrokerInControlledShutdown(1, 101, records); + ctx.replay(records); + + PartitionRegistration partition = replication.getPartition(topicId, 0); + assertNotNull(partition); + assertArrayEquals(new int[] {2, 3}, partition.isr); + assertTrue(partition.leader == 2 || partition.leader == 3); + } + + // ================================================================================== + // Additional Phase 0 tests: Broker unregistration and multiple partition scenarios + // ================================================================================== + + // Broker unregistration removes broker from ISR across all partitions. + // Finding: handleBrokerUnregistered is metadata-driven. + @Test + void brokerUnregistrationRemovesFromAllPartitions_classicTopic() { + ReplicationControlTestContext ctx = new ReplicationControlTestContext.Builder() + .setDisklessStorageSystemEnabled(true) + .build(); + ReplicationControlManager replication = ctx.replicationControl; + ctx.registerBrokers(1, 2, 3); + ctx.unfenceBrokers(1, 2, 3); + + // Create multiple partitions with broker 2 in ISR + CreatableTopicResult topic = ctx.createTestTopic( + "multi-partition", new int[][] { + new int[] {1, 2, 3}, // partition 0 + new int[] {2, 3, 1}, // partition 1 (leader is 2) + new int[] {3, 1, 2} // partition 2 + }); + + Uuid topicId = topic.topicId(); + + // Unregister broker 2 + List records = new ArrayList<>(); + replication.handleBrokerUnregistered(2, 102, records); + ctx.replay(records); + + // Verify broker 2 is removed from ISR of all partitions + for (int partitionId = 0; partitionId < 3; partitionId++) { + PartitionRegistration partition = replication.getPartition(topicId, partitionId); + assertNotNull(partition, "Partition " + partitionId + " should exist"); + for (int isr : partition.isr) { + assertNotEquals(2, isr, + "Broker 2 should be removed from ISR of partition " + partitionId); + } + assertNotEquals(2, partition.leader, + "Broker 2 should not be leader of partition " + partitionId); + } + } + + // [FUTURE] Broker unregistration for diskless topics. + // Blocked on: Diskless RF>1 topic creation. + @DisabledIf("disklessRfFutureTestsDisabled") + @Test + void brokerUnregistrationRemovesFromAllPartitions_disklessTopic() { + ReplicationControlTestContext ctx = new ReplicationControlTestContext.Builder() + .setDefaultDisklessEnable(true) + .setDisklessStorageSystemEnabled(true) + .build(); + ReplicationControlManager replication = ctx.replicationControl; + ctx.registerBrokers(1, 2, 3); + ctx.unfenceBrokers(1, 2, 3); + + CreatableTopicResult topic = ctx.createTestTopic( + "diskless-multi", new int[][] { + new int[] {1, 2, 3}, + new int[] {2, 3, 1} + }, + Map.of(DISKLESS_ENABLE_CONFIG, "true"), + NONE.code()); + + Uuid topicId = topic.topicId(); + + List records = new ArrayList<>(); + replication.handleBrokerUnregistered(2, 102, records); + ctx.replay(records); + + for (int partitionId = 0; partitionId < 2; partitionId++) { + PartitionRegistration partition = replication.getPartition(topicId, partitionId); + assertNotNull(partition); + for (int isr : partition.isr) { + assertNotEquals(2, isr); + } + } + } + } private static final Logger log = LoggerFactory.getLogger(ReplicationControlManagerTest.class); private static final int BROKER_SESSION_TIMEOUT_MS = 1000; From 8b2a9506e3da4b9a93dfc129e3990b267a6c87fb Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Mon, 19 Jan 2026 16:01:07 +0200 Subject: [PATCH 18/21] add controller server-config to gate feature --- .../ts-unification/DISKLESS_MANAGED_RF.md | 36 ++++++++++++++++--- 1 file changed, 31 insertions(+), 5 deletions(-) diff --git a/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md b/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md index 0cc1fde583..d0d0e60b87 100644 --- a/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md +++ b/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md @@ -58,7 +58,7 @@ moving replicas in KRaft. This keeps the design simple while preserving the "alw 1. [Motivation](#motivation) 2. [Objectives](#objectives) 3. [Design: Transformer-First Availability](#design-transformer-first-availability) -4. [Activation Model](#activation-model-binary-version) +4. [Activation Model](#activation-model-controller-server-config) 5. [Placement Model](#placement-model) 6. [Controller Behavior](#controller-behavior) 7. [Broker Behavior](#broker-behavior) @@ -506,17 +506,43 @@ It can make instant routing decisions without waiting for controller. --- -## Activation Model (Binary Version) +## Activation Model (Controller Server Config) -Managed RF is activated by **deploying a new binary version**. No feature flags, no topic attributes, no server configs. +Managed RF is activated by a **controller server configuration**. This allows incremental PRs to be merged without impacting +existing clusters, and enables controlled rollout per environment. + +### Configuration + +```properties +# Controller config (server.properties) +diskless.managed.rf.enabled=false # default: disabled for backward compatibility +``` + +When `diskless.managed.rf.enabled=true`: +- New diskless topics use KRaft-managed placement (one replica per rack at creation) +- Transformer filters KRaft placement by client AZ +- **Falls back to alive brokers if assigned replicas are offline** + +When `diskless.managed.rf.enabled=false` (default): +- Diskless topics use legacy "RF=1 / faked metadata" behavior +- Transformer calculates synthetic placement via hashing + +### Rollout Strategy + +This config-based activation enables: + +1. **Safe incremental PRs**: Code changes can be merged without affecting existing clusters +2. **Per-environment enablement**: Enable in dev/staging first, then production +3. **Instant rollback**: Disable config without binary rollback if issues arise +4. **Mixed-state clusters**: Existing RF=1 topics continue working; new topics get RF=rack_count when enabled ### Behavior Summary -**Old binary:** +**Config disabled (default):** - Diskless topics use legacy "RF=1 / faked metadata" behavior - Transformer calculates synthetic placement via hashing -**New binary:** +**Config enabled:** - Diskless topics use KRaft-managed placement (one replica per rack at creation) - Transformer filters KRaft placement by client AZ - **Falls back to alive brokers if assigned replicas are offline** From 00628a52d224fceca97ad8d56a8844e2fa2e6626 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Tue, 20 Jan 2026 22:16:47 +0200 Subject: [PATCH 19/21] fixup! add controller server-config to gate feature --- .../ts-unification/DISKLESS_MANAGED_RF.md | 41 ++++++++++++------- 1 file changed, 26 insertions(+), 15 deletions(-) diff --git a/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md b/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md index d0d0e60b87..a45bc71fa0 100644 --- a/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md +++ b/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md @@ -518,14 +518,20 @@ existing clusters, and enables controlled rollout per environment. diskless.managed.rf.enabled=false # default: disabled for backward compatibility ``` +**This config only affects topic creation.** Routing behavior is derived from topic config and KRaft metadata, not this +server config. + +| Aspect | What controls it | +|--------|------------------| +| Topic creation (RF assignment) | `diskless.managed.rf.enabled` server config | +| Routing mode (`DISKLESS_TIERED` vs `DISKLESS_ONLY`) | Topic config (`remote.storage.enable` + `diskless.enable`) | +| Routing logic | KRaft metadata — always prefers assigned replicas first; hash fallback only when replicas offline | + When `diskless.managed.rf.enabled=true`: -- New diskless topics use KRaft-managed placement (one replica per rack at creation) -- Transformer filters KRaft placement by client AZ -- **Falls back to alive brokers if assigned replicas are offline** +- New diskless topics get RF=rack_count with one replica per AZ When `diskless.managed.rf.enabled=false` (default): -- Diskless topics use legacy "RF=1 / faked metadata" behavior -- Transformer calculates synthetic placement via hashing +- New diskless topics get RF=1 (legacy behavior) ### Rollout Strategy @@ -533,19 +539,24 @@ This config-based activation enables: 1. **Safe incremental PRs**: Code changes can be merged without affecting existing clusters 2. **Per-environment enablement**: Enable in dev/staging first, then production -3. **Instant rollback**: Disable config without binary rollback if issues arise -4. **Mixed-state clusters**: Existing RF=1 topics continue working; new topics get RF=rack_count when enabled +3. **Instant rollback**: Disable config to stop creating new RF>1 topics; existing topics unaffected +4. **Mixed-state clusters**: RF=1 and RF>1 diskless topics coexist seamlessly + +### Backward Compatibility -### Behavior Summary +**Config toggle is safe and reversible:** -**Config disabled (default):** -- Diskless topics use legacy "RF=1 / faked metadata" behavior -- Transformer calculates synthetic placement via hashing +| Scenario | Behavior | +|----------|----------| +| Disable after RF>1 topics exist | Existing topics keep working with replica-first routing; new topics get RF=1 | +| Re-enable after disabling | New topics get RF>1 again; no impact on existing topics | +| Mixed RF=1 and RF>1 topics | All topics prefer assigned replicas; hash fallback only when replicas offline | -**Config enabled:** -- Diskless topics use KRaft-managed placement (one replica per rack at creation) -- Transformer filters KRaft placement by client AZ -- **Falls back to alive brokers if assigned replicas are offline** +**Why this works:** +- Transformer derives routing from **KRaft metadata** (assigned replicas), not server config +- Transformer derives mode from **topic config** (`remote.storage.enable`), not server config +- **All RF values** (including RF=1) prefer assigned replicas first; hash is fallback only +- No behavioral cliff when toggling config — topics behave consistently based on their own state ### Existing Diskless Topics From 2a0ebeebb85ddf08ddc61fba66088d82ce5cdc23 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Tue, 20 Jan 2026 22:19:50 +0200 Subject: [PATCH 20/21] clarify az-availability over replica placement consistency Ensure cross-az is the last resort when routing requests, and that replica placement is fully decoupled. --- .../ts-unification/DISKLESS_MANAGED_RF.md | 118 +++++++++++++----- 1 file changed, 88 insertions(+), 30 deletions(-) diff --git a/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md b/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md index a45bc71fa0..2094ff0aca 100644 --- a/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md +++ b/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md @@ -26,8 +26,8 @@ the tiering pipeline. - Real KRaft-managed replicas with rack-aware placement **At runtime (routing):** -- **Hybrid/tiered partitions**: metadata returns replica brokers only (tiered reads require `UnifiedLog`) -- **Diskless-only partitions**: metadata prefers replicas but falls back to any alive broker if all are offline +- **Hybrid/tiered partitions (`DISKLESS_TIERED`)**: metadata returns replica brokers only (tiered reads require `UnifiedLog`); prefers same-AZ replica, falls back to cross-AZ replica +- **Diskless-only partitions (`DISKLESS_ONLY`)**: metadata prefers same-AZ replica, but can fall back to any same-AZ broker before going cross-AZ **Key simplification:** No controller auto-reassignment. Availability is handled by the transformer at routing time, not by moving replicas in KRaft. This keeps the design simple while preserving the "always available" property. @@ -36,7 +36,7 @@ moving replicas in KRaft. This keeps the design simple while preserving the "alw | Trade-off | Accepted | Rationale | |-----------|----------|-----------| -| KRaft metadata may show offline brokers | Yes | Availability is not blocked; eventual consistency is acceptable | +| KRaft metadata (internal) may show offline brokers | Yes | This is internal controller state only; clients never see offline brokers — transformer filters them out before responding | | `describe topic` may be temporarily stale | Yes | Operator can see actual state via metrics; availability is real | | Tiered reads require replica brokers | Yes | Necessary for RLM correctness; can revisit with "tiered reads from any broker" option | @@ -480,8 +480,9 @@ Unlike classic topics where data lives on specific brokers (making KRaft metadat **Proposed behavior (RF=rack_count, real replicas):** - KRaft stores RF=rack_count with real broker IDs - Transformer filters by client AZ from KRaft replicas -- **If assigned replica is offline → fall back to any alive broker in AZ** -- Partition is always available (same as today!) +- **`DISKLESS_TIERED`**: same-AZ replica preferred, cross-AZ replica as fallback (replicas only) +- **`DISKLESS_ONLY`**: same-AZ replica preferred, same-AZ any broker next, then cross-AZ +- Partition is always available for `DISKLESS_ONLY` (same as today!) - KRaft metadata may be temporarily stale (shows offline broker) ### Why This Works for Diskless @@ -563,10 +564,21 @@ This config-based activation enables: Existing diskless topics (RF=1) continue to work: **Immediate availability:** -- Transformer sees RF=1 with potentially offline broker -- Falls back to hash-based selection (same as today) +- Transformer sees RF=1 with a single assigned replica +- **If replica is online**: routes to that replica (preferred) +- **If replica is offline**: falls back to hash-based selection (same availability as today) - No downtime +**Note on RF=1 as valid managed state:** + +RF=1 is a valid outcome of managed RF when brokers have no rack configuration (rack_count=1). In this case: +- The single replica is still a "real" KRaft-managed replica +- Routing **prefers** this replica when online +- Hash fallback only triggers when the replica is offline + +This is intentional: even with RF=1, we want deterministic routing to the assigned broker when possible (for metrics, +debugging, job ownership), while preserving "always available" semantics via fallback. + **Note on RF < rack_count:** The same fallback logic applies to any RF < rack_count (including RF=1). This is intentional: @@ -736,50 +748,96 @@ ensure diskless partitions never create fetcher state. ### Filtering Logic -This is the concrete encoding of the “offset-unaware metadata” constraint described in -[Tiered Storage (TS) reading — Important Consideration](#tiered-storage-ts-reading--important-consideration) and the “key -decision” in [Approach Comparison](#approach-comparison). +This is the concrete encoding of the "offset-unaware metadata" constraint described in +[Tiered Storage (TS) reading — Important Consideration](#tiered-storage-ts-reading--important-consideration) and the "key +decision" in [Approach Comparison](#approach-comparison). + +**Key principle:** Cross-AZ routing is a **last resort** for both modes. The transformer prioritizes same-AZ brokers first, +only falling back to cross-AZ when no same-AZ option exists. + +**Why `DISKLESS_TIERED` is constrained to replicas:** Tiered Storage reads via `RemoteLogManager` require `UnifiedLog` state, +which only exists on replica brokers. Therefore, `DISKLESS_TIERED` cannot fall back to non-replica brokers even if they are +in the same AZ — doing so would result in fetch failures for tiered data. + +**Scope note:** This feature defines the routing logic for both modes, but `DISKLESS_TIERED` integration with +`Partition`/`UnifiedLog`/RLM is a **follow-up effort** (see [Broker Behavior](#broker-behavior)). The routing logic is +defined here to ensure the transformer design is forward-compatible. + +**Future flexibility:** If [Tiered Storage reads from any broker](#decision-option-keep-alive-allow-tiered-storage-rlm-reads-from-any-broker) +is implemented (making RLM reads stateless/proxy-able), then `DISKLESS_TIERED` could adopt the same fallback flexibility as +`DISKLESS_ONLY`. This enhancement can be safely built on top of this feature foundation. + +| Priority | `DISKLESS_TIERED` (replicas only) | `DISKLESS_ONLY` (any broker allowed) | +|----------|-----------------------------------|--------------------------------------| +| 1st | Replica in client's AZ | Replica in client's AZ | +| 2nd | Replica in other AZ (cross-AZ) | Any broker in client's AZ | +| 3rd | Unavailable (Kafka semantics) | Replica in other AZ (cross-AZ) | +| 4th | — | Any broker in any AZ (absolute last resort) | ``` FOR each diskless partition: # Important: Metadata responses are offset-unaware, so routing must be # conservative when tiered reads are possible. - mode = migrationState(tp) # DISKLESS_ONLY or DISKLESS_TIERED (has tiered data) + + # Mode is derived from topic config: + # - DISKLESS_TIERED: remote.storage.enable=true AND diskless.enable=true + # - DISKLESS_ONLY: remote.storage.enable=false AND diskless.enable=true + mode = IF topic.remote.storage.enable THEN DISKLESS_TIERED ELSE DISKLESS_ONLY assigned_replicas = KRaft replica set alive_replicas = assigned_replicas ∩ alive_brokers + alive_replicas_in_client_az = alive_replicas ∩ brokers_in(clientAZ) + all_alive_brokers_in_client_az = alive_brokers ∩ brokers_in(clientAZ) IF mode == DISKLESS_TIERED: # Tiered reads may be required -> must stay on replicas that have UnifiedLog/RLM state - IF alive_replicas is not empty: - RETURN alive_replicas (prefer clientAZ, else cross-AZ) + # Priority: same-AZ replica > cross-AZ replica > unavailable + + IF alive_replicas_in_client_az is not empty: + RETURN alive_replicas_in_client_az # Best: replica in same AZ + + ELSE IF alive_replicas is not empty: + # No replica in client AZ; use replica in other AZ (cross-AZ, but still a replica) + RETURN alive_replicas + ELSE: - # Yes, this loses availability — but tiered reads REQUIRE replica brokers. - # This is standard Kafka semantics: partition unavailable until replica returns. + # All replicas offline — partition unavailable (standard Kafka semantics) + # Tiered reads REQUIRE replica brokers with UnifiedLog/RLM state. + # Cannot fall back to non-replica brokers even if they're in the same AZ. # Note: This scenario is rare (all replicas down) and temporary. - # Rolling restarts or upgrades replacing nodes won't hit this path — at least one replica is alive. RETURN assigned_replicas (partition unavailable) + ELSE: # DISKLESS_ONLY # Diskless-only: any broker can serve, so we can preserve "always available" - IF alive_replicas is not empty: - # Normal case: use KRaft placement - IF any alive_replica in clientAZ: - RETURN local replica (AZ-aware routing) - ELSE: - RETURN all alive_replicas (cross-AZ fallback) + # Priority: same-AZ replica > same-AZ any broker > cross-AZ replica > cross-AZ any broker + + IF alive_replicas_in_client_az is not empty: + RETURN alive_replicas_in_client_az # Best: assigned replica in same AZ + + ELSE IF all_alive_brokers_in_client_az is not empty: + # Assigned replica in client AZ is offline, but other brokers in AZ are alive + RETURN hash-based selection from all_alive_brokers_in_client_az + + ELSE IF alive_replicas is not empty: + # No brokers in client AZ available; use replica in other AZ (cross-AZ) + RETURN alive_replicas + ELSE: - # All assigned replicas offline: fall back to hash - RETURN hash-based selection from all alive brokers in clientAZ + # All assigned replicas offline AND no brokers in client AZ + # Absolute last resort: any alive broker (cross-AZ, non-replica) + RETURN hash-based selection from all alive_brokers ``` ### Key Properties -1. **Instant availability (`DISKLESS_ONLY`)**: No waiting for controller; falls back to hash if all replicas offline -2. **AZ-aware when possible**: Uses KRaft placement if alive -3. **Graceful degradation (`DISKLESS_ONLY`)**: Falls back to hash-based selection if needed -4. **Tiered-safe routing (`DISKLESS_TIERED`)**: Only replica brokers are returned (RLM requires `UnifiedLog`) -5. **Availability trade-off (`DISKLESS_TIERED`)**: If all replicas offline, partition is unavailable (standard Kafka semantics) -6. **State-dependent semantics**: Hybrid/tiered behaves like Kafka; diskless-only is always available +1. **Cross-AZ is last resort (both modes)**: Same-AZ routing is always preferred; cross-AZ only when no same-AZ option exists +2. **Consistent AZ priority**: Both `DISKLESS_TIERED` and `DISKLESS_ONLY` try same-AZ first, then fall back to cross-AZ +3. **Instant availability (`DISKLESS_ONLY`)**: No waiting for controller; multiple fallback tiers including non-replica brokers +4. **Graceful degradation (`DISKLESS_ONLY`)**: Falls back through: same-AZ replica → same-AZ any broker → cross-AZ replica → cross-AZ any broker +5. **Tiered-safe routing (`DISKLESS_TIERED`)**: Only replica brokers are returned (RLM requires `UnifiedLog`); same-AZ replica preferred, then cross-AZ replica +6. **Availability trade-off (`DISKLESS_TIERED`)**: If all replicas offline, partition is unavailable — cannot use non-replica brokers even in same AZ (standard Kafka semantics) +7. **State-dependent semantics**: Both modes prioritize AZ locality; difference is whether non-replica fallback is allowed +8. **RF=1 handled correctly**: The same logic applies to RF=1 — the single replica is preferred when online; hash fallback only when that replica is offline --- From cedfb4d5345d074935fd7d7a8fa914f7a3be6cee Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Tue, 20 Jan 2026 22:20:34 +0200 Subject: [PATCH 21/21] adjust dependencies on other work streams and phases --- docs/inkless/ts-unification/DESIGN.md | 40 +++++-------------- .../ts-unification/DISKLESS_MANAGED_RF.md | 24 ++++++----- 2 files changed, 23 insertions(+), 41 deletions(-) diff --git a/docs/inkless/ts-unification/DESIGN.md b/docs/inkless/ts-unification/DESIGN.md index 855033e0e0..12e922544b 100644 --- a/docs/inkless/ts-unification/DESIGN.md +++ b/docs/inkless/ts-unification/DESIGN.md @@ -560,6 +560,9 @@ public class DisklessMigrationHandler { **Objective:** Enable diskless topics to use 3 actual Kafka replicas while preserving write-to-any semantics. +**Activation:** Controlled by `diskless.managed.rf.enabled` controller config (default: `false`). This allows incremental +rollout — when enabled, new diskless topics get RF=rack_count; existing topics are unaffected. + #### 4.4.1 Current vs. Proposed Model **Current Model:** @@ -621,37 +624,12 @@ public class AppendHandler { #### 4.4.4 Metadata Transformer Updates -```java -// storage/inkless/src/main/java/io/aiven/inkless/metadata/InklessTopicMetadataTransformer.java -public class InklessTopicMetadataTransformer { - - public void transformClusterMetadata( - final String clientId, - final Iterable topicMetadata - ) { - for (final var topic : topicMetadata) { - if (!metadataView.isDisklessTopic(topic.name())) { - continue; - } - - for (final var partition : topic.partitions()) { - if (metadataView.isHybridTopic(topic.name())) { - // Hybrid topic: use actual replica assignment for RLM compatibility - // Leader selection based on partition metadata - // No transformation needed - use real replicas - } else { - // Pure diskless: current behavior - route to any broker - int leader = selectLeaderForInklessPartitions(clientId, topic.topicId(), - partition.partitionIndex()); - partition.setLeaderId(leader); - partition.setReplicaNodes(List.of(leader)); - partition.setIsrNodes(List.of(leader)); - } - } - } - } -} -``` +> **📄 See [DISKLESS_MANAGED_RF.md](DISKLESS_MANAGED_RF.md)** for the complete routing algorithm, including AZ-priority +> logic, mode derivation, and fallback behavior. + +**Summary:** The transformer derives routing mode from topic config (`remote.storage.enable` + `diskless.enable`) and +routes requests with AZ-priority. Both modes prefer assigned replicas first; `DISKLESS_ONLY` can fall back to any broker +when replicas are unavailable, while `DISKLESS_TIERED` stays on replicas only (RLM requires `UnifiedLog` state). ### 4.5 Stream 5: RLM Integration for Hybrid Topics diff --git a/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md b/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md index 2094ff0aca..d3fa42e8ab 100644 --- a/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md +++ b/docs/inkless/ts-unification/DISKLESS_MANAGED_RF.md @@ -736,12 +736,17 @@ reassignment tooling remains available. ## Broker Behavior - No replica fetcher threads for diskless topics (invariant: must not regress) -- No local `Partition` objects (for now — may add for RLM coordination later) +- No local `Partition` objects for this phase - Inkless handlers (`AppendHandler`, `FetchHandler`) serve requests **Regression guardrail:** diskless topics must not start follower fetchers or attempt log replication. Add tests/metrics to ensure diskless partitions never create fetcher state. +**Note on `DISKLESS_TIERED`:** This document defines the routing logic for `DISKLESS_TIERED` (replicas-only, AZ-priority), +but the full `Partition`/`UnifiedLog` integration for serving tiered data is a **follow-up effort**. See [DESIGN.md](DESIGN.md) +for the overall tiered storage unification roadmap. This feature (Managed RF) provides the foundation — real KRaft-managed +replicas — that the tiered integration will build upon. + --- ## Metadata Transformation @@ -945,18 +950,17 @@ on synthetic hashing. ### Phase 1: Topic Creation with Rack-Aware Placement (2 weeks) -1. Modify `ReplicationControlManager` to detect diskless topics -2. Compute RF = rack count at creation -3. Implement one-per-rack broker selection -4. Reject `replicationFactor > 1` and manual assignments +1. Add `diskless.managed.rf.enabled` controller config (default: `false`) +2. Modify `ReplicationControlManager` to detect diskless topics when config enabled +3. Compute RF = rack count at creation +4. Implement one-per-rack broker selection +5. Reject `replicationFactor > 1` and manual assignments ### Phase 2: Transformer Changes (2 weeks) -1. Update `InklessTopicMetadataTransformer` to read KRaft placement -2. Implement AZ filtering logic -3. Add offline replica routing — if KRaft replica offline, route to alive replica -4. Add hash fallback — if all replicas offline, use legacy hash -5. Add metrics for fallback tracking +1. Update `InklessTopicMetadataTransformer` to read KRaft placement when RF > 1 +2. Implement AZ-priority routing with mode-aware fallback (`DISKLESS_TIERED` vs `DISKLESS_ONLY`) +3. Add metrics for fallback tracking ### Phase 3: Add Partitions Support (1 week)