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
Copy file name to clipboardExpand all lines: CHANGELOG.md
+16-1Lines changed: 16 additions & 1 deletion
Display the source diff
Display the rich diff
Original file line number
Diff line number
Diff line change
@@ -2,6 +2,7 @@
2
2
3
3
librdkafka v2.11.0 is a feature release:
4
4
5
+
*[KIP-1102](https://cwiki.apache.org/confluence/display/KAFKA/KIP-1102%3A+Enable+clients+to+rebootstrap+based+on+timeout+or+error+code) Enable clients to rebootstrap based on timeout or error code (#4981).
5
6
* Fix for poll ratio calculation in case the queues are forwarded (#5017).
6
7
7
8
@@ -124,8 +125,22 @@ librdkafka v2.10.0 is a feature release:
124
125
> The [KIP-848](https://cwiki.apache.org/confluence/display/KAFKA/KIP-848%3A+The+Next+Generation+of+the+Consumer+Rebalance+Protocol) consumer is currently in **Preview** and should not be used in production environments. Implementation is feature complete but contract could have minor changes before General Availability.
125
126
126
127
128
+
## Upgrade considerations
129
+
130
+
131
+
Starting from this version, brokers not reported in Metadata RPC call are
132
+
removed along with their threads. Brokers and their threads are added back
133
+
when they appear in a Metadata RPC response again. When no brokers are left
134
+
or they're not reachable, the client will start a re-bootstrap sequence
135
+
by default. `metadata.recovery.strategy` controls this,
136
+
which defaults to `rebootstrap`.
137
+
Setting `metadata.recovery.strategy` to `none` avoids any re-bootstrapping and
138
+
leaves only the broker received in last successful metadata response.
139
+
140
+
127
141
## Enhancements and Fixes
128
142
143
+
*[KIP-899](https://cwiki.apache.org/confluence/display/KAFKA/KIP-899%3A+Allow+producer+and+consumer+clients+to+rebootstrap) Allow producer and consumer clients to rebootstrap
129
144
* Identify brokers only by broker id (#4557, @mfleming)
130
145
* Remove unavailable brokers and their thread (#4557, @mfleming)
131
146
* Commits during a cooperative incremental rebalance aren't causing
@@ -175,7 +190,7 @@ librdkafka v2.10.0 is a feature release:
175
190
and connection.
176
191
Happens since 1.x (#4557, @mfleming).
177
192
* Issues: #4557
178
-
Remove brokers not reported in a metadata call, along with their thread.
193
+
Remove brokers not reported in a metadata call, along with their threads.
179
194
Avoids that unavailable brokers are selected for a new connection when
180
195
there's no one available. We cannot tell if a broker was removed
181
196
temporarily or permanently so we always remove it and it'll be added back when
receive.message.max.bytes | * | 1000 .. 2147483647 | 100000000 | medium | Maximum Kafka protocol response message size. This serves as a safety precaution to avoid memory exhaustion in case of protocol hickups. This value must be at least `fetch.max.bytes` + 512 to allow for protocol overhead; the value is adjusted automatically unless the configuration property is explicitly set. <br>*Type: integer*
13
13
max.in.flight.requests.per.connection | * | 1 .. 1000000 | 1000000 | low | Maximum number of in-flight requests per broker connection. This is a generic property applied to all broker communication, however it is primarily relevant to produce requests. In particular, note that other mechanisms limit the number of outstanding consumer fetch request per broker to one. <br>*Type: integer*
14
14
max.in.flight | * | 1 .. 1000000 | 1000000 | low | Alias for `max.in.flight.requests.per.connection`: Maximum number of in-flight requests per broker connection. This is a generic property applied to all broker communication, however it is primarily relevant to produce requests. In particular, note that other mechanisms limit the number of outstanding consumer fetch request per broker to one. <br>*Type: integer*
15
-
metadata.recovery.strategy | * | none, rebootstrap | rebootstrap | low | Controls how the client recovers when none of the brokers known to it is available. If set to `none`, the client fails with a fatal error. If set to `rebootstrap`, the client repeats the bootstrap process using `bootstrap.servers` and brokers added through `rd_kafka_brokers_add()`. Rebootstrapping is useful when a client communicates with brokers so infrequently that the set of brokers may change entirely before the client refreshes metadata. Metadata recovery is triggered when all last-known brokers appear unavailable simultaneously. <br>*Type: enum value*
15
+
metadata.recovery.strategy | * | none, rebootstrap | rebootstrap | low | Controls how the client recovers when none of the brokers known to it is available. If set to `none`, the client doesn't re-bootstrap. If set to `rebootstrap`, the client repeats the bootstrap process using `bootstrap.servers` and brokers added through `rd_kafka_brokers_add()`. Rebootstrapping is useful when a client communicates with brokers so infrequently that the set of brokers may change entirely before the client refreshes metadata. Metadata recovery is triggered when all last-known brokers appear unavailable simultaneously or the client cannot refresh metadata within `metadata.recovery.rebootstrap.trigger.ms` or it's requested in a metadata response. <br>*Type: enum value*
16
+
metadata.recovery.rebootstrap.trigger.ms | * | 0 .. 2147483647 | 300000 | low | If a client configured to rebootstrap using `metadata.recovery.strategy=rebootstrap` is unable to obtain metadata from any of the brokers for this interval, client repeats the bootstrap process using `bootstrap.servers` configuration and brokers added through `rd_kafka_brokers_add()`. <br>*Type: integer*
16
17
topic.metadata.refresh.interval.ms | * | -1 .. 3600000 | 300000 | low | Period of time in milliseconds at which topic and broker metadata is refreshed in order to proactively discover any new brokers, topics, partitions or partition leader changes. Use -1 to disable the intervalled refresh (not recommended). If there are no locally referenced topics (no topic objects created, no messages produced, no subscription or no assignment) then only the broker list will be refreshed every interval but no more often than every 10s. <br>*Type: integer*
topic.metadata.refresh.fast.interval.ms | * | 1 .. 60000 | 100 | low | When a topic loses its leader a new metadata request will be enqueued immediately and then with this initial interval, exponentially increasing upto `retry.backoff.max.ms`, until the topic metadata has been refreshed. If not set explicitly, it will be defaulted to `retry.backoff.ms`. This is used to recover quickly from transitioning leader brokers. <br>*Type: integer*
0 commit comments