summaryrefslogtreecommitdiff
path: root/kafka/coordinator/consumer.py
blob: 48d5e148f34abbf82ddd820a8720e946183120ab (plain)
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
import copy
import collections
import logging
import time

import six

from .base import BaseCoordinator
import kafka.common as Errors
from kafka.common import OffsetAndMetadata, TopicPartition
from kafka.future import Future
from kafka.protocol.commit import (
    OffsetCommitRequest_v2, OffsetCommitRequest_v1, OffsetCommitRequest_v0,
    OffsetFetchRequest_v0, OffsetFetchRequest_v1)
from kafka.protocol.struct import Struct
from kafka.protocol.types import Array, Bytes, Int16, Int32, Schema, String

log = logging.getLogger(__name__)


class ConsumerProtocolMemberMetadata(Struct):
    SCHEMA = Schema(
        ('version', Int16),
        ('subscription', Array(String('utf-8'))),
        ('user_data', Bytes))


class ConsumerProtocolMemberAssignment(Struct):
    SCHEMA = Schema(
        ('version', Int16),
        ('assignment', Array(
            ('topic', String('utf-8')),
            ('partitions', Array(Int32)))),
        ('user_data', Bytes))

    def partitions(self):
        return [TopicPartition(topic, partition)
                for topic, partitions in self.assignment # pylint: disable-msg=no-member
                for partition in partitions]


class ConsumerProtocol(object):
    PROTOCOL_TYPE = 'consumer'
    ASSIGNMENT_STRATEGIES = ('roundrobin',)
    METADATA = ConsumerProtocolMemberMetadata
    ASSIGNMENT = ConsumerProtocolMemberAssignment


