You signed in with another tab or window. Reload to refresh your session.You signed out in another tab or window. Reload to refresh your session.You switched accounts on another tab or window. Reload to refresh your session.Dismiss alert
This API enables Filters to map topic ids to topic names. Invoking this
method causes an out-of-band Metadata request to be sent to the broker
requesting the Metadata for the specified set of topic ids. The result
is then made available to clients.
We aim to always complete the stage with a TopicNameMapping that maps
every input topic id to a name or TopicNameMappingException.
TopicNameMappingException always offers a
org.apache.kafka.common.protocol.Errors enum to make it convenient for
clients to detect common cases like UNKNOWN_TOPIC_ID. Exceptions within
the proxy will be presented as UNKNOWN_SERVER_ERROR.
Why:
Kafka has gradually been transitioning to their RPCs using topicIds in
many cases to identify the topic to act on. The topic id is an internal
identifier intended to prevent some bugs that are possible if the RPCs
only carry names, around topic deletion especially. Kafka has recently
introduce topic id to the Produce request. Many of the proxy Filters
rely on the topic name in the Produce request to execute their logic, we
need a mechanism to obtain the topic names. Rather than having each
Filter implement this themselves through sendRequest, we offer it as a
framework method.
Future extensions:
- We could add framework level caching of the names per Filter
instance. Caching will have to be carefully considered to ensure
we invalidate appropriately.
- We could intercept client Metadata requests to learn what version the
client prefers to send. This would enable the proxy to more accurately
mimic the client and avoid exposing the Kafka cluster to novel patterns
like receiving multiple Metadata requests at different API versions.
Though this is not always possible, we do not always observe Metadata
requests on every connection.
Signed-off-by: Robert Young <[email protected]>
Copy file name to clipboardExpand all lines: CHANGELOG.md
+3Lines changed: 3 additions & 0 deletions
Display the source diff
Display the rich diff
Original file line number
Diff line number
Diff line change
@@ -6,6 +6,9 @@ For changes that effect a public API, the [deprecation policy](./DEV_GUIDE.md#de
6
6
Format `<github issue/pr number>: <short description>`.
7
7
8
8
## SNAPSHOT
9
+
10
+
*[#1318](https://github.com/kroxylicious/kroxylicious/issues/1318): Add FilterContext#topicNames to enable filters to retrieve names for topic ids
11
+
9
12
## 0.17.0
10
13
11
14
*[#2830](https://github.com/kroxylicious/kroxylicious/pull/2830): Refactor authentication configuration class names and optional fields in Azure KMS provider for Record Encryption
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
5
+
*/
6
+
7
+
packageio.kroxylicious.proxy.filter.metadata;
8
+
9
+
importjava.util.Map;
10
+
11
+
importorg.apache.kafka.common.Uuid;
12
+
importorg.apache.kafka.common.protocol.Errors;
13
+
14
+
/**
15
+
* The result of discovering the topic names for a collection of topic ids
16
+
*/
17
+
publicinterfaceTopicNameMapping {
18
+
/**
19
+
* @return true if there are any failures
20
+
*/
21
+
booleananyFailures();
22
+
23
+
/**
24
+
* @return immutable map from topic id to successfully mapped topic name
25
+
*/
26
+
Map<Uuid, String> topicNames();
27
+
28
+
/**
29
+
* Describes the reason for every failed mapping. Expected exception types are:
30
+
* <ul>
31
+
* <li>{@link TopLevelMetadataErrorException} indicates that we attempted to obtain Metadata from upstream, but received a top-level error in the response</li>
32
+
* <li>{@link TopicLevelMetadataErrorException} indicates that we attempted to obtain Metadata from upstream, but received a topic-level error in the response</li>
33
+
* <li>{@link TopicNameMappingException} can be used to convey any other exception</li>
34
+
* </ul>
35
+
* All the exception types offer {@link TopicNameMappingException#getError()} for conveniently determining the cause. Unhandled
36
+
* exceptions will be mapped to an {@link Errors#UNKNOWN_SERVER_ERROR}. Callers will be able to use this to detect expected
37
+
* cases like {@link Errors#UNKNOWN_TOPIC_ID}.
38
+
* @return immutable map from topic id to kafka server error
0 commit comments