summaryrefslogtreecommitdiff
path: root/kafka/conn.py
blob: 4c21b8cab5734462524121c51ad9e5f7a5986ec8 (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
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
1001
1002
1003
1004
1005
1006
1007
1008
1009
1010
1011
1012
1013
1014
1015
1016
1017
1018
1019
1020
1021
1022
1023
1024
1025
1026
1027
1028
1029
1030
1031
1032
1033
1034
1035
1036
1037
1038
1039
1040
1041
1042
1043
1044
1045
1046
1047
1048
1049
1050
1051
1052
1053
1054
1055
1056
1057
1058
1059
1060
1061
1062
1063
1064
1065
1066
1067
1068
1069
1070
1071
1072
1073
1074
1075
1076
1077
1078
1079
1080
1081
1082
1083
1084
1085
1086
1087
1088
1089
1090
1091
1092
1093
1094
1095
1096
1097
1098
1099
1100
1101
1102
1103
1104
1105
1106
1107
1108
1109
1110
1111
1112
1113
1114
1115
1116
1117
1118
1119
1120
1121
1122
1123
1124
1125
1126
1127
1128
1129
1130
1131
1132
1133
1134
1135
1136
1137
1138
1139
1140
1141
1142
1143
1144
1145
1146
1147
1148
1149
1150
1151
1152
1153
1154
1155
1156
1157
1158
1159
1160
1161
1162
1163
1164
1165
1166
from __future__ import absolute_import

import collections
import copy
import errno
import logging
from random import shuffle, uniform

# selectors in stdlib as of py3.4
try:
    import selectors  # pylint: disable=import-error
except ImportError:
    # vendored backport module
    from .vendor import selectors34 as selectors

import socket
import struct
import sys
import time

from kafka.vendor import six

import kafka.errors as Errors
from kafka.future import Future
from kafka.metrics.stats import Avg, Count, Max, Rate
from kafka.protocol.admin import SaslHandShakeRequest
from kafka.protocol.commit import OffsetFetchRequest
from kafka.protocol.metadata import MetadataRequest
from kafka.protocol.parser import KafkaProtocol
from kafka.protocol.types import Int32
from kafka.version import __version__


if six.PY2:
    ConnectionError = socket.error
    BlockingIOError = Exception

log = logging.getLogger(__name__)

DEFAULT_KAFKA_PORT = 9092

try:
    import ssl
    ssl_available = True
    try:
        SSLEOFError = ssl.SSLEOFError
        SSLWantReadError = ssl.SSLWantReadError
        SSLWantWriteError = ssl.SSLWantWriteError
        SSLZeroReturnError = ssl.SSLZeroReturnError
    except:
        # support older ssl libraries
        log.warning('Old SSL module detected.'
                    ' SSL error handling may not operate cleanly.'
                    ' Consider upgrading to Python 3.3 or 2.7.9')
        SSLEOFError = ssl.SSLError
        SSLWantReadError = ssl.SSLError
        SSLWantWriteError = ssl.SSLError
        SSLZeroReturnError = ssl.SSLError
except ImportError:
    # support Python without ssl libraries
    ssl_available = False
    class SSLWantReadError(Exception):
        pass
    class SSLWantWriteError(Exception):
        pass

# needed for SASL_GSSAPI authentication:
try:
    import gssapi
    from gssapi.raw.misc import GSSError
except ImportError:
    #no gssapi available, will disable gssapi mechanism
    gssapi = None
    GSSError = None

class ConnectionStates(object):
    DISCONNECTING = '<disconnecting>'
    DISCONNECTED = '<disconnected>'
    CONNECTING = '<connecting>'
    HANDSHAKE = '<handshake>'
    CONNECTED = '<connected>'
    AUTHENTICATING = '<authenticating>'


class BrokerConnection(object):
    """Initialize a Kafka broker connection

    Keyword Arguments:
        client_id (str): a name for this client. This string is passed in
            each request to servers and can be used to identify specific
            server-side log entries that correspond to this client. Also
            submitted to GroupCoordinator for logging with respect to
            consumer group administration. Default: 'kafka-python-{version}'
        reconnect_backoff_ms (int): The amount of time in milliseconds to
            wait before attempting to reconnect to a given host.
            Default: 50.
        reconnect_backoff_max_ms (int): The maximum amount of time in
            milliseconds to wait when reconnecting to a broker that has
            repeatedly failed to connect. If provided, the backoff per host
            will increase exponentially for each consecutive connection
            failure, up to this maximum. To avoid connection storms, a
            randomization factor of 0.2 will be applied to the backoff
            resulting in a random range between 20% below and 20% above
            the computed value. Default: 1000.
        request_timeout_ms (int): Client request timeout in milliseconds.
            Default: 40000.
        max_in_flight_requests_per_connection (int): Requests are pipelined
            to kafka brokers up to this number of maximum requests per
            broker connection. Default: 5.
        receive_buffer_bytes (int): The size of the TCP receive buffer
            (SO_RCVBUF) to use when reading data. Default: None (relies on
            system defaults). Java client defaults to 32768.
        send_buffer_bytes (int): The size of the TCP send buffer
            (SO_SNDBUF) to use when sending data. Default: None (relies on
            system defaults). Java client defaults to 131072.
        socket_options (list): List of tuple-arguments to socket.setsockopt
            to apply to broker connection sockets. Default:
            [(socket.IPPROTO_TCP, socket.TCP_NODELAY, 1)]
        security_protocol (str): Protocol used to communicate with brokers.
            Valid values are: PLAINTEXT, SSL, SASL_PLAINTEXT, SASL_SSL.
            Default: PLAINTEXT.
        ssl_context (ssl.SSLContext): pre-configured SSLContext for wrapping
            socket connections. If provided, all other ssl_* configurations
            will be ignored. Default: None.
        ssl_check_hostname (bool): flag to configure whether ssl handshake
            should verify that the certificate matches the brokers hostname.
            default: True.
        ssl_cafile (str): optional filename of ca file to use in certificate
            veriication. default: None.
        ssl_certfile (str): optional filename of file in pem format containing
            the client certificate, as well as any ca certificates needed to
            establish the certificate's authenticity. default: None.
        ssl_keyfile (str): optional filename containing the client private key.
            default: None.
        ssl_password (callable, str, bytes, bytearray): optional password or
            callable function that returns a password, for decrypting the
            client private key. Default: None.
        ssl_crlfile (str): optional filename containing the CRL to check for
            certificate expiration. By default, no CRL check is done. When
            providing a file, only the leaf certificate will be checked against
            this CRL. The CRL can only be checked with Python 3.4+ or 2.7.9+.
            default: None.
        api_version (tuple): Specify which Kafka API version to use.
            Accepted values are: (0, 8, 0), (0, 8, 1), (0, 8, 2), (0, 9),
            (0, 10). Default: (0, 8, 2)
        api_version_auto_timeout_ms (int): number of milliseconds to throw a
            timeout exception from the constructor when checking the broker
            api version. Only applies if api_version is None
        selector (selectors.BaseSelector): Provide a specific selector
            implementation to use for I/O multiplexing.
            Default: selectors.DefaultSelector
        state_change_callback (callable): function to be called when the
            connection state changes from CONNECTING to CONNECTED etc.
        metrics (kafka.metrics.Metrics): Optionally provide a metrics
            instance for capturing network IO stats. Default: None.
        metric_group_prefix (str): Prefix for metric names. Default: ''
        sasl_mechanism (str): Authentication mechanism when security_protocol
            is configured for SASL_PLAINTEXT or SASL_SSL. Valid values are:
            PLAIN, GSSAPI. Default: PLAIN
        sasl_plain_username (str): username for sasl PLAIN authentication.
            Default: None
        sasl_plain_password (str): password for sasl PLAIN authentication.
            Default: None
        sasl_kerberos_service_name (str): Service name to include in GSSAPI
            sasl mechanism handshake. Default: 'kafka'
    """

    DEFAULT_CONFIG = {
        'client_id': 'kafka-python-' + __version__,
        'node_id': 0,
        'request_timeout_ms': 40000,
        'reconnect_backoff_ms': 50,
        'reconnect_backoff_max_ms': 1000,
        'max_in_flight_requests_per_connection': 5,
        'receive_buffer_bytes': None,
        'send_buffer_bytes': None,
        'socket_options': [(socket.IPPROTO_TCP, socket.TCP_NODELAY, 1)],
        'security_protocol': 'PLAINTEXT',
        'ssl_context': None,
        'ssl_check_hostname': True,
        'ssl_cafile': None,
        'ssl_certfile': None,
        'ssl_keyfile': None,
        'ssl_crlfile': None,
        'ssl_password': None,
        'api_version': (0, 8, 2),  # default to most restrictive
        'selector': selectors.DefaultSelector,
        'state_change_callback': lambda conn: True,
        'metrics': None,
        'metric_group_prefix': '',
        'sasl_mechanism': 'PLAIN',
        'sasl_plain_username': None,
        'sasl_plain_password': None,
        'sasl_kerberos_service_name': 'kafka'
    }
    SECURITY_PROTOCOLS = ('PLAINTEXT', 'SSL', 'SASL_PLAINTEXT', 'SASL_SSL')
    SASL_MECHANISMS = ('PLAIN', 'GSSAPI')

    def __init__(self, host, port, afi, **configs):
        self.hostname = host
        self.host = host
        self.port = port
        self.afi = afi
        self._init_host = host
        self._init_port = port
        self._init_afi = afi
        self.in_flight_requests = collections.deque()
        self._api_versions = None

        self.config = copy.copy(self.DEFAULT_CONFIG)
        for key in self.config:
            if key in configs:
                self.config[key] = configs[key]

        self.node_id = self.config.pop('node_id')

        if self.config['receive_buffer_bytes'] is not None:
            self.config['socket_options'].append(
                (socket.SOL_SOCKET, socket.SO_RCVBUF,
                 self.config['receive_buffer_bytes']))
        if self.config['send_buffer_bytes'] is not None:
            self.config['socket_options'].append(
                 (socket.SOL_SOCKET, socket.SO_SNDBUF,
                 self.config['send_buffer_bytes']))

        assert self.config['security_protocol'] in self.SECURITY_PROTOCOLS, (
            'security_protcol must be in ' + ', '.join(self.SECURITY_PROTOCOLS))

        if self.config['security_protocol'] in ('SSL', 'SASL_SSL'):
            assert ssl_available, "Python wasn't built with SSL support"

        if self.config['security_protocol'] in ('SASL_PLAINTEXT', 'SASL_SSL'):
            assert self.config['sasl_mechanism'] in self.SASL_MECHANISMS, (
                'sasl_mechanism must be in ' + ', '.join(self.SASL_MECHANISMS))
            if self.config['sasl_mechanism'] == 'PLAIN':
                assert self.config['sasl_plain_username'] is not None, 'sasl_plain_username required for PLAIN sasl'
                assert self.config['sasl_plain_password'] is not None, 'sasl_plain_password required for PLAIN sasl'
            if self.config['sasl_mechanism'] == 'GSSAPI':
                assert gssapi is not None, 'GSSAPI lib not available'
                assert self.config['sasl_kerberos_service_name'] is not None, 'sasl_kerberos_service_name required for GSSAPI sasl'

        self._protocol = KafkaProtocol(
            client_id=self.config['client_id'],
            api_version=self.config['api_version'])
        self.state = ConnectionStates.DISCONNECTED
        self._reset_reconnect_backoff()
        self._sock = None
        self._ssl_context = None
        if self.config['ssl_context'] is not None:
            self._ssl_context = self.config['ssl_context']
        self._sasl_auth_future = None
        self.last_attempt = 0
        self._gai = None
        self._gai_index = 0
        self._sensors = None
        if self.config['metrics']:
            self._sensors = BrokerConnectionMetrics(self.config['metrics'],
                                                    self.config['metric_group_prefix'],
                                                    self.node_id)

    def connect(self):
        """Attempt to connect and return ConnectionState"""
        if self.state is ConnectionStates.DISCONNECTED:
            log.debug('%s: creating new socket', self)
            # if self.afi is set to AF_UNSPEC, then we need to do a name
            # resolution and try all available address families
            if self._init_afi == socket.AF_UNSPEC:
                if self._gai is None:
                    # XXX: all DNS functions in Python are blocking. If we really
                    # want to be non-blocking here, we need to use a 3rd-party
                    # library like python-adns, or move resolution onto its
                    # own thread. This will be subject to the default libc
                    # name resolution timeout (5s on most Linux boxes)
                    try:
                        self._gai = socket.getaddrinfo(self._init_host,
                                                       self._init_port,
                                                       socket.AF_UNSPEC,
                                                       socket.SOCK_STREAM)
                    except socket.gaierror as ex:
                        log.warning('DNS lookup failed for %s:%d,'
                                    ' exception was %s. Is your'
                                    ' advertised.listeners (called'
                                    ' advertised.host.name before Kafka 9)'
                                    ' correct and resolvable?',
                                    self._init_host, self._init_port, ex)
                        self._gai = []
                    self._gai_index = 0
                else:
                    # if self._gai already exists, then we should try the next
                    # name
                    self._gai_index += 1
                while True:
                    if self._gai_index >= len(self._gai):
                        error = 'Unable to connect to any of the names for {0}:{1}'.format(
                            self._init_host, self._init_port)
                        log.error(error)
                        self.close(Errors.ConnectionError(error))
                        return
                    afi, _, __, ___, sockaddr = self._gai[self._gai_index]
                    if afi not in (socket.AF_INET, socket.AF_INET6):
                        self._gai_index += 1
                        continue
                    break
                self.host, self.port = sockaddr[:2]
                self._sock = socket.socket(afi, socket.SOCK_STREAM)
            else:
                self._sock = socket.socket(self._init_afi, socket.SOCK_STREAM)

            for option in self.config['socket_options']:
                log.debug('%s: setting socket option %s', self, option)
                self._sock.setsockopt(*option)

            self._sock.setblocking(False)
            self.last_attempt = time.time()
            self.state = ConnectionStates.CONNECTING
            if self.config['security_protocol'] in ('SSL', 'SASL_SSL'):
                self._wrap_ssl()
            # _wrap_ssl can alter the connection state -- disconnects on failure
            # so we need to double check that we are still connecting before
            if self.connecting():
                self.config['state_change_callback'](self)
                log.info('%s: connecting to %s:%d', self, self.host, self.port)

        if self.state is ConnectionStates.CONNECTING:
            # in non-blocking mode, use repeated calls to socket.connect_ex
            # to check connection status
            request_timeout = self.config['request_timeout_ms'] / 1000.0
            ret = None
            try:
                ret = self._sock.connect_ex((self.host, self.port))
                # if we got here through a host lookup, we've found a host,port,af tuple
                # that works save it so we don't do a GAI lookup again
                if self._gai is not None:
                    self.afi = self._sock.family
                    self._gai = None
            except socket.error as err:
                ret = err.errno

            # Connection succeeded
            if not ret or ret == errno.EISCONN:
                log.debug('%s: established TCP connection', self)
                if self.config['security_protocol'] in ('SSL', 'SASL_SSL'):
                    log.debug('%s: initiating SSL handshake', self)
                    self.state = ConnectionStates.HANDSHAKE
                elif self.config['security_protocol'] == 'SASL_PLAINTEXT':
                    log.debug('%s: initiating SASL authentication', self)
                    self.state = ConnectionStates.AUTHENTICATING
                else:
                    # security_protocol PLAINTEXT
                    log.debug('%s: Connection complete.', self)
                    self.state = ConnectionStates.CONNECTED
                    self._reset_reconnect_backoff()
                self.config['state_change_callback'](self)

            # Connection failed
            # WSAEINVAL == 10022, but errno.WSAEINVAL is not available on non-win systems
            elif ret not in (errno.EINPROGRESS, errno.EALREADY, errno.EWOULDBLOCK, 10022):
                log.error('Connect attempt to %s returned error %s.'
                          ' Disconnecting.', self, ret)
                self.close(Errors.ConnectionError(ret))

            # Connection timed out
            elif time.time() > request_timeout + self.last_attempt:
                log.error('Connection attempt to %s timed out', self)
                self.close(Errors.ConnectionError('timeout'))

            # Needs retry
            else:
                pass

        if self.state is ConnectionStates.HANDSHAKE:
            if self._try_handshake():
                log.debug('%s: completed SSL handshake.', self)
                if self.config['security_protocol'] == 'SASL_SSL':
                    log.debug('%s: initiating SASL authentication', self)
                    self.state = ConnectionStates.AUTHENTICATING
                else:
                    log.debug('%s: Connection complete.', self)
                    self.state = ConnectionStates.CONNECTED
                self.config['state_change_callback'](self)

        if self.state is ConnectionStates.AUTHENTICATING:
            assert self.config['security_protocol'] in ('SASL_PLAINTEXT', 'SASL_SSL')
            if self._try_authenticate():
                # _try_authenticate has side-effects: possibly disconnected on socket errors
                if self.state is ConnectionStates.AUTHENTICATING:
                    log.debug('%s: Connection complete.', self)
                    self.state = ConnectionStates.CONNECTED
                    self._reset_reconnect_backoff()
                    self.config['state_change_callback'](self)

        return self.state

    def _wrap_ssl(self):
        assert self.config['security_protocol'] in ('SSL', 'SASL_SSL')
        if self._ssl_context is None:
            log.debug('%s: configuring default SSL Context', self)
            self._ssl_context = ssl.SSLContext(ssl.PROTOCOL_SSLv23)  # pylint: disable=no-member
            self._ssl_context.options |= ssl.OP_NO_SSLv2  # pylint: disable=no-member
            self._ssl_context.options |= ssl.OP_NO_SSLv3  # pylint: disable=no-member
            self._ssl_context.verify_mode = ssl.CERT_OPTIONAL
            if self.config['ssl_check_hostname']:
                self._ssl_context.check_hostname = True
            if self.config['ssl_cafile']:
                log.info('%s: Loading SSL CA from %s', self, self.config['ssl_cafile'])
                self._ssl_context.load_verify_locations(self.config['ssl_cafile'])
                self._ssl_context.verify_mode = ssl.CERT_REQUIRED
            if self.config['ssl_certfile'] and self.config['ssl_keyfile']:
                log.info('%s: Loading SSL Cert from %s', self, self.config['ssl_certfile'])
                log.info('%s: Loading SSL Key from %s', self, self.config['ssl_keyfile'])
                self._ssl_context.load_cert_chain(
                    certfile=self.config['ssl_certfile'],
                    keyfile=self.config['ssl_keyfile'],
                    password=self.config['ssl_password'])
            if self.config['ssl_crlfile']:
                if not hasattr(ssl, 'VERIFY_CRL_CHECK_LEAF'):
                    raise RuntimeError('This version of Python does not support ssl_crlfile!')
                log.info('%s: Loading SSL CRL from %s', self, self.config['ssl_crlfile'])
                self._ssl_context.load_verify_locations(self.config['ssl_crlfile'])
                # pylint: disable=no-member
                self._ssl_context.verify_flags |= ssl.VERIFY_CRL_CHECK_LEAF
        log.debug('%s: wrapping socket in ssl context', self)
        try:
            self._sock = self._ssl_context.wrap_socket(
                self._sock,
                server_hostname=self.hostname,
                do_handshake_on_connect=False)
        except ssl.SSLError as e:
            log.exception('%s: Failed to wrap socket in SSLContext!', self)
            self.close(e)

    def _try_handshake(self):
        assert self.config['security_protocol'] in ('SSL', 'SASL_SSL')
        try:
            self._sock.do_handshake()
            return True
        # old ssl in python2.6 will swallow all SSLErrors here...
        except (SSLWantReadError, SSLWantWriteError):
            pass
        except (SSLZeroReturnError, ConnectionError, SSLEOFError):
            log.warning('SSL connection closed by server during handshake.')
            self.close(Errors.ConnectionError('SSL connection closed by server during handshake'))
        # Other SSLErrors will be raised to user

        return False

    def _try_authenticate(self):
        assert self.config['api_version'] is None or self.config['api_version'] >= (0, 10)

        if self._sasl_auth_future is None:
            # Build a SaslHandShakeRequest message
            request = SaslHandShakeRequest[0](self.config['sasl_mechanism'])
            future = Future()
            sasl_response = self._send(request)
            sasl_response.add_callback(self._handle_sasl_handshake_response, future)
            sasl_response.add_errback(lambda f, e: f.failure(e), future)
            self._sasl_auth_future = future
        self.recv()
        # A connection error could trigger close() which will reset the future
        if self._sasl_auth_future is None:
            return False
        elif self._sasl_auth_future.failed():
            ex = self._sasl_auth_future.exception
            if not isinstance(ex, Errors.ConnectionError):
                raise ex  # pylint: disable-msg=raising-bad-type
        return self._sasl_auth_future.succeeded()

    def _handle_sasl_handshake_response(self, future, response):
        error_type = Errors.for_code(response.error_code)
        if error_type is not Errors.NoError:
            error = error_type(self)
            self.close(error=error)
            return future.failure(error_type(self))

        if self.config['sasl_mechanism'] not in response.enabled_mechanisms:
            return future.failure(
                Errors.UnsupportedSaslMechanismError(
                    'Kafka broker does not support %s sasl mechanism. Enabled mechanisms are: %s'
                    % (self.config['sasl_mechanism'], response.enabled_mechanisms)))
        elif self.config['sasl_mechanism'] == 'PLAIN':
            return self._try_authenticate_plain(future)
        elif self.config['sasl_mechanism'] == 'GSSAPI':
            return self._try_authenticate_gssapi(future)
        else:
            return future.failure(
                Errors.UnsupportedSaslMechanismError(
                    'kafka-python does not support SASL mechanism %s' %
                    self.config['sasl_mechanism']))

    def _send_bytes_blocking(self, data):
        self._sock.setblocking(True)
        total_sent = 0
        try:
            while total_sent < len(data):
                sent_bytes = self._sock.send(data[total_sent:])
                total_sent += sent_bytes
            if total_sent != len(data):
                raise ConnectionError('Buffer overrun during socket send')
            return total_sent
        finally:
            self._sock.setblocking(False)

    def _recv_bytes_blocking(self, n):
        self._sock.setblocking(True)
        try:
            data = b''
            while len(data) < n:
                fragment = self._sock.recv(n - len(data))
                if not fragment:
                    raise ConnectionError('Connection reset during recv')
                data += fragment
            return data
        finally:
            self._sock.setblocking(False)

    def _try_authenticate_plain(self, future):
        if self.config['security_protocol'] == 'SASL_PLAINTEXT':
            log.warning('%s: Sending username and password in the clear', self)

        data = b''
        # Send PLAIN credentials per RFC-4616
        msg = bytes('\0'.join([self.config['sasl_plain_username'],
                               self.config['sasl_plain_username'],
                               self.config['sasl_plain_password']]).encode('utf-8'))
        size = Int32.encode(len(msg))
        try:
            self._send_bytes_blocking(size + msg)

            # The server will send a zero sized message (that is Int32(0)) on success.
            # The connection is closed on failure
            self._recv_bytes_blocking(4)

        except ConnectionError as e:
            log.exception("%s: Error receiving reply from server",  self)
            error = Errors.ConnectionError("%s: %s" % (self, e))
            self.close(error=error)
            return future.failure(error)

        if data != b'\x00\x00\x00\x00':
            error = Errors.AuthenticationFailedError('Unrecognized response during authentication')
            return future.failure(error)

        log.info('%s: Authenticated as %s via PLAIN', self, self.config['sasl_plain_username'])
        return future.success(True)

    def _try_authenticate_gssapi(self, future):
        gssapi_name = gssapi.Name(
            self.config['sasl_kerberos_service_name'] + '@' + self.hostname,
            name_type=gssapi.NameType.hostbased_service
        ).canonicalize(gssapi.MechType.kerberos)
        log.debug('%s: GSSAPI name: %s', self, gssapi_name)

        # Exchange tokens until authentication either succeeds or fails
        client_ctx = gssapi.SecurityContext(name=gssapi_name, usage='initiate')
        received_token = None
        try:
            while not client_ctx.complete:
                # calculate an output token from kafka token (or None if first iteration)
                output_token = client_ctx.step(received_token)

                # pass output token to kafka
                try:
                    msg = output_token
                    size = Int32.encode(len(msg))
                    self._send_bytes_blocking(size + msg)

                    # The server will send a token back. Processing of this token either
                    # establishes a security context, or it needs further token exchange.
                    # The gssapi will be able to identify the needed next step.
                    # The connection is closed on failure.
                    header = self._recv_bytes_blocking(4)
                    token_size = struct.unpack('>i', header)
                    received_token = self._recv_bytes_blocking(token_size)

                except ConnectionError as e:
                    log.exception("%s: Error receiving reply from server",  self)
                    error = Errors.ConnectionError("%s: %s" % (self, e))
                    self.close(error=error)
                    return future.failure(error)

        except Exception as e:
            return future.failure(e)

        log.info('%s: Authenticated as %s via GSSAPI', self, gssapi_name)
        return future.success(True)

    def blacked_out(self):
        """
        Return true if we are disconnected from the given node and can't
        re-establish a connection yet
        """
        if self.state is ConnectionStates.DISCONNECTED:
            if time.time() < self.last_attempt + self._reconnect_backoff:
                return True
        return False

    def connection_delay(self):
        time_waited_ms = time.time() - (self.last_attempt or 0)
        if self.state is ConnectionStates.DISCONNECTED:
            return max(self._reconnect_backoff - time_waited_ms, 0)
        elif self.connecting():
            return 0
        else:
            return 999999999

    def connected(self):
        """Return True iff socket is connected."""
        return self.state is ConnectionStates.CONNECTED

    def connecting(self):
        """Returns True if still connecting (this may encompass several
        different states, such as SSL handshake, authorization, etc)."""
        return self.state in (ConnectionStates.CONNECTING,
                              ConnectionStates.HANDSHAKE,
                              ConnectionStates.AUTHENTICATING)

    def disconnected(self):
        """Return True iff socket is closed"""
        return self.state is ConnectionStates.DISCONNECTED

    def _reset_reconnect_backoff(self):
        self._failures = 0
        self._reconnect_backoff = self.config['reconnect_backoff_ms'] / 1000.0

    def _update_reconnect_backoff(self):
        if self.config['reconnect_backoff_max_ms'] > self.config['reconnect_backoff_ms']:
            self._failures += 1
            self._reconnect_backoff = self.config['reconnect_backoff_ms'] * 2 ** (self._failures - 1)
            self._reconnect_backoff = min(self._reconnect_backoff, self.config['reconnect_backoff_max_ms'])
            self._reconnect_backoff *= uniform(0.8, 1.2)
            self._reconnect_backoff /= 1000.0
            log.debug('%s: reconnect backoff %s after %s failures', self, self._reconnect_backoff, self._failures)

    def close(self, error=None):
        """Close socket and fail all in-flight-requests.

        Arguments:
            error (Exception, optional): pending in-flight-requests
                will be failed with this exception.
                Default: kafka.errors.ConnectionError.
        """
        if self.state is ConnectionStates.DISCONNECTED:
            if error is not None:
                if sys.version_info >= (3, 2):
                    log.warning('%s: close() called on disconnected connection with error: %s', self, error, stack_info=True)
                else:
                    log.warning('%s: close() called on disconnected connection with error: %s', self, error)
            return

        log.info('%s: Closing connection. %s', self, error or '')
        self.state = ConnectionStates.DISCONNECTING
        self.config['state_change_callback'](self)
        self._update_reconnect_backoff()
        if self._sock:
            self._sock.close()
            self._sock = None
        self.state = ConnectionStates.DISCONNECTED
        self.last_attempt = time.time()
        self._sasl_auth_future = None
        self._protocol = KafkaProtocol(
            client_id=self.config['client_id'],
            api_version=self.config['api_version'])
        if error is None:
            error = Errors.Cancelled(str(self))
        while self.in_flight_requests:
            (_, future, _) = self.in_flight_requests.popleft()
            future.failure(error)
        self.config['state_change_callback'](self)

    def send(self, request):
        """send request, return Future()

        Can block on network if request is larger than send_buffer_bytes
        """
        future = Future()
        if self.connecting():
            return future.failure(Errors.NodeNotReadyError(str(self)))
        elif not self.connected():
            return future.failure(Errors.ConnectionError(str(self)))
        elif not self.can_send_more():
            return future.failure(Errors.TooManyInFlightRequests(str(self)))
        return self._send(request)

    def _send(self, request):
        assert self.state in (ConnectionStates.AUTHENTICATING, ConnectionStates.CONNECTED)
        future = Future()
        correlation_id = self._protocol.send_request(request)
        data = self._protocol.send_bytes()
        try:
            # In the future we might manage an internal write buffer
            # and send bytes asynchronously. For now, just block
            # sending each request payload
            total_bytes = self._send_bytes_blocking(data)
            if self._sensors:
                self._sensors.bytes_sent.record(total_bytes)
        except ConnectionError as e:
            log.exception("Error sending %s to %s", request, self)
            error = Errors.ConnectionError("%s: %s" % (self, e))
            self.close(error=error)
            return future.failure(error)
        log.debug('%s Request %d: %s', self, correlation_id, request)

        if request.expect_response():
            ifr = (correlation_id, future, time.time())
            self.in_flight_requests.append(ifr)
        else:
            future.success(None)

        return future

    def can_send_more(self):
        """Return True unless there are max_in_flight_requests_per_connection."""
        max_ifrs = self.config['max_in_flight_requests_per_connection']
        return len(self.in_flight_requests) < max_ifrs

    def recv(self):
        """Non-blocking network receive.

        Return list of (response, future)
        """
        if not self.connected() and not self.state is ConnectionStates.AUTHENTICATING:
            log.warning('%s cannot recv: socket not connected', self)
            # If requests are pending, we should close the socket and
            # fail all the pending request futures
            if self.in_flight_requests:
                self.close(Errors.ConnectionError('Socket not connected during recv with in-flight-requests'))
            return ()

        elif not self.in_flight_requests:
            log.warning('%s: No in-flight-requests to recv', self)
            return ()

        responses = self._recv()
        if not responses and self.requests_timed_out():
            log.warning('%s timed out after %s ms. Closing connection.',
                        self, self.config['request_timeout_ms'])
            self.close(error=Errors.RequestTimedOutError(
                'Request timed out after %s ms' %
                self.config['request_timeout_ms']))
            return ()

        # augment respones w/ correlation_id, future, and timestamp
        for i in range(len(responses)):
            (correlation_id, future, timestamp) = self.in_flight_requests.popleft()
            latency_ms = (time.time() - timestamp) * 1000
            if self._sensors:
                self._sensors.request_time.record(latency_ms)

            response = responses[i]
            log.debug('%s Response %d (%s ms): %s', self, correlation_id, latency_ms, response)
            responses[i] = (response, future)

        return responses

    def _recv(self):
        responses = []
        SOCK_CHUNK_BYTES = 4096
        while True:
            try:
                data = self._sock.recv(SOCK_CHUNK_BYTES)
                # We expect socket.recv to raise an exception if there is not
                # enough data to read the full bytes_to_read
                # but if the socket is disconnected, we will get empty data
                # without an exception raised
                if not data:
                    log.error('%s: socket disconnected', self)
                    self.close(error=Errors.ConnectionError('socket disconnected'))
                    break

            except SSLWantReadError:
                break
            except ConnectionError as e:
                if six.PY2 and e.errno == errno.EWOULDBLOCK:
                    break
                log.exception('%s: Error receiving network data'
                              ' closing socket', self)
                self.close(error=Errors.ConnectionError(e))
                break
            except BlockingIOError:
                if six.PY3:
                    break
                raise

            if self._sensors:
                self._sensors.bytes_received.record(len(data))

            try:
                more_responses = self._protocol.receive_bytes(data)
            except Errors.KafkaProtocolError as e:
                self.close(e)
                break
            else:
                responses.extend([resp for (_, resp) in more_responses])

            if len(data) < SOCK_CHUNK_BYTES:
                break

        return responses

    def requests_timed_out(self):
        if self.in_flight_requests:
            (_, _, oldest_at) = self.in_flight_requests[0]
            timeout = self.config['request_timeout_ms'] / 1000.0
            if time.time() >= oldest_at + timeout:
                return True
        return False

    def _next_correlation_id(self):
        self._correlation_id = (self._correlation_id + 1) % 2**31
        return self._correlation_id

    def _handle_api_version_response(self, response):
        error_type = Errors.for_code(response.error_code)
        assert error_type is Errors.NoError, "API version check failed"
        self._api_versions = dict([
            (api_key, (min_version, max_version))
            for api_key, min_version, max_version in response.api_versions
        ])
        return self._api_versions

    def _infer_broker_version_from_api_versions(self, api_versions):
        # The logic here is to check the list of supported request versions
        # in reverse order. As soon as we find one that works, return it
        test_cases = [
            # format (<broker version>, <needed struct>)
            ((0, 11, 0), MetadataRequest[4]),
            ((0, 10, 2), OffsetFetchRequest[2]),
            ((0, 10, 1), MetadataRequest[2]),
        ]

        # Get the best match of test cases
        for broker_version, struct in sorted(test_cases, reverse=True):
            if struct.API_KEY not in api_versions:
                continue
            min_version, max_version = api_versions[struct.API_KEY]
            if min_version <= struct.API_VERSION <= max_version:
                return broker_version

        # We know that ApiVersionResponse is only supported in 0.10+
        # so if all else fails, choose that
        return (0, 10, 0)

    def check_version(self, timeout=2, strict=False):
        """Attempt to guess the broker version.

        Note: This is a blocking call.

        Returns: version tuple, i.e. (0, 10), (0, 9), (0, 8, 2), ...
        """
        # Monkeypatch some connection configurations to avoid timeouts
        override_config = {
            'request_timeout_ms': timeout * 1000,
            'max_in_flight_requests_per_connection': 5
        }
        stashed = {}
        for key in override_config:
            stashed[key] = self.config[key]
            self.config[key] = override_config[key]

        # kafka kills the connection when it doesn't recognize an API request
        # so we can send a test request and then follow immediately with a
        # vanilla MetadataRequest. If the server did not recognize the first
        # request, both will be failed with a ConnectionError that wraps
        # socket.error (32, 54, or 104)
        from .protocol.admin import ApiVersionRequest, ListGroupsRequest
        from .protocol.commit import OffsetFetchRequest, GroupCoordinatorRequest

        # Socket errors are logged as exceptions and can alarm users. Mute them
        from logging import Filter

        class ConnFilter(Filter):
            def filter(self, record):
                if record.funcName == 'check_version':
                    return True
                return False
        log_filter = ConnFilter()
        log.addFilter(log_filter)

        test_cases = [
            # All cases starting from 0.10 will be based on ApiVersionResponse
            ((0, 10), ApiVersionRequest[0]()),
            ((0, 9), ListGroupsRequest[0]()),
            ((0, 8, 2), GroupCoordinatorRequest[0]('kafka-python-default-group')),
            ((0, 8, 1), OffsetFetchRequest[0]('kafka-python-default-group', [])),
            ((0, 8, 0), MetadataRequest[0]([])),
        ]

        def connect():
            self.connect()
            if self.connected():
                return
            timeout_at = time.time() + timeout
            while time.time() < timeout_at and self.connecting():
                if self.connect() is ConnectionStates.CONNECTED:
                    return
                time.sleep(0.05)
            raise Errors.NodeNotReadyError()

        for version, request in test_cases:
            connect()
            f = self.send(request)
            # HACK: sleeping to wait for socket to send bytes
            time.sleep(0.1)
            # when broker receives an unrecognized request API
            # it abruptly closes our socket.
            # so we attempt to send a second request immediately
            # that we believe it will definitely recognize (metadata)
            # the attempt to write to a disconnected socket should
            # immediately fail and allow us to infer that the prior
            # request was unrecognized
            mr = self.send(MetadataRequest[0]([]))

            selector = self.config['selector']()
            selector.register(self._sock, selectors.EVENT_READ)
            while not (f.is_done and mr.is_done):
                for response, future in self.recv():
                    future.success(response)
                selector.select(1)

            if f.succeeded():
                if isinstance(request, ApiVersionRequest[0]):
                    # Starting from 0.10 kafka broker we determine version
                    # by looking at ApiVersionResponse
                    api_versions = self._handle_api_version_response(f.value)
                    version = self._infer_broker_version_from_api_versions(api_versions)
                log.info('Broker version identifed as %s', '.'.join(map(str, version)))
                log.info('Set configuration api_version=%s to skip auto'
                         ' check_version requests on startup', version)
                break

            # Only enable strict checking to verify that we understand failure
            # modes. For most users, the fact that the request failed should be
            # enough to rule out a particular broker version.
            if strict:
                # If the socket flush hack did not work (which should force the
                # connection to close and fail all pending requests), then we
                # get a basic Request Timeout. This is not ideal, but we'll deal
                if isinstance(f.exception, Errors.RequestTimedOutError):
                    pass

                # 0.9 brokers do not close the socket on unrecognized api
                # requests (bug...). In this case we expect to see a correlation
                # id mismatch
                elif (isinstance(f.exception, Errors.CorrelationIdError) and
                      version == (0, 10)):
                    pass
                elif six.PY2:
                    assert isinstance(f.exception.args[0], socket.error)
                    assert f.exception.args[0].errno in (32, 54, 104)
                else:
                    assert isinstance(f.exception.args[0], ConnectionError)
            log.info("Broker is not v%s -- it did not recognize %s",
                     version, request.__class__.__name__)
        else:
            raise Errors.UnrecognizedBrokerVersion()

        log.removeFilter(log_filter)
        for key in stashed:
            self.config[key] = stashed[key]
        return version

    def __repr__(self):
        return "<BrokerConnection node_id=%s host=%s/%s port=%d>" % (
            self.node_id, self.hostname, self.host, self.port)


class BrokerConnectionMetrics(object):
    def __init__(self, metrics, metric_group_prefix, node_id):
        self.metrics = metrics

        # Any broker may have registered summary metrics already
        # but if not, we need to create them so we can set as parents below
        all_conns_transferred = metrics.get_sensor('bytes-sent-received')
        if not all_conns_transferred:
            metric_group_name = metric_group_prefix + '-metrics'

            bytes_transferred = metrics.sensor('bytes-sent-received')
            bytes_transferred.add(metrics.metric_name(
                'network-io-rate', metric_group_name,
                'The average number of network operations (reads or writes) on all'
                ' connections per second.'), Rate(sampled_stat=Count()))

            bytes_sent = metrics.sensor('bytes-sent',
                                        parents=[bytes_transferred])
            bytes_sent.add(metrics.metric_name(
                'outgoing-byte-rate', metric_group_name,
                'The average number of outgoing bytes sent per second to all'
                ' servers.'), Rate())
            bytes_sent.add(metrics.metric_name(
                'request-rate', metric_group_name,
                'The average number of requests sent per second.'),
                Rate(sampled_stat=Count()))
            bytes_sent.add(metrics.metric_name(
                'request-size-avg', metric_group_name,
                'The average size of all requests in the window.'), Avg())
            bytes_sent.add(metrics.metric_name(
                'request-size-max', metric_group_name,
                'The maximum size of any request sent in the window.'), Max())

            bytes_received = metrics.sensor('bytes-received',
                                            parents=[bytes_transferred])
            bytes_received.add(metrics.metric_name(
                'incoming-byte-rate', metric_group_name,
                'Bytes/second read off all sockets'), Rate())
            bytes_received.add(metrics.metric_name(
                'response-rate', metric_group_name,
                'Responses received sent per second.'),
                Rate(sampled_stat=Count()))

            request_latency = metrics.sensor('request-latency')
            request_latency.add(metrics.metric_name(
                'request-latency-avg', metric_group_name,
                'The average request latency in ms.'),
                Avg())
            request_latency.add(metrics.metric_name(
                'request-latency-max', metric_group_name,
                'The maximum request latency in ms.'),
                Max())

        # if one sensor of the metrics has been registered for the connection,
        # then all other sensors should have been registered; and vice versa
        node_str = 'node-{0}'.format(node_id)
        node_sensor = metrics.get_sensor(node_str + '.bytes-sent')
        if not node_sensor:
            metric_group_name = metric_group_prefix + '-node-metrics.' + node_str

            bytes_sent = metrics.sensor(
                node_str + '.bytes-sent',
                parents=[metrics.get_sensor('bytes-sent')])
            bytes_sent.add(metrics.metric_name(
                'outgoing-byte-rate', metric_group_name,
                'The average number of outgoing bytes sent per second.'),
                Rate())
            bytes_sent.add(metrics.metric_name(
                'request-rate', metric_group_name,
                'The average number of requests sent per second.'),
                Rate(sampled_stat=Count()))
            bytes_sent.add(metrics.metric_name(
                'request-size-avg', metric_group_name,
                'The average size of all requests in the window.'),
                Avg())
            bytes_sent.add(metrics.metric_name(
                'request-size-max', metric_group_name,
                'The maximum size of any request sent in the window.'),
                Max())

            bytes_received = metrics.sensor(
                node_str + '.bytes-received',
                parents=[metrics.get_sensor('bytes-received')])
            bytes_received.add(metrics.metric_name(
                'incoming-byte-rate', metric_group_name,
                'Bytes/second read off node-connection socket'),
                Rate())
            bytes_received.add(metrics.metric_name(
                'response-rate', metric_group_name,
                'The average number of responses received per second.'),
                Rate(sampled_stat=Count()))

            request_time = metrics.sensor(
                node_str + '.latency',
                parents=[metrics.get_sensor('request-latency')])
            request_time.add(metrics.metric_name(
                'request-latency-avg', metric_group_name,
                'The average request latency in ms.'),
                Avg())
            request_time.add(metrics.metric_name(
                'request-latency-max', metric_group_name,
                'The maximum request latency in ms.'),
                Max())

        self.bytes_sent = metrics.sensor(node_str + '.bytes-sent')
        self.bytes_received = metrics.sensor(node_str + '.bytes-received')
        self.request_time = metrics.sensor(node_str + '.latency')


def _address_family(address):
    """
        Attempt to determine the family of an address (or hostname)

        :return: either socket.AF_INET or socket.AF_INET6 or socket.AF_UNSPEC if the address family
                 could not be determined
    """
    if address.startswith('[') and address.endswith(']'):
        return socket.AF_INET6
    for af in (socket.AF_INET, socket.AF_INET6):
        try:
            socket.inet_pton(af, address)
            return af
        except (ValueError, AttributeError, socket.error):
            continue
    return socket.AF_UNSPEC


def get_ip_port_afi(host_and_port_str):
    """
        Parse the IP and port from a string in the format of:

            * host_or_ip          <- Can be either IPv4 address literal or hostname/fqdn
            * host_or_ipv4:port   <- Can be either IPv4 address literal or hostname/fqdn
            * [host_or_ip]        <- IPv6 address literal
            * [host_or_ip]:port.  <- IPv6 address literal

        .. note:: IPv6 address literals with ports *must* be enclosed in brackets

        .. note:: If the port is not specified, default will be returned.

        :return: tuple (host, port, afi), afi will be socket.AF_INET or socket.AF_INET6 or socket.AF_UNSPEC
    """
    host_and_port_str = host_and_port_str.strip()
    if host_and_port_str.startswith('['):
        af = socket.AF_INET6
        host, rest = host_and_port_str[1:].split(']')
        if rest:
            port = int(rest[1:])
        else:
            port = DEFAULT_KAFKA_PORT
        return host, port, af
    else:
        if ':' not in host_and_port_str:
            af = _address_family(host_and_port_str)
            return host_and_port_str, DEFAULT_KAFKA_PORT, af
        else:
            # now we have something with a colon in it and no square brackets. It could be
            # either an IPv6 address literal (e.g., "::1") or an IP:port pair or a host:port pair
            try:
                # if it decodes as an IPv6 address, use that
                socket.inet_pton(socket.AF_INET6, host_and_port_str)
                return host_and_port_str, DEFAULT_KAFKA_PORT, socket.AF_INET6
            except AttributeError:
                log.warning('socket.inet_pton not available on this platform.'
                            ' consider `pip install win_inet_pton`')
                pass
            except (ValueError, socket.error):
                # it's a host:port pair
                pass
            host, port = host_and_port_str.rsplit(':', 1)
            port = int(port)

            af = _address_family(host)
            return host, port, af


def collect_hosts(hosts, randomize=True):
    """
    Collects a comma-separated set of hosts (host:port) and optionally
    randomize the returned list.
    """

    if isinstance(hosts, six.string_types):
        hosts = hosts.strip().split(',')

    result = []
    afi = socket.AF_INET
    for host_port in hosts:

        host, port, afi = get_ip_port_afi(host_port)

        if port < 0:
            port = DEFAULT_KAFKA_PORT

        result.append((host, port, afi))

    if randomize:
        shuffle(result)

    return result