Skip to content

feat(diskless): add managed replicas routing to metadata transformer#504

Merged
giuseppelillo merged 5 commits intomainfrom
jeqo/pod-2121-transformer-replicas
Mar 13, 2026
Merged

feat(diskless): add managed replicas routing to metadata transformer#504
giuseppelillo merged 5 commits intomainfrom
jeqo/pod-2121-transformer-replicas

Conversation

@jeqo
Copy link
Copy Markdown
Contributor

@jeqo jeqo commented Feb 9, 2026

Adds AZ-aware routing support for diskless topics with managed replicas (RF > 1). The transformer now routes metadata requests based on replica assignments and remote storage configuration.

Changes

  • Refactor: Change MetadataView.getAliveBrokerNodes() to return List<Node> for consistent API
  • Routing Logic: Add isRemoteStorageEnabled check to determine routing constraints:
    • Tiered mode (remote storage enabled): same-AZ replica → cross-AZ replica → unavailable
    • Diskless-only mode: same-AZ replica → same-AZ any broker → cross-AZ replica → cross-AZ any broker
  • Offline replicas: Track and report offline replicas per partition; ISR reflects only alive replicas
  • Error code: Preserve original KRaft error code (e.g. LEADER_NOT_AVAILABLE) when partition is unavailable; clear to NONE only when a routable leader is found
  • Metrics: Add transformer metrics for observability:
    • fallback-total: Count of fallbacks to non-replica brokers (diskless-only)
    • offline-replicas-routed-around: Routing decisions when some replicas are offline
    • cross-az-routing-total: Requests routed to different AZ than client
  • Logging: Remove per-partition logs in favor of metrics for sustained observability
  • Performance: Lazy-init alive broker snapshot — only computed on first diskless topic, reused across all topics in the response

Routing Priority

Mode Priority
Tiered same-AZ replica → cross-AZ replica → unavailable
Diskless-only same-AZ replica → same-AZ any broker → cross-AZ replica → cross-AZ any broker

For unmanaged replicas (RF = 1), the legacy hash-based selection from all brokers is preserved.

Test plan

  • Unit tests for managed replica routing (same-AZ, cross-AZ, fallback paths)
  • Unit tests for tiered vs diskless-only mode behavior
  • Unit tests for partial replica failure (some replicas offline)
  • Unit tests for all-replicas-offline (unavailable vs fallback)
  • Assertions for leaderEpoch, errorCode, offlineReplicas, isrNodes on all managed tests
  • Unit tests for transformer metrics (ClientAzAwarenessMetrics)
  • Unit tests for describe-topic response path (both modes)

@jeqo jeqo force-pushed the jeqo/pod-2001-controller-topic-metrics branch from 6ddf9d8 to 742bb4f Compare February 9, 2026 14:48
@jeqo jeqo force-pushed the jeqo/pod-2121-transformer-replicas branch from deb173b to 2d516d0 Compare February 9, 2026 14:48
@jeqo jeqo force-pushed the jeqo/pod-2001-controller-topic-metrics branch from 742bb4f to 4de2bb4 Compare February 9, 2026 19:36
@jeqo jeqo force-pushed the jeqo/pod-2121-transformer-replicas branch from 2d516d0 to 8763a1f Compare February 9, 2026 19:37
@jeqo jeqo force-pushed the jeqo/pod-2001-controller-topic-metrics branch from 4de2bb4 to 919d6a7 Compare March 4, 2026 13:54
@jeqo jeqo force-pushed the jeqo/pod-2121-transformer-replicas branch from 8763a1f to 064cb93 Compare March 4, 2026 13:56
@jeqo jeqo force-pushed the jeqo/pod-2001-controller-topic-metrics branch 6 times, most recently from 0380c0b to 90fa55a Compare March 10, 2026 17:22
Base automatically changed from jeqo/pod-2001-controller-topic-metrics to main March 11, 2026 12:53
… List

Change return type from Iterable<Node> to List<Node> for simpler
downstream usage.
The underlying KRaftMetadataCache already returns a List, so this
removes unnecessary abstraction.
@jeqo jeqo force-pushed the jeqo/pod-2121-transformer-replicas branch from 064cb93 to 22ee4b1 Compare March 11, 2026 12:57
@jeqo jeqo requested a review from Copilot March 11, 2026 12:58
Copy link
Copy Markdown
Contributor