class ConsumerCoordinator(BaseCoordinator):
    """This class manages the coordination process with the consumer coordinator."""
    DEFAULT_CONFIG = {
        'group_id': 'kafka-python-default-group',
        'enable_auto_commit': True,
        'auto_commit_interval_ms': 5000,
        'default_offset_commit_callback': lambda offsets, response: True,
        'assignors': (),
        'session_timeout_ms': 30000,
        'heartbeat_interval_ms': 3000,
        'retry_backoff_ms': 100,
        'api_version': (0, 9),
    }

    def __init__(self, client, subscription, **configs):
        """Initialize the coordination manager.

        Keyword Arguments:
            group_id (str): name of the consumer group to join for dynamic
                partition assignment (if enabled), and to use for fetching and
                committing offsets. Default: 'kafka-python-default-group'
            enable_auto_commit (bool): If true the consumer's offset will be
                periodically committed in the background. Default: True.
            auto_commit_interval_ms (int): milliseconds between automatic
                offset commits, if enable_auto_commit is True. Default: 5000.
            default_offset_commit_callback (callable): called as
                callback(offsets, response) response will be either an Exception
                or a OffsetCommitResponse struct. This callback can be used to
                trigger custom actions when a commit request completes.
            assignors (list): List of objects to use to distribute partition
                ownership amongst consumer instances when group management is
                used. Default: [RoundRobinPartitionAssignor]
            heartbeat_interval_ms (int): The expected time in milliseconds
                between heartbeats to the consumer coordinator when using
                Kafka's group management feature. 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 session_timeout_ms, 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. Default: 3000
            session_timeout_ms (int): The timeout used to detect failures when
                using Kafka's group managementment facilities. Default: 30000
            retry_backoff_ms (int): Milliseconds to backoff when retrying on
                errors. Default: 100.
        """
        super(ConsumerCoordinator, self).__init__(client, **configs)
        self.config = copy.copy(self.DEFAULT_CONFIG)
        for key in self.config:
            if key in configs:
                self.config[key] = configs[key]

        self._cluster = client.cluster
        self._subscription = subscription
        self._partitions_per_topic = {}
        self._auto_commit_task = None
        if self.config['api_version'] >= (0, 9):
            assert self.config['assignors'], 'Coordinator require assignors'

        self._cluster.request_update()
        self._cluster.add_listener(self._handle_metadata_update)

        if self.config['api_version'] >= (0, 8, 1):
            if self.config['enable_auto_commit']:
                interval = self.config['auto_commit_interval_ms'] / 1000.0
                self._auto_commit_task = AutoCommitTask(self, interval)

        # metrics=None,
        # metric_group_prefix=None,
        # metric_tags=None,
        # self.sensors = ConsumerCoordinatorMetrics(metrics, metric_group_prefix, metric_tags)

    def protocol_type(self):
        return ConsumerProtocol.PROTOCOL_TYPE

    def group_protocols(self):
        """Returns list of preferred (protocols, metadata)"""
        topics = self._subscription.subscription
        metadata_list = []
        for assignor in self.config['assignors']:
            metadata = assignor.metadata(topics)
            group_protocol = (assignor.name, metadata)
            metadata_list.append(group_protocol)
        return metadata_list

    def _handle_metadata_update(self, cluster):
        # if we encounter any unauthorized topics, raise an exception
        # TODO
        #if self._cluster.unauthorized_topics:
        #    raise Errors.TopicAuthorizationError(self._cluster.unauthorized_topics)

        if self._subscription.subscribed_pattern:
            topics = []
            for topic in cluster.topics():
                if self._subscription.subscribed_pattern.match(topic):
                    topics.append(topic)

            self._subscription.change_subscription(topics)
            self._client.set_topics(self._subscription.group_subscription())

        # check if there are any changes to the metadata which should trigger a rebalance
        if self._subscription_metadata_changed():
            if self.config['api_version'] >= (0, 9):
                self._subscription.mark_for_reassignment()

            # If we haven't got group coordinator support,
            # just assign all partitions locally
            else:
                self._subscription.assign_from_subscribed([
                    TopicPartition(topic, partition)
                    for topic in self._subscription.subscription
                    for partition in self._partitions_per_topic[topic]
                ])

    def _subscription_metadata_changed(self):
        if not self._subscription.partitions_auto_assigned():
            return False

        old_partitions_per_topic = self._partitions_per_topic
        self._partitions_per_topic = {}
        for topic in self._subscription.group_subscription():
            self._partitions_per_topic[topic] = set(self._cluster.partitions_for_topic(topic))

        if self._partitions_per_topic != old_partitions_per_topic:
            return True
        return False

    def _lookup_assignor(self, name):
        for assignor in self.config['assignors']:
            if assignor.name == name:
                return assignor
        return None

    def _on_join_complete(self, generation, member_id, protocol,
                          member_assignment_bytes):
        assignor = self._lookup_assignor(protocol)
        assert assignor, 'invalid assignment protocol: %s' % protocol

        assignment = ConsumerProtocol.ASSIGNMENT.decode(member_assignment_bytes)

        # set the flag to refresh last committed offsets
        self._subscription.needs_fetch_committed_offsets = True

        # update partition assignment
        self._subscription.assign_from_subscribed(assignment.partitions())

        # give the assignor a chance to update internal state
        # based on the received assignment
        assignor.on_assignment(assignment)

        # restart the autocommit task if needed
        if self.config['enable_auto_commit']:
            self._auto_commit_task.enable()

        assigned = set(self._subscription.assigned_partitions())
        log.debug("Set newly assigned partitions %s", assigned)

        # execute the user's callback after rebalance
        if self._subscription.listener:
            try:
                self._subscriptions.listener.on_partitions_assigned(assigned)
            except Exception:
                log.exception("User provided listener failed on partition"
                              " assignment: %s", assigned)

    def _perform_assignment(self, leader_id, assignment_strategy, members):
        assignor = self._lookup_assignor(assignment_strategy)
        assert assignor, 'Invalid assignment protocol: %s' % assignment_strategy
        member_metadata = {}
        all_subscribed_topics = set()
        for member_id, metadata_bytes in members:
            metadata = ConsumerProtocol.METADATA.decode(metadata_bytes)
            member_metadata[member_id] = metadata
            all_subscribed_topics.update(metadata.subscription) # pylint: disable-msg=no-member

        # the leader will begin watching for changes to any of the topics
        # the group is interested in, which ensures that all metadata changes
        # will eventually be seen
        # Because assignment typically happens within response callbacks,
        # we cannot block on metadata updates here (no recursion into poll())
        self._subscription.group_subscribe(all_subscribed_topics)
        self._client.set_topics(self._subscription.group_subscription())

        log.debug("Performing %s assignment for subscriptions %s",
                  assignor.name, member_metadata)

        assignments = assignor.assign(self._cluster, member_metadata)

        log.debug("Finished assignment: %s", assignments)

        group_assignment = {}
        for member_id, assignment in six.iteritems(assignments):
            group_assignment[member_id] = assignment
        return group_assignment

    def _on_join_prepare(self, generation, member_id):
        # commit offsets prior to rebalance if auto-commit enabled
        self._maybe_auto_commit_offsets_sync()

        # execute the user's callback before rebalance
        log.debug("Revoking previously assigned partitions %s",
                  self._subscription.assigned_partitions())
        if self._subscription.listener:
            try:
                revoked = set(self._subscription.assigned_partitions())
                self._subscription.listener.on_partitions_revoked(revoked)
            except Exception:
                log.exception("User provided subscription listener failed"
                              " on_partitions_revoked")

        self._subscription.mark_for_reassignment()

    def need_rejoin(self):
        """Check whether the group should be rejoined

        Returns:
            bool: True if consumer should rejoin group, False otherwise
        """
        return (self._subscription.partitions_auto_assigned() and
               (super(ConsumerCoordinator, self).need_rejoin() or
                self._subscription.needs_partition_assignment))

    def refresh_committed_offsets_if_needed(self):
        """Fetch committed offsets for assigned partitions."""
        if self._subscription.needs_fetch_committed_offsets:
            offsets = self.fetch_committed_offsets(self._subscription.assigned_partitions())
            for partition, offset in six.iteritems(offsets):
                # verify assignment is still active
                if self._subscription.is_assigned(partition):
                    self._subscription.assignment[partition].committed = offset.offset
            self._subscription.needs_fetch_committed_offsets = False

    def fetch_committed_offsets(self, partitions):
        """Fetch the current committed offsets for specified partitions

        Arguments:
            partitions (list of TopicPartition): partitions to fetch

        Returns:
            dict: {TopicPartition: OffsetAndMetadata}
        """
        while True:
            if self.config['api_version'] >= (0, 8, 2):
                self.ensure_coordinator_known()

            # contact coordinator to fetch committed offsets
            future = self._send_offset_fetch_request(partitions)
            self._client.poll(future=future)

            if future.succeeded():
                return future.value

            if not future.retriable():
                raise future.exception # pylint: disable-msg=raising-bad-type

            time.sleep(self.config['retry_backoff_ms'] / 1000.0)

    def close(self):
        try:
            self._maybe_auto_commit_offsets_sync()
        finally:
            super(ConsumerCoordinator, self).close()

    def commit_offsets_async(self, offsets, callback=None):
        """Commit specific offsets asynchronously.

        Arguments:
            offsets (dict {TopicPartition: OffsetAndMetadata}): what to commit
            callback (callable, optional): called as callback(offsets, response)
                response will be either an Exception or a OffsetCommitResponse
                struct. This callback can be used to trigger custom actions when
                a commit request completes.
        Returns:
            Future: indicating whether the commit was successful or not
        """
        if callback is None:
            callback = self.config['default_offset_commit_callback']
        self._subscription.needs_fetch_committed_offsets = True
        future = self._send_offset_commit_request(offsets)
        future.add_both(callback, offsets)

    def commit_offsets_sync(self, offsets):
        """Commit specific offsets synchronously.

        This method will retry until the commit completes successfully or an
        unrecoverable error is encountered.

        Arguments:
            offsets (dict {TopicPartition: OffsetAndMetadata}): what to commit

        Raises error on failure
        """
        if not offsets:
            return

        while True:
            if self.config['api_version'] >= (0, 8, 2):
                self.ensure_coordinator_known()

            future = self._send_offset_commit_request(offsets)
            self._client.poll(future=future)

            if future.succeeded():
                return

            if not future.retriable():
                raise future.exception # pylint: disable-msg=raising-bad-type

            time.sleep(self.config['retry_backoff_ms'] / 1000.0)

    def _maybe_auto_commit_offsets_sync(self):
        if self.config['api_version'] < (0, 8, 1):
            return
        if self.config['enable_auto_commit']:
            # disable periodic commits prior to committing synchronously. note that they will
            # be re-enabled after a rebalance completes
            self._auto_commit_task.disable()

            try:
                self.commit_offsets_sync(self._subscription.all_consumed_offsets())

            # The three main group membership errors are known and should not
            # require a stacktrace -- just a warning
            except (Errors.UnknownMemberIdError,
                    Errors.IllegalGenerationError,
                    Errors.RebalanceInProgressError):
                log.warning("Offset commit failed: group membership out of date"
                            " This is likely to cause duplicate message"
                            " delivery.")
            except Exception:
                log.exception("Offset commit failed: This is likely to cause"
                              " duplicate message delivery")

    def _send_offset_commit_request(self, offsets):
        """Commit offsets for the specified list of topics and partitions.

        This is a non-blocking call which returns a request future that can be
        polled in the case of a synchronous commit or ignored in the
        asynchronous case.

        Arguments:
            offsets (dict of {TopicPartition: OffsetAndMetadata}): what should
                be committed

        Returns:
            Future: indicating whether the commit was successful or not
        """
        if self.config['api_version'] >= (0, 8, 2):
            if self.coordinator_unknown():
                return Future().failure(Errors.GroupCoordinatorNotAvailableError)
            node_id = self.coordinator_id
        else:
            node_id = self._client.least_loaded_node()

        if not offsets:
            return Future().failure(None)

        # create the offset commit request
        offset_data = collections.defaultdict(dict)
        for tp, offset in six.iteritems(offsets):
            offset_data[tp.topic][tp.partition] = offset

        if self.config['api_version'] >= (0, 9):
            request = OffsetCommitRequest_v2(
                self.group_id,
                self.generation,
                self.member_id,
                OffsetCommitRequest_v2.DEFAULT_RETENTION_TIME,
                [(
                    topic, [(
                        partition,
                        offset.offset,
                        offset.metadata
                    ) for partition, offset in six.iteritems(partitions)]
                ) for topic, partitions in six.iteritems(offset_data)]
            )
        elif self.config['api_version'] >= (0, 8, 2):
            request = OffsetCommitRequest_v1(
                self.group_id, -1, '',
                [(
                    topic, [(
                        partition,
                        offset.offset,
                        -1,
                        offset.metadata
                    ) for partition, offset in six.iteritems(partitions)]
                ) for topic, partitions in six.iteritems(offset_data)]
            )
        elif self.config['api_version'] >= (0, 8, 1):
            request = OffsetCommitRequest_v0(
                self.group_id,
                [(
                    topic, [(
                        partition,
                        offset.offset,
                        offset.metadata
                    ) for partition, offset in six.iteritems(partitions)]
                ) for topic, partitions in six.iteritems(offset_data)]
            )

        log.debug("Sending offset-commit request with %s to %s",
                  offsets, node_id)

        future = Future()
        _f = self._client.send(node_id, request)
        _f.add_callback(self._handle_offset_commit_response, offsets, future)
        _f.add_errback(self._failed_request, future)
        return future

    def _handle_offset_commit_response(self, offsets, future, response):
        #self.sensors.commit_latency.record(response.requestLatencyMs())
        unauthorized_topics = set()

        for topic, partitions in response.topics:
            for partition, error_code in partitions:
                tp = TopicPartition(topic, partition)
                offset = offsets[tp]

                error_type = Errors.for_code(error_code)
                if error_type is Errors.NoError:
                    log.debug("Committed offset %s for partition %s", offset, tp)
                    if self._subscription.is_assigned(tp):
                        self._subscription.assignment[tp].committed = offset.offset
                elif error_type is Errors.GroupAuthorizationFailedError:
                    log.error("OffsetCommit failed for group %s - %s",
                              self.group_id, error_type.__name__)
                    future.failure(error_type(self.group_id))
                    return
                elif error_type is Errors.TopicAuthorizationFailedError:
                    unauthorized_topics.add(topic)
                elif error_type in (Errors.OffsetMetadataTooLargeError,
                                    Errors.InvalidCommitOffsetSizeError):
                    # raise the error to the user
                    log.info("OffsetCommit failed for group %s on partition %s"
                             " due to %s, will retry", self.group_id, tp,
                             error_type.__name__)
                    future.failure(error_type())
                    return
                elif error_type is Errors.GroupLoadInProgressError:
                    # just retry
                    log.info("OffsetCommit failed for group %s because group is"
                             " initializing (%s), will retry", self.group_id,
                             error_type.__name__)
                    future.failure(error_type(self.group_id))
                    return
                elif error_type in (Errors.GroupCoordinatorNotAvailableError,
                                    Errors.NotCoordinatorForGroupError,
                                    Errors.RequestTimedOutError):
                    log.info("OffsetCommit failed for group %s due to a"
                             " coordinator error (%s), will find new coordinator"
                             " and retry", self.group_id, error_type.__name__)
                    self.coordinator_dead()
                    future.failure(error_type(self.group_id))
                    return
                elif error_type in (Errors.UnknownMemberIdError,
                                    Errors.IllegalGenerationError,
                                    Errors.RebalanceInProgressError):
                    # need to re-join group
                    error = error_type(self.group_id)
                    log.error("OffsetCommit failed for group %s due to group"
                              " error (%s), will rejoin", self.group_id, error)
                    self._subscription.mark_for_reassignment()
                    # Errors.CommitFailedError("Commit cannot be completed due to group rebalance"))
                    future.failure(error)
                    return
                else:
                    log.error("OffsetCommit failed for group % on partition %s"
                              " with offset %s: %s", self.group_id, tp, offset,
                              error_type.__name__)
                    future.failure(error_type())
                    return

        if unauthorized_topics:
            log.error("OffsetCommit failed for unauthorized topics %s",
                      unauthorized_topics)
            future.failure(Errors.TopicAuthorizationFailedError(unauthorized_topics))
        else:
            future.success(True)

    def _send_offset_fetch_request(self, partitions):
        """Fetch the committed offsets for a set of partitions.

        This is a non-blocking call. The returned future can be polled to get
        the actual offsets returned from the broker.

        Arguments:
            partitions (list of TopicPartition): the partitions to fetch

        Returns:
            Future: resolves to dict of offsets: {TopicPartition: int}
        """
        if self.config['api_version'] >= (0, 8, 2):
            if self.coordinator_unknown():
                return Future().failure(Errors.GroupCoordinatorNotAvailableError)
            node_id = self.coordinator_id
        else:
            node_id = self._client.least_loaded_node()

        log.debug("Fetching committed offsets for partitions: %s", partitions)
        # construct the request
        topic_partitions = collections.defaultdict(set)
        for tp in partitions:
            topic_partitions[tp.topic].add(tp.partition)

        if self.config['api_version'] >= (0, 8, 2):
            request = OffsetFetchRequest_v1(
                self.group_id,
                list(topic_partitions.items())
            )
        else:
            request = OffsetFetchRequest_v0(
                self.group_id,
                list(topic_partitions.items())
            )

        # send the request with a callback
        future = Future()
        _f = self._client.send(node_id, request)
        _f.add_callback(self._handle_offset_fetch_response, future)
        _f.add_errback(self._failed_request, future)
        return future

    def _handle_offset_fetch_response(self, future, response):
        offsets = {}
        for topic, partitions in response.topics:
            for partition, offset, metadata, error_code in partitions:
                tp = TopicPartition(topic, partition)
                error_type = Errors.for_code(error_code)
                if error_type is not Errors.NoError:
                    error = error_type()
                    log.debug("Error fetching offset for %s: %s", tp, error_type())
                    if error_type is Errors.GroupLoadInProgressError:
                        # just retry
                        future.failure(error)
                    elif error_type is Errors.NotCoordinatorForGroupError:
                        # re-discover the coordinator and retry
                        self.coordinator_dead()
                        future.failure(error)
                    elif error_type in (Errors.UnknownMemberIdError,
                                        Errors.IllegalGenerationError):
                        # need to re-join group
                        self._subscription.mark_for_reassignment()
                        future.failure(error)
                    elif error_type is Errors.UnknownTopicOrPartitionError:
                        log.warning("OffsetFetchRequest -- unknown topic %s",
                                    topic)
                        continue
                    else:
                        log.error("Unknown error fetching offsets for %s: %s",
                                  tp, error)
                        future.failure(error)
                    return
                elif offset >= 0:
                    # record the position with the offset (-1 indicates no committed offset to fetch)
                    offsets[tp] = OffsetAndMetadata(offset, metadata)
                else:
                    log.debug("No committed offset for partition %s", tp)
        future.success(offsets)


