Browse Source

KAFKA-8636; Add documentation change for max poll interval with static members (#7048)

Clarify the behavior of `max.poll.interval.ms` for static consumers since it is slightly different from dynamic members.

Reviewers: Bill Bejeck <bbejeck@gmail.com>, Matthias J. Sax <mjsax@apache.org>, Jason Gustafson <jason@confluent.io>
pull/7064/head
Boyang Chen 5 years ago committed by Jason Gustafson
parent
commit
65b044b200
  1. 55
      clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java

55
clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java

@ -43,8 +43,8 @@ public class CommonClientConfigs { @@ -43,8 +43,8 @@ public class CommonClientConfigs {
public static final String CLIENT_DNS_LOOKUP_CONFIG = "client.dns.lookup";
public static final String CLIENT_DNS_LOOKUP_DOC = "<p>Controls how the client uses DNS lookups.</p><p>If set to <code>use_all_dns_ips</code> then, when the lookup returns multiple IP addresses for a hostname,"
+ " they will all be attempted to connect to before failing the connection. Applies to both bootstrap and advertised servers.</p>"
+ "<p>If the value is <code>resolve_canonical_bootstrap_servers_only</code> each entry will be resolved and expanded into a list of canonical names.</p>";
+ " they will all be attempted to connect to before failing the connection. Applies to both bootstrap and advertised servers.</p>"
+ "<p>If the value is <code>resolve_canonical_bootstrap_servers_only</code> each entry will be resolved and expanded into a list of canonical names.</p>";
public static final String METADATA_MAX_AGE_CONFIG = "metadata.max.age.ms";
public static final String METADATA_MAX_AGE_DOC = "The period of time in milliseconds after which we force a refresh of metadata even if we haven't seen any partition leadership changes to proactively discover any new brokers or partitions.";
@ -105,38 +105,41 @@ public class CommonClientConfigs { @@ -105,38 +105,41 @@ public class CommonClientConfigs {
public static final String GROUP_ID_DOC = "A unique string that identifies the consumer group this consumer belongs to. This property is required if the consumer uses either the group management functionality by using <code>subscribe(topic)</code> or the Kafka-based offset management strategy.";
public static final String GROUP_INSTANCE_ID_CONFIG = "group.instance.id";
public static final String GROUP_INSTANCE_ID_DOC = "A unique identifier of the consumer instance provided by the end user. " +
"Only non-empty strings are permitted. If set, the consumer is treated as a static member, " +
"which means that only one instance with this ID is allowed in the consumer group at any time. " +
"This can be used in combination with a larger session timeout to avoid group rebalances caused by transient unavailability " +
"(e.g. process restarts). If not set, the consumer will join the group as a dynamic member, which is the traditional behavior.";
public static final String GROUP_INSTANCE_ID_DOC = "A unique identifier of the consumer instance provided by the end user. "
+ "Only non-empty strings are permitted. If set, the consumer is treated as a static member, "
+ "which means that only one instance with this ID is allowed in the consumer group at any time. "
+ "This can be used in combination with a larger session timeout to avoid group rebalances caused by transient unavailability "
+ "(e.g. process restarts). If not set, the consumer will join the group as a dynamic member, which is the traditional behavior.";
public static final String MAX_POLL_INTERVAL_MS_CONFIG = "max.poll.interval.ms";
public static final String MAX_POLL_INTERVAL_MS_DOC = "The maximum delay between invocations of poll() when using " +
"consumer group management. This places an upper bound on the amount of time that the consumer can be idle " +
"before fetching more records. If poll() is not called before expiration of this timeout, then the consumer " +
"is considered failed and the group will rebalance in order to reassign the partitions to another member. ";
public static final String MAX_POLL_INTERVAL_MS_DOC = "The maximum delay between invocations of poll() when using "
+ "consumer group management. This places an upper bound on the amount of time that the consumer can be idle "
+ "before fetching more records. If poll() is not called before expiration of this timeout, then the consumer "
+ "is considered failed and the group will rebalance in order to reassign the partitions to another member. "
+ "For consumers using a non-null <code>group.instance.id</code> which reach this timeout, partitions will not be immediately reassigned. "
+ "Instead, the consumer will stop sending heartbeats and partitions will be reassigned "
+ "after expiration of <code>session.timeout.ms</code>. This mirrors the behavior of a static consumer which has shutdown.";
public static final String REBALANCE_TIMEOUT_MS_CONFIG = "rebalance.timeout.ms";
public static final String REBALANCE_TIMEOUT_MS_DOC = "The maximum allowed time for each worker to join the group " +
"once a rebalance has begun. This is basically a limit on the amount of time needed for all tasks to " +
"flush any pending data and commit offsets. If the timeout is exceeded, then the worker will be removed " +
"from the group, which will cause offset commit failures.";
public static final String REBALANCE_TIMEOUT_MS_DOC = "The maximum allowed time for each worker to join the group "
+ "once a rebalance has begun. This is basically a limit on the amount of time needed for all tasks to "
+ "flush any pending data and commit offsets. If the timeout is exceeded, then the worker will be removed "
+ "from the group, which will cause offset commit failures.";
public static final String SESSION_TIMEOUT_MS_CONFIG = "session.timeout.ms";
public static final String SESSION_TIMEOUT_MS_DOC = "The timeout used to detect client failures when using " +
"Kafka's group management facility. The client sends periodic heartbeats to indicate its liveness " +
"to the broker. If no heartbeats are received by the broker before the expiration of this session timeout, " +
"then the broker will remove this client from the group and initiate a rebalance. Note that the value " +
"must be in the allowable range as configured in the broker configuration by <code>group.min.session.timeout.ms</code> " +
"and <code>group.max.session.timeout.ms</code>.";
public static final String SESSION_TIMEOUT_MS_DOC = "The timeout used to detect client failures when using "
+ "Kafka's group management facility. The client sends periodic heartbeats to indicate its liveness "
+ "to the broker. If no heartbeats are received by the broker before the expiration of this session timeout, "
+ "then the broker will remove this client from the group and initiate a rebalance. Note that the value "
+ "must be in the allowable range as configured in the broker configuration by <code>group.min.session.timeout.ms</code> "
+ "and <code>group.max.session.timeout.ms</code>.";
public static final String HEARTBEAT_INTERVAL_MS_CONFIG = "heartbeat.interval.ms";
public static final String HEARTBEAT_INTERVAL_MS_DOC = "The expected time between heartbeats to the consumer " +
"coordinator when using Kafka's group management facilities. Heartbeats are used to ensure that the " +
"consumer's session stays active and to facilitate rebalancing when new consumers join or leave the group. " +
"The value must be set lower than <code>session.timeout.ms</code>, but typically should be set no higher " +
"than 1/3 of that value. It can be adjusted even lower to control the expected time for normal rebalances.";
public static final String HEARTBEAT_INTERVAL_MS_DOC = "The expected time between heartbeats to the consumer "
+ "coordinator when using Kafka's group management facilities. Heartbeats are used to ensure that the "
+ "consumer's session stays active and to facilitate rebalancing when new consumers join or leave the group. "
+ "The value must be set lower than <code>session.timeout.ms</code>, but typically should be set no higher "
+ "than 1/3 of that value. It can be adjusted even lower to control the expected time for normal rebalances.";
/**
* Postprocess the configuration so that exponential backoff is disabled when reconnect backoff

Loading…
Cancel
Save