summaryrefslogtreecommitdiff
path: root/kafka/client_async.py
blob: 5c11fc5001a60619846b26f8e96dd43884a30962 (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
import heapq
import itertools
import logging
import random
import select
import sys
import time

import six

import kafka.common as Errors # TODO: make Errors a separate class

from .cluster import ClusterMetadata
from .conn import BrokerConnection, ConnectionStates, collect_hosts
from .future import Future
from .protocol.metadata import MetadataRequest
from .protocol.produce import ProduceRequest

log = logging.getLogger(__name__)


class KafkaClient(object):
    """
    A network client for asynchronous request/response network i/o.
    This is an internal class used to implement the
    user-facing producer and consumer clients.

    This class is not thread-safe!
    """
    _bootstrap_servers = 'localhost'
    _client_id = 'kafka-python-0.10.0'
    _reconnect_backoff_ms = 50
    _retry_backoff_ms = 100
    _send_buffer_bytes = 131072
    _receive_buffer_bytes = 32768
    _request_timeout_ms = 40000
    _max_in_flight_requests_per_connection=5

    def __init__(self, **kwargs):
        for config in (
            'client_id', 'max_in_flight_requests_per_connection',
            'reconnect_backoff_ms', 'retry_backoff_ms',
            'send_buffer_bytes', 'receive_buffer_bytes',
            'request_timeout_ms', 'bootstrap_servers'
        ):
            if config in kwargs:
                setattr(self, '_' + config, kwargs.pop(config))

        self.cluster = ClusterMetadata(**kwargs)
        self._topics = set() # empty set will fetch all topic metadata
        self._metadata_refresh_in_progress = False
        self._conns = {}
        self._connecting = set()
        self._delayed_tasks = DelayedTaskQueue()
        self._last_bootstrap = 0
        self._bootstrap_fails = 0
        self._bootstrap(collect_hosts(self._bootstrap_servers))

    def _bootstrap(self, hosts):
        # Exponential backoff if bootstrap fails
        backoff_ms = self._reconnect_backoff_ms * 2 ** self._bootstrap_fails
        next_at = self._last_bootstrap + backoff_ms / 1000.0
        now = time.time()
        if next_at > now:
            log.debug("Sleeping %0.4f before bootstrapping again", next_at - now)
            time.sleep(next_at - now)
        self._last_bootstrap = time.time()

        metadata_request = MetadataRequest([])
        for host, port in hosts:
            log.debug("Attempting to bootstrap via node at %s:%s", host, port)
            bootstrap = BrokerConnection(
                host, port,
                client_id=self._client_id,
                receive_buffer_bytes=self._receive_buffer_bytes,
                send_buffer_bytes=self._send_buffer_bytes,
                request_timeout_ms=self._request_timeout_ms,
                max_in_flight_requests_per_connection=self._max_in_flight_requests_per_connection,
                reconnect_backoff_ms=self._reconnect_backoff_ms
            )
            bootstrap.connect()
            while bootstrap.state is ConnectionStates.CONNECTING:
                bootstrap.connect()
            if bootstrap.state is not ConnectionStates.CONNECTED:
                bootstrap.close()
                continue
            future = bootstrap.send(metadata_request)
            while not future.is_done:
                bootstrap.recv()
            if future.failed():
                bootstrap.close()
                continue
            self.cluster.update_metadata(future.value)

            # A cluster with no topics can return no broker metadata
            # in that case, we should keep the bootstrap connection
            if not len(self.cluster.brokers()):
                self._conns['bootstrap'] = bootstrap
            self._bootstrap_fails = 0
            break
        # No bootstrap found...
        else:
            log.error('Unable to bootstrap from %s', hosts)
            # Max exponential backoff is 2^12, x4000 (50ms -> 200s)
            self._bootstrap_fails = min(self._bootstrap_fails + 1, 12)

    def _can_connect(self, node_id):
        if node_id not in self._conns:
            if self.cluster.broker_metadata(node_id):
                return True
            return False
        conn = self._conns[node_id]
        return conn.state is ConnectionStates.DISCONNECTED and not conn.blacked_out()

    def _initiate_connect(self, node_id):
        """Initiate a connection to the given node"""
        broker = self.cluster.broker_metadata(node_id)
        if not broker:
            raise Errors.IllegalArgumentError('Broker %s not found in current cluster metadata', node_id)

        if node_id not in self._conns:
            log.debug("Initiating connection to node %s at %s:%s",
                      node_id, broker.host, broker.port)
            self._conns[node_id] = BrokerConnection(
                broker.host, broker.port,
                client_id=self._client_id,
                receive_buffer_bytes=self._receive_buffer_bytes,
                send_buffer_bytes=self._send_buffer_bytes,
                request_timeout_ms=self._request_timeout_ms,
                max_in_flight_requests_per_connection=self._max_in_flight_requests_per_connection,
                reconnect_backoff_ms=self._reconnect_backoff_ms
            )
        return self._finish_connect(node_id)

    def _finish_connect(self, node_id):
        if node_id not in self._conns:
            raise Errors.IllegalArgumentError('Node %s not found in connections', node_id)
        state = self._conns[node_id].connect()
        if state is ConnectionStates.CONNECTING:
            self._connecting.add(node_id)
        elif node_id in self._connecting:
            log.debug("Node %s connection state is %s", node_id, state)
            self._connecting.remove(node_id)
        return state

    def ready(self, node_id):
        """
        Begin connecting to the given node, return true if we are already
        connected and ready to send to that node.

        @param node_id The id of the node to check
        @return True if we are ready to send to the given node
        """
        if self.is_ready(node_id):
            return True

        if self._can_connect(node_id):
            # if we are interested in sending to a node
            # and we don't have a connection to it, initiate one
            self._initiate_connect(node_id)

        if node_id in self._connecting:
            self._finish_connect(node_id)

        return self.is_ready(node_id)

    def close(self, node_id=None):
        """Closes the connection to a particular node (if there is one).

        @param node_id The id of the node
        """
        if node_id is None:
            for conn in self._conns.values():
                conn.close()
        elif node_id in self._conns:
            self._conns[node_id].close()
        else:
            log.warning("Node %s not found in current connection list; skipping", node_id)
            return

    def connection_delay(self, node_id):
        """
        Returns the number of milliseconds to wait, based on the connection
        state, before attempting to send data. When disconnected, this respects
        the reconnect backoff time. When connecting or connected, this handles
        slow/stalled connections.

        @param node_id The id of the node to check
        @return The number of milliseconds to wait.
        """
        if node_id not in self._conns:
            return 0

        conn = self._conns[node_id]
        time_waited_ms = time.time() - (conn.last_attempt or 0)
        if conn.state is ConnectionStates.DISCONNECTED:
            return max(self._reconnect_backoff_ms - time_waited_ms, 0)
        else:
            return sys.maxint

    def connection_failed(self, node_id):
        """
        Check if the connection of the node has failed, based on the connection
        state. Such connection failures are usually transient and can be resumed
        in the next ready(node) call, but there are cases where transient
        failures need to be caught and re-acted upon.

        @param node_id the id of the node to check
        @return true iff the connection has failed and the node is disconnected
        """
        if node_id not in self._conns:
            return False
        return self._conns[node_id].state is ConnectionStates.DISCONNECTED

    def is_ready(self, node_id):
        """
        Check if the node with the given id is ready to send more requests.

        @param node_id The id of the node
        @return true if the node is ready
        """
        # if we need to update our metadata now declare all requests unready to
        # make metadata requests first priority
        if not self._metadata_refresh_in_progress and not self.cluster.ttl() == 0:
            if self._can_send_request(node_id):
                return True
        return False

    def _can_send_request(self, node_id):
        if node_id not in self._conns:
            return False
        conn = self._conns[node_id]
        return conn.connected() and conn.can_send_more()

    def send(self, node_id, request):
        """
        Send the given request. Requests can only be sent out to ready nodes.

        @param node destination node
        @param request The request
        @param now The current timestamp
        """
        if not self._can_send_request(node_id):
            raise Errors.IllegalStateError("Attempt to send a request to node %s which is not ready." % node_id)

        # Every request gets a response, except one special case:
        expect_response = True
        if isinstance(request, ProduceRequest) and request.required_acks == 0:
            expect_response = False

        return self._conns[node_id].send(request, expect_response=expect_response)

    def poll(self, timeout_ms=None, future=None):
        """Do actual reads and writes to sockets.

        @param timeout_ms The maximum amount of time to wait (in ms) for
                          responses if there are none available immediately.
                          Must be non-negative. The actual timeout will be the
                          minimum of timeout, request timeout and metadata
                          timeout. If unspecified, default to request_timeout_ms
        @param future Optionally block until the provided future completes.
        @return The list of responses received.
        """
        if timeout_ms is None:
            timeout_ms = self._request_timeout_ms

        responses = []

        # Loop for futures, break after first loop if None
        while True:

            # Attempt to complete pending connections
            for node_id in list(self._connecting):
                self._finish_connect(node_id)

            # Send a metadata request if needed
            metadata_timeout = self._maybe_refresh_metadata()

            # Send scheduled tasks
            for task in self._delayed_tasks.pop_ready():
                try:
                    task()
                except Exception as e:
                    log.error("Task %s failed: %s", task, e)

            timeout = min(timeout_ms, metadata_timeout, self._request_timeout_ms)
            timeout /= 1000.0

            responses.extend(self._poll(timeout))
            if not future or future.is_done:
                break

        return responses

    def _poll(self, timeout):
        # select on reads across all connected sockets, blocking up to timeout
        sockets = [conn._sock for conn in six.itervalues(self._conns)
                   if (conn.state is ConnectionStates.CONNECTED and
                       conn.in_flight_requests)]
        if sockets:
            select.select(sockets, [], [], timeout)

        responses = []
        # list, not iterator, because inline callbacks may add to self._conns
        for conn in list(self._conns.values()):
            if conn.state is ConnectionStates.CONNECTING:
                conn.connect()

            if conn.in_flight_requests:
                response = conn.recv() # This will run callbacks / errbacks
                if response:
                    responses.append(response)
        return responses

    def in_flight_request_count(self, node_id=None):
        """Get the number of in-flight requests"""
        if node_id is not None:
            if node_id not in self._conns:
                return 0
            return len(self._conns[node_id].in_flight_requests)
        else:
            return sum([len(conn.in_flight_requests) for conn in self._conns.values()])

    def least_loaded_node(self):
        """
        Choose the node with the fewest outstanding requests which is at least
        eligible for connection. This method will prefer a node with an
        existing connection, but will potentially choose a node for which we
        don't yet have a connection if all existing connections are in use.
        This method will never choose a node for which there is no existing
        connection and from which we have disconnected within the reconnect
        backoff period.

        @return The node_id with the fewest in-flight requests.
        """
        nodes = list(self._conns.keys())
        random.shuffle(nodes)
        inflight = sys.maxint
        found = None
        for node_id in nodes:
            conn = self._conns[node_id]
            curr_inflight = len(conn.in_flight_requests)
            if curr_inflight == 0 and conn.connected():
                # if we find an established connection with no in-flight requests we can stop right away
                return node_id
            elif not conn.blacked_out() and curr_inflight < inflight:
                # otherwise if this is the best we have found so far, record that
                inflight = curr_inflight
                found = node_id

        if found is not None:
            return found

        # if we found no connected node, return a disconnected one
        log.debug("No connected nodes found. Trying disconnected nodes.")
        for node_id in nodes:
            if not self._conns[node_id].is_blacked_out():
                return node_id

        # if still no luck, look for a node not in self._conns yet
        log.debug("No luck. Trying all broker metadata")
        for broker in self.cluster.brokers():
            if broker.nodeId not in self._conns:
                return broker.nodeId

        # Last option: try to bootstrap again
        log.error('No nodes found in metadata -- retrying bootstrap')
        self._bootstrap(collect_hosts(self._bootstrap_servers))
        return None

    def set_topics(self, topics):
        """
        Set specific topics to track for metadata

        Returns a future that will complete after metadata request/response
        """
        if set(topics).difference(self._topics):
            future = self.cluster.request_update()
        else:
            future = Future().success(set(topics))
        self._topics = set(topics)
        return future

    # request metadata update on disconnect and timedout
    def _maybe_refresh_metadata(self):
        """Send a metadata request if needed"""
        ttl = self.cluster.ttl()
        if ttl > 0:
            return ttl

        if self._metadata_refresh_in_progress:
            return sys.maxint

        node_id = self.least_loaded_node()

        if self._can_send_request(node_id):
            request = MetadataRequest(list(self._topics))
            log.debug("Sending metadata request %s to node %s", request, node_id)
            future = self.send(node_id, request)
            future.add_callback(self.cluster.update_metadata)
            future.add_errback(self.cluster.failed_update)

            self._metadata_refresh_in_progress = True
            def refresh_done(val_or_error):
                self._metadata_refresh_in_progress = False
            future.add_callback(refresh_done)
            future.add_errback(refresh_done)

        elif self._can_connect(node_id):
            log.debug("Initializing connection to node %s for metadata request", node_id)
            self._initiate_connect(node_id)

        return 0

    def schedule(self, task, at):
        """
        Schedule a new task to be executed at the given time.

        This is "best-effort" scheduling and should only be used for coarse
        synchronization. A task cannot be scheduled for multiple times
        simultaneously; any previously scheduled instance of the same task
        will be cancelled.

        @param task The task to be scheduled -- function or implement __call__
        @param at Epoch seconds when it should run (see time.time())
        @returns Future
        """
        return self._delayed_tasks.add(task, at)

    def unschedule(self, task):
        """
        Unschedule a task. This will remove all instances of the task from the task queue.
        This is a no-op if the task is not scheduled.

        @param task The task to be unscheduled.
        """
        self._delayed_tasks.remove(task)


class DelayedTaskQueue(object):
    # see https://docs.python.org/2/library/heapq.html
    def __init__(self):
        self._tasks = [] # list of entries arranged in a heap
        self._task_map = {} # mapping of tasks to entries
        self._counter = itertools.count() # unique sequence count

    def add(self, task, at):
        """Add a task to run at a later time

        task: anything
        at: seconds from epoch to schedule task (see time.time())
        """
        if task in self._task_map:
            self.remove(task)
        count = next(self._counter)
        future = Future()
        entry = [at, count, (task, future)]
        self._task_map[task] = entry
        heapq.heappush(self._tasks, entry)
        return future

    def remove(self, task):
        """Remove a previously scheduled task

        Raises KeyError if task is not found
        """
        entry = self._task_map.pop(task)
        task, future = entry[-1]
        future.failure(Errors.Cancelled)
        entry[-1] = 'REMOVED'

    def _drop_removed(self):
        while self._tasks and self._tasks[0][-1] is 'REMOVED':
            at, count, task = heapq.heappop(self._tasks)

    def _pop_next(self):
        self._drop_removed()
        if not self._tasks:
            raise KeyError('pop from an empty DelayedTaskQueue')
        _, _, maybe_task = heapq.heappop(self._tasks)
        if maybe_task is 'REMOVED':
            raise ValueError('popped a removed tasks from queue - bug')
        else:
            task, future = maybe_task
        del self._task_map[task]
        return task

    def next_at(self):
        """Number of seconds until next task is ready"""
        self._drop_removed()
        if not self._tasks:
            return sys.maxint
        else:
            return max(self._tasks[0][0] - time.time(), 0)

    def pop_ready(self):
        """Pop and return a list of all ready (task, future) tuples"""
        self._drop_removed()
        ready_tasks = []
        while self._tasks and self._tasks[0][0] < time.time():
            ready_tasks.append(self._pop_next())
        return ready_tasks