summaryrefslogtreecommitdiff
diff options
context:
space:
mode:
authorJeff Widman <jeff@jeffwidman.com>2019-03-14 17:34:59 -0700
committerDana Powers <dana.powers@gmail.com>2019-03-14 17:34:59 -0700
commit965d21b21387e69c53f583fd76cdcec2a4d0f944 (patch)
tree81b157cc73458aec84f5fc4687763f6a869c0d97
parent812de351f75beefe73bd9bef55847ab61ccc951d (diff)
downloadkafka-python-965d21b21387e69c53f583fd76cdcec2a4d0f944.tar.gz
Error if connections_max_idle_ms not larger than request_timeout_ms (#1688)
-rw-r--r--kafka/consumer/group.py10
-rw-r--r--test/test_consumer.py8
2 files changed, 13 insertions, 5 deletions
diff --git a/kafka/consumer/group.py b/kafka/consumer/group.py
index 531c107..f521891 100644
--- a/kafka/consumer/group.py
+++ b/kafka/consumer/group.py
@@ -313,11 +313,15 @@ class KafkaConsumer(six.Iterator):
new_config, self.config['auto_offset_reset'])
self.config['auto_offset_reset'] = new_config
+ connections_max_idle_ms = self.config['connections_max_idle_ms']
request_timeout_ms = self.config['request_timeout_ms']
fetch_max_wait_ms = self.config['fetch_max_wait_ms']
- if request_timeout_ms <= fetch_max_wait_ms:
- raise KafkaConfigurationError("Request timeout (%s) must be larger than fetch-max-wait-ms (%s)" %
- (request_timeout_ms, fetch_max_wait_ms))
+ if not (fetch_max_wait_ms < request_timeout_ms < connections_max_idle_ms):
+ raise KafkaConfigurationError(
+ "connections_max_idle_ms ({}) must be larger than "
+ "request_timeout_ms ({}) which must be larger than "
+ "fetch_max_wait_ms ({})."
+ .format(connections_max_idle_ms, request_timeout_ms, fetch_max_wait_ms))
metrics_tags = {'client-id': self.config['client_id']}
metric_config = MetricConfig(samples=self.config['metrics_num_samples'],
diff --git a/test/test_consumer.py b/test/test_consumer.py
index 4ea01c8..edcc2d8 100644
--- a/test/test_consumer.py
+++ b/test/test_consumer.py
@@ -15,11 +15,15 @@ from kafka.structs import (
class TestKafkaConsumer:
def test_session_timeout_larger_than_request_timeout_raises(self):
with pytest.raises(KafkaConfigurationError):
- KafkaConsumer(bootstrap_servers='localhost:9092', api_version=(0,9), group_id='foo', session_timeout_ms=60000, request_timeout_ms=40000)
+ KafkaConsumer(bootstrap_servers='localhost:9092', api_version=(0, 9), group_id='foo', session_timeout_ms=50000, request_timeout_ms=40000)
def test_fetch_max_wait_larger_than_request_timeout_raises(self):
with pytest.raises(KafkaConfigurationError):
- KafkaConsumer(bootstrap_servers='localhost:9092', fetch_max_wait_ms=41000, request_timeout_ms=40000)
+ KafkaConsumer(bootstrap_servers='localhost:9092', fetch_max_wait_ms=50000, request_timeout_ms=40000)
+
+ def test_request_timeout_larger_than_connections_max_idle_ms_raises(self):
+ with pytest.raises(KafkaConfigurationError):
+ KafkaConsumer(bootstrap_servers='localhost:9092', api_version=(0, 9), request_timeout_ms=50000, connections_max_idle_ms=40000)
def test_subscription_copy(self):
consumer = KafkaConsumer('foo', api_version=(0, 10))