class AutoCommitTask(object):
    def __init__(self, coordinator, interval):
        self._coordinator = coordinator
        self._client = coordinator._client
        self._interval = interval
        self._enabled = False
        self._request_in_flight = False

    def enable(self):
        if self._enabled:
            log.warning("AutoCommitTask is already enabled")
            return

        self._enabled = True
        if not self._request_in_flight:
            self._client.schedule(self, time.time() + self._interval)

    def disable(self):
        self._enabled = False
        try:
            self._client.unschedule(self)
        except KeyError:
            pass

    def _reschedule(self, at):
        assert self._enabled, 'AutoCommitTask not enabled'
        self._client.schedule(self, at)

    def __call__(self):
        if not self._enabled:
            return

        if self._coordinator.coordinator_unknown():
            log.debug("Cannot auto-commit offsets because the coordinator is"
                      " unknown, will retry after backoff")
            backoff = self._coordinator.config['retry_backoff_ms'] / 1000.0
            self._client.schedule(self, time.time() + backoff)
            return

        self._request_in_flight = True
        self._coordinator.commit_offsets_async(
            self._coordinator._subscription.all_consumed_offsets(),
            self._handle_commit_response)

    def _handle_commit_response(self, offsets, result):
        self._request_in_flight = False
        if result is True:
            log.debug("Successfully auto-committed offsets")
            next_at = time.time() + self._interval
        elif not isinstance(result, BaseException):
            raise Errors.IllegalStateError(
                'Unrecognized result in _handle_commit_response: %s'
                % result)
        elif hasattr(result, 'retriable') and result.retriable:
            log.debug("Failed to auto-commit offsets: %s, will retry"
                      " immediately", result)
            next_at = time.time()
        else:
            log.warning("Auto offset commit failed: %s", result)
            next_at = time.time() + self._interval

        if not self._enabled:
            log.warning("Skipping auto-commit reschedule -- it is disabled")
            return
        self._reschedule(next_at)


# TODO
"""
class ConsumerCoordinatorMetrics(object):
    def __init__(self, metrics, prefix, tags):
        self.metrics = metrics
        self.group_name = prefix + "-coordinator-metrics"

        self.commit_latency = metrics.sensor("commit-latency")
        self.commit_latency.add(metrics.MetricName(
            "commit-latency-avg", self.group_name,
            "The average time taken for a commit request",
            tags), metrics.Avg())
        self.commit_latency.add(metrics.MetricName(
            "commit-latency-max", self.group_name,
            "The max time taken for a commit request",
            tags), metrics.Max())
        self.commit_latency.add(metrics.MetricName(
            "commit-rate", self.group_name,
            "The number of commit calls per second",
            tags), metrics.Rate(metrics.Count()))

        '''
        def _num_partitions(config, now):
            new Measurable() {
                public double measure(MetricConfig config, long now) {
                    return subscriptions.assignedPartitions().size();
                }
            };
        metrics.addMetric(new MetricName("assigned-partitions",
            this.metricGrpName,
            "The number of partitions currently assigned to this consumer",
            tags),
            numParts);
        '''
"""