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
KAFKA-18193 Refactor Kafka Streams CloseOptions to Fluent API Style (#19955)
In Kafka Streams, configuration classes typically follow a fluent API
pattern to ensure a consistent and intuitive developer experience.
However, the current implementation of
`org.apache.kafka.streams.KafkaStreams$CloseOptions` deviates from this
convention by exposing a public constructor, breaking the uniformity
expected across the API.
To address this inconsistency, we propose introducing a new
`CloseOptions` class that adheres to the fluent API style, replacing the
existing implementation. The new class will retain the existing
`timeout(Duration)` and `leaveGroup(boolean)` methods but will enforce
fluent instantiation and configuration. Given the design shift, we will
not maintain backward compatibility with the current class.
This change aligns with the goal of standardizing configuration objects
across Kafka Streams, offering developers a more cohesive and
predictable API.
Reviewers: Bill Bejeck<[email protected]>
Copy file name to clipboardExpand all lines: docs/upgrade.html
+7Lines changed: 7 additions & 0 deletions
Original file line number
Diff line number
Diff line change
@@ -189,6 +189,13 @@ <h5><a id="upgrade_420_notable" href="#upgrade_420_notable">Notable changes in 4
189
189
A new metric <code>AvgIdleRatio</code> has been added to the <code>ControllerEventManager</code> group. This metric measures the average idle ratio of the controller event queue thread,
190
190
providing visibility into how much time the controller spends waiting for events versus processing them. The metric value ranges from 0.0 (always busy) to 1.0 (always idle).
191
191
</li>
192
+
<li>
193
+
Deprecated <code>org.apache.kafka.streams.KafkaStreams$CloseOptions</code> and its related methods, such as
Copy file name to clipboardExpand all lines: streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsCloseOptionsIntegrationTest.java
* Shutdown this {@code KafkaStreams} by signaling all the threads to stop, and then wait up to the timeout for the
1596
1606
* threads to join.
1607
+
* This method is deprecated and replaced by {@link #close(org.apache.kafka.streams.CloseOptions)}.
1597
1608
* @param options contains timeout to specify how long to wait for the threads to shut down, and a flag leaveGroup to
1598
1609
* trigger consumer leave call
1599
1610
* @return {@code true} if all threads were successfully stopped—{@code false} if the timeout was reached
1600
1611
* before all threads stopped
1601
1612
* Note that this method must not be called in the {@link StateListener#onChange(KafkaStreams.State, KafkaStreams.State)} callback of {@link StateListener}.
1602
1613
* @throws IllegalArgumentException if {@code timeout} can't be represented as {@code long milliseconds}
* Shutdown this {@code KafkaStreams} by signaling all the threads to stop, and then wait up to the timeout for the
1626
+
* threads to join.
1627
+
* @param options contains timeout to specify how long to wait for the threads to shut down,
1628
+
* and a {@link org.apache.kafka.streams.CloseOptions.GroupMembershipOperation}
1629
+
* to trigger consumer leave call or remain in the group
1630
+
* @return {@code true} if all threads were successfully stopped—{@code false} if the timeout was reached
1631
+
* before all threads stopped
1632
+
* Note that this method must not be called in the {@link StateListener#onChange(KafkaStreams.State, KafkaStreams.State)} callback of {@link StateListener}.
1633
+
* @throws IllegalArgumentException if {@code timeout} can't be represented as {@code long milliseconds}
0 commit comments