Copilot AI left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Pull request overview

Adds AZ-aware routing for diskless topics with managed replicas (RF > 1), with different fallback behavior depending on whether remote (tiered) storage is enabled, and introduces new transformer observability metrics.

Changes:

  • Refactors diskless metadata transformation to support managed-replica routing with tiered vs diskless-only priority rules.
  • Extends MetadataView to expose isRemoteStorageEnabled(topicName) and returns List<Node> for alive brokers.
  • Adds new metrics (fallback-total, offline-replicas-routed-around, cross-az-routing-total) and expands unit tests.

Reviewed changes

Copilot reviewed 7 out of 7 changed files in this pull request and generated 5 comments.

Show a summary per file
File Description
storage/inkless/src/main/java/io/aiven/inkless/metadata/InklessTopicMetadataTransformer.java Implements managed-replica routing logic and integrates new metrics.
storage/inkless/src/main/java/io/aiven/inkless/metadata/ClientAzAwarenessMetrics.java Adds new meters and recording APIs for transformer routing behavior.
storage/inkless/src/main/java/io/aiven/inkless/control_plane/MetadataView.java Updates API for alive brokers list and adds remote-storage flag method.
storage/inkless/src/test/java/io/aiven/inkless/metadata/InklessTopicMetadataTransformerTest.java Updates legacy RF=1 assumptions and adds managed-replica routing tests (cluster metadata).
storage/inkless/src/test/java/io/aiven/inkless/metadata/ClientAzAwarenessMetricsTest.java Adds tests for newly introduced meters.
core/src/main/scala/kafka/server/metadata/InklessMetadataView.scala Implements isRemoteStorageEnabled and updates alive brokers return type.
core/src/test/scala/kafka/server/metadata/InklessMetadataViewTest.scala Adds unit coverage for isRemoteStorageEnabled.

💡 Add Copilot custom instructions for smarter, more guided reviews. Learn how to get started.

Comment thread core/src/test/scala/kafka/server/metadata/InklessMetadataViewTest.scala Outdated
Add MetadataView.isRemoteStorageEnabled() to check topic tiering
config, needed for managed replicas routing decisions.
@jeqo jeqo force-pushed the jeqo/pod-2121-transformer-replicas branch from 22ee4b1 to 0a3563d Compare March 11, 2026 17:31
@jeqo jeqo requested a review from Copilot March 11, 2026 17:32
Copy link
Copy Markdown
Contributor

Copilot AI left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Pull request overview

Copilot reviewed 7 out of 7 changed files in this pull request and generated 2 comments.

Comments suppressed due to low confidence (1)

storage/inkless/src/main/java/io/aiven/inkless/metadata/InklessTopicMetadataTransformer.java:399

  • selectLeaderLegacy claims null client AZ is handled without a special check, but brokersInAZ(listenerName, clientAZ) will return brokers whose rack() is null when clientAZ is null (because Objects.equals(bm.rack(), null) is true). That makes routing prefer rack-less brokers instead of selecting from all alive brokers when the client AZ is unknown. Consider short-circuiting brokersInAZ/this call to return an empty list when clientAZ is null so the logic truly falls back to allAliveBrokers.
        final String clientAZ = ClientAZExtractor.getClientAZ(clientId);
        // This gracefully handles the null client AZ, no need for a special check.
        final List<Node> brokersInClientAZ = brokersInAZ(listenerName, clientAZ);
        // Fall back on all brokers if no broker in the client AZ.
        final List<Node> brokersToPickFrom = brokersInClientAZ.isEmpty()
            ? allAliveBrokers(listenerName)
            : brokersInClientAZ;

💡 Add Copilot custom instructions for smarter, more guided reviews. Learn how to get started.

@jeqo jeqo force-pushed the jeqo/pod-2121-transformer-replicas branch from 0a3563d to adf1424 Compare March 11, 2026 18:49
@jeqo jeqo requested a review from Copilot March 11, 2026 18:49
Copy link
Copy Markdown
Contributor

Copilot AI left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Pull request overview

Copilot reviewed 7 out of 7 changed files in this pull request and generated 5 comments.


💡 Add Copilot custom instructions for smarter, more guided reviews. Learn how to get started.

@jeqo jeqo force-pushed the jeqo/pod-2121-transformer-replicas branch 2 times, most recently from 02dd3c1 to 1e40a71 Compare March 12, 2026 12:17
@jeqo jeqo requested a review from Copilot March 12, 2026 12:33

