diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java
index 3851a65ded5..80f19d06941 100644
--- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java
+++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java
@@ -113,17 +113,17 @@ import static org.apache.kafka.common.config.ConfigDef.ValidString.in;
*
* By default, Kafka Streams does not allow users to overwrite the following properties (Streams setting shown in parentheses):
*
+ * - {@link ConsumerConfig#GROUP_ID_CONFIG "group.id"} (<application.id>) - Streams client will always use the application ID a consumer group ID
* - {@link ConsumerConfig#ENABLE_AUTO_COMMIT_CONFIG "enable.auto.commit"} (false) - Streams client will always disable/turn off auto committing
+ * - {@link ConsumerConfig#PARTITION_ASSIGNMENT_STRATEGY_CONFIG "partition.assignment.strategy"} (
StreamsPartitionAssignor
) - Streams client will always use its own partition assignor
*
*
* If {@link #PROCESSING_GUARANTEE_CONFIG "processing.guarantee"} is set to {@link #EXACTLY_ONCE "exactly_once"}, Kafka Streams does not allow users to overwrite the following properties (Streams setting shown in parentheses):
*
* - {@link ConsumerConfig#ISOLATION_LEVEL_CONFIG "isolation.level"} (read_committed) - Consumers will always read committed data only
* - {@link ProducerConfig#ENABLE_IDEMPOTENCE_CONFIG "enable.idempotence"} (true) - Producer will always have idempotency enabled
- * - {@link ProducerConfig#MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION "max.in.flight.requests.per.connection"} (5) - Producer will always have one in-flight request per connection
*
*
- *
* @see KafkaStreams#KafkaStreams(org.apache.kafka.streams.Topology, Properties)
* @see ConsumerConfig
* @see ProducerConfig