This comment was marked as outdated.

@jeqo jeqo force-pushed the jeqo/pod-2121-transformer-replicas branch from 153b6c2 to 5691af2 Compare March 12, 2026 13:25
@jeqo jeqo requested a review from Copilot March 12, 2026 13:25

This comment was marked as outdated.

@jeqo jeqo force-pushed the jeqo/pod-2121-transformer-replicas branch from 5691af2 to 1468d2b Compare March 12, 2026 14:02
@jeqo jeqo requested a review from Copilot March 12, 2026 14:03

This comment was marked as outdated.

@jeqo jeqo force-pushed the jeqo/pod-2121-transformer-replicas branch from 1468d2b to 2881e84 Compare March 12, 2026 14:29
@jeqo jeqo requested a review from Copilot March 12, 2026 14:29
Copy link
Copy Markdown
Contributor

Copilot AI left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Pull request overview

Copilot reviewed 7 out of 7 changed files in this pull request and generated no new comments.


💡 Add Copilot custom instructions for smarter, more guided reviews. Learn how to get started.

Add support for managed replicas (RF > 1) in diskless topic metadata
transformation:

- For managed replicas, routing priority depends on remote storage:
  - Tiered (remote storage enabled): same-AZ replica > cross-AZ replica
    > unavailable
  - Diskless-only: same-AZ replica > same-AZ any broker > cross-AZ
    replica > cross-AZ any broker
- For unmanaged replicas (RF=1), preserve legacy hash-based selection
- Lazy-init alive broker snapshot to avoid unnecessary calls when no
  diskless topics are in the response

Add metrics to track routing behavior and remove per-partition logging
in favor of metrics for sustained observability:

- fallback-total: Count of fallbacks to non-replica brokers
  (diskless-only)
- offline-replicas-routed-around: Routing decisions when some replicas
  are offline
- cross-az-routing-total: Requests routed to a different AZ than the
  client

Update existing tests to use RF=1 to preserve original unmanaged
behavior.

Existing logging removed to rely on introduced metrics for Diskless
partitions.
@jeqo jeqo force-pushed the jeqo/pod-2121-transformer-replicas branch from 2881e84 to ddc2889 Compare March 12, 2026 14:44
@jeqo jeqo marked this pull request as ready for review March 12, 2026 15:17
metricsGroup.removeMetric(FALLBACK_TOTAL);
metricsGroup.removeMetric(OFFLINE_REPLICAS_ROUTED_AROUND);
metricsGroup.removeMetric(CROSS_AZ_ROUTING_TOTAL);
clientAzHitRatesPerAz.keySet().forEach(az -> metricsGroup.removeMetric(CLIENT_AZ_HIT_RATE, Map.of("client-az", az)));
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why is this renamed from "az" to "client-az" ?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It's a left-over bug -- when the tag was updated to "client-az" on creation it was left with the old value here. Adding a constant to fix this.

continue;
}

if (aliveNodes == null) {
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

As I see this part is reused below in transformDescribeTopicResponse. Moving it to a helper method might reduce the code the duplication.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Doing that in a fixup commit, PTAL

Copy link
Copy Markdown
Contributor

@giuseppelillo giuseppelillo left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM

@giuseppelillo giuseppelillo merged commit 039ecf3 into main Mar 13, 2026
6 checks passed
@giuseppelillo giuseppelillo deleted the jeqo/pod-2121-transformer-replicas branch March 13, 2026 14:59
AnatolyPopov pushed a commit that referenced this pull request Mar 23, 2026
…504)

* refactor(metadata): change MetadataView.getAliveBrokerNodes to return List

Change return type from Iterable<Node> to List<Node> for simpler
downstream usage.
The underlying KRaftMetadataCache already returns a List, so this
removes unnecessary abstraction.

* feat(metadata): add isRemoteStorageEnabled to MetadataView

Add MetadataView.isRemoteStorageEnabled() to check topic tiering
config, needed for managed replicas routing decisions.

* feat(metadata): add managed replicas routing and transformer metrics

Add support for managed replicas (RF > 1) in diskless topic metadata
transformation:

- For managed replicas, routing priority depends on remote storage:
  - Tiered (remote storage enabled): same-AZ replica > cross-AZ replica
    > unavailable
  - Diskless-only: same-AZ replica > same-AZ any broker > cross-AZ
    replica > cross-AZ any broker
- For unmanaged replicas (RF=1), preserve legacy hash-based selection
- Lazy-init alive broker snapshot to avoid unnecessary calls when no
  diskless topics are in the response

Add metrics to track routing behavior and remove per-partition logging
in favor of metrics for sustained observability:

- fallback-total: Count of fallbacks to non-replica brokers
  (diskless-only)
- offline-replicas-routed-around: Routing decisions when some replicas
  are offline
- cross-az-routing-total: Requests routed to a different AZ than the
  client

Update existing tests to use RF=1 to preserve original unmanaged
behavior.

Existing logging removed to rely on introduced metrics for Diskless
partitions.

* fixup! feat(metadata): add managed replicas routing and transformer metrics

* fixup! feat(metadata): add managed replicas routing and transformer metrics

(cherry picked from commit 039ecf3)
jeqo added a commit that referenced this pull request Mar 23, 2026
…504)

* refactor(metadata): change MetadataView.getAliveBrokerNodes to return List

Change return type from Iterable<Node> to List<Node> for simpler
downstream usage.
The underlying KRaftMetadataCache already returns a List, so this
removes unnecessary abstraction.

* feat(metadata): add isRemoteStorageEnabled to MetadataView

Add MetadataView.isRemoteStorageEnabled() to check topic tiering
config, needed for managed replicas routing decisions.

* feat(metadata): add managed replicas routing and transformer metrics

Add support for managed replicas (RF > 1) in diskless topic metadata
transformation:

- For managed replicas, routing priority depends on remote storage:
  - Tiered (remote storage enabled): same-AZ replica > cross-AZ replica
    > unavailable
  - Diskless-only: same-AZ replica > same-AZ any broker > cross-AZ
    replica > cross-AZ any broker
- For unmanaged replicas (RF=1), preserve legacy hash-based selection
- Lazy-init alive broker snapshot to avoid unnecessary calls when no
  diskless topics are in the response

Add metrics to track routing behavior and remove per-partition logging
in favor of metrics for sustained observability:

- fallback-total: Count of fallbacks to non-replica brokers
  (diskless-only)
- offline-replicas-routed-around: Routing decisions when some replicas
  are offline
- cross-az-routing-total: Requests routed to a different AZ than the
  client

Update existing tests to use RF=1 to preserve original unmanaged
behavior.

Existing logging removed to rely on introduced metrics for Diskless
partitions.

* fixup! feat(metadata): add managed replicas routing and transformer metrics

* fixup! feat(metadata): add managed replicas routing and transformer metrics
jeqo added a commit that referenced this pull request Mar 23, 2026
…504)

* refactor(metadata): change MetadataView.getAliveBrokerNodes to return List

Change return type from Iterable<Node> to List<Node> for simpler
downstream usage.
The underlying KRaftMetadataCache already returns a List, so this
removes unnecessary abstraction.

* feat(metadata): add isRemoteStorageEnabled to MetadataView

Add MetadataView.isRemoteStorageEnabled() to check topic tiering
config, needed for managed replicas routing decisions.

* feat(metadata): add managed replicas routing and transformer metrics

Add support for managed replicas (RF > 1) in diskless topic metadata
transformation:

- For managed replicas, routing priority depends on remote storage:
  - Tiered (remote storage enabled): same-AZ replica > cross-AZ replica
    > unavailable
  - Diskless-only: same-AZ replica > same-AZ any broker > cross-AZ
    replica > cross-AZ any broker
- For unmanaged replicas (RF=1), preserve legacy hash-based selection
- Lazy-init alive broker snapshot to avoid unnecessary calls when no
  diskless topics are in the response

Add metrics to track routing behavior and remove per-partition logging
in favor of metrics for sustained observability:

- fallback-total: Count of fallbacks to non-replica brokers
  (diskless-only)
- offline-replicas-routed-around: Routing decisions when some replicas
  are offline
- cross-az-routing-total: Requests routed to a different AZ than the
  client

Update existing tests to use RF=1 to preserve original unmanaged
behavior.

Existing logging removed to rely on introduced metrics for Diskless
partitions.

* fixup! feat(metadata): add managed replicas routing and transformer metrics

* fixup! feat(metadata): add managed replicas routing and transformer metrics

(cherry picked from commit 039ecf3)
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants