summaryrefslogtreecommitdiff
path: root/kafka/producer/base.py
blob: c9dd6c3a114070000ad465c5b3f80655ff6908cb (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
from __future__ import absolute_import

import atexit
import logging
import time

try:
    from queue import Empty, Full, Queue  # pylint: disable=import-error
except ImportError:
    from Queue import Empty, Full, Queue  # pylint: disable=import-error
from collections import defaultdict

from threading import Thread, Event

from kafka.vendor import six

from kafka.structs import (
    ProduceRequestPayload, ProduceResponsePayload, TopicPartition, RetryOptions)
from kafka.errors import (
    kafka_errors, UnsupportedCodecError, FailedPayloadsError,
    RequestTimedOutError, AsyncProducerQueueFull, UnknownError,
    RETRY_ERROR_TYPES, RETRY_BACKOFF_ERROR_TYPES, RETRY_REFRESH_ERROR_TYPES)
from kafka.protocol import CODEC_NONE, ALL_CODECS, create_message_set

log = logging.getLogger('kafka.producer')

BATCH_SEND_DEFAULT_INTERVAL = 20
BATCH_SEND_MSG_COUNT = 20

# unlimited
ASYNC_QUEUE_MAXSIZE = 0
ASYNC_QUEUE_PUT_TIMEOUT = 0
# unlimited retries by default
ASYNC_RETRY_LIMIT = None
ASYNC_RETRY_BACKOFF_MS = 100
ASYNC_RETRY_ON_TIMEOUTS = True
ASYNC_LOG_MESSAGES_ON_ERROR = True

STOP_ASYNC_PRODUCER = -1
ASYNC_STOP_TIMEOUT_SECS = 30

SYNC_FAIL_ON_ERROR_DEFAULT = True


def _send_upstream(queue, client, codec, batch_time, batch_size,
                   req_acks, ack_timeout, retry_options, stop_event,
                   log_messages_on_error=ASYNC_LOG_MESSAGES_ON_ERROR,
                   stop_timeout=ASYNC_STOP_TIMEOUT_SECS,
                   codec_compresslevel=None):
    """Private method to manage producing messages asynchronously

    Listens on the queue for a specified number of messages or until
    a specified timeout and then sends messages to the brokers in grouped
    requests (one per broker).

    Messages placed on the queue should be tuples that conform to this format:
        ((topic, partition), message, key)

    Currently does not mark messages with task_done. Do not attempt to
    :meth:`join`!

    Arguments:
        queue (threading.Queue): the queue from which to get messages
        client (kafka.SimpleClient): instance to use for communicating
            with brokers
        codec (kafka.protocol.ALL_CODECS): compression codec to use
        batch_time (int): interval in seconds to send message batches
        batch_size (int): count of messages that will trigger an immediate send
        req_acks: required acks to use with ProduceRequests. see server protocol
        ack_timeout: timeout to wait for required acks. see server protocol
        retry_options (RetryOptions): settings for retry limits, backoff etc
        stop_event (threading.Event): event to monitor for shutdown signal.
            when this event is 'set', the producer will stop sending messages.
        log_messages_on_error (bool, optional): log stringified message-contents
            on any produce error, otherwise only log a hash() of the contents,
            defaults to True.
        stop_timeout (int or float, optional): number of seconds to continue
            retrying messages after stop_event is set, defaults to 30.
    """
    request_tries = {}

    while not stop_event.is_set():
        try:
            client.reinit()
        except Exception as e:
            log.warn('Async producer failed to connect to brokers; backoff for %s(ms) before retrying', retry_options.backoff_ms)
            time.sleep(float(retry_options.backoff_ms) / 1000)
        else:
            break

    stop_at = None
    while not (stop_event.is_set() and queue.empty() and not request_tries):

        # Handle stop_timeout
        if stop_event.is_set():
            if not stop_at:
                stop_at = stop_timeout + time.time()
            if time.time() > stop_at:
                log.debug('Async producer stopping due to stop_timeout')
                break

        timeout = batch_time
        count = batch_size
        send_at = time.time() + timeout
        msgset = defaultdict(list)

        # Merging messages will require a bit more work to manage correctly
        # for now, don't look for new batches if we have old ones to retry
        if request_tries:
            count = 0
            log.debug('Skipping new batch collection to handle retries')
        else:
            log.debug('Batching size: %s, timeout: %s', count, timeout)

        # Keep fetching till we gather enough messages or a
        # timeout is reached
        while count > 0 and timeout >= 0:
            try:
                topic_partition, msg, key = queue.get(timeout=timeout)
            except Empty:
                break

            # Check if the controller has requested us to stop
            if topic_partition == STOP_ASYNC_PRODUCER:
                stop_event.set()
                break

            # Adjust the timeout to match the remaining period
            count -= 1
            timeout = send_at - time.time()
            msgset[topic_partition].append((msg, key))

        # Send collected requests upstream
        for topic_partition, msg in msgset.items():
            messages = create_message_set(msg, codec, key, codec_compresslevel)
            req = ProduceRequestPayload(
                topic_partition.topic,
                topic_partition.partition,
                tuple(messages))
            request_tries[req] = 0

        if not request_tries:
            continue

        reqs_to_retry, error_cls = [], None
        retry_state = {
            'do_backoff': False,
            'do_refresh': False
        }

        def _handle_error(error_cls, request):
            if issubclass(error_cls, RETRY_ERROR_TYPES) or (retry_options.retry_on_timeouts and issubclass(error_cls, RequestTimedOutError)):
                reqs_to_retry.append(request)
            if issubclass(error_cls, RETRY_BACKOFF_ERROR_TYPES):
                retry_state['do_backoff'] |= True
            if issubclass(error_cls, RETRY_REFRESH_ERROR_TYPES):
                retry_state['do_refresh'] |= True

        requests = list(request_tries.keys())
        log.debug('Sending: %s', requests)
        responses = client.send_produce_request(requests,
                                                acks=req_acks,
                                                timeout=ack_timeout,
                                                fail_on_error=False)

        log.debug('Received: %s', responses)
        for i, response in enumerate(responses):
            error_cls = None
            if isinstance(response, FailedPayloadsError):
                error_cls = response.__class__
                orig_req = response.payload

            elif isinstance(response, ProduceResponsePayload) and response.error:
                error_cls = kafka_errors.get(response.error, UnknownError)
                orig_req = requests[i]

            if error_cls:
                _handle_error(error_cls, orig_req)
                log.error('%s sending ProduceRequestPayload (#%d of %d) '
                          'to %s:%d with msgs %s',
                          error_cls.__name__, (i + 1), len(requests),
                          orig_req.topic, orig_req.partition,
                          orig_req.messages if log_messages_on_error
                                            else hash(orig_req.messages))

        if not reqs_to_retry:
            request_tries = {}
            continue

        # doing backoff before next retry
        if retry_state['do_backoff'] and retry_options.backoff_ms:
            log.warn('Async producer backoff for %s(ms) before retrying', retry_options.backoff_ms)
            time.sleep(float(retry_options.backoff_ms) / 1000)

        # refresh topic metadata before next retry
        if retry_state['do_refresh']:
            log.warn('Async producer forcing metadata refresh metadata before retrying')
            try:
                client.load_metadata_for_topics()
            except Exception:
                log.exception("Async producer couldn't reload topic metadata.")

        # Apply retry limit, dropping messages that are over
        request_tries = dict(
            (key, count + 1)
            for (key, count) in request_tries.items()
                if key in reqs_to_retry
                    and (retry_options.limit is None
                    or (count < retry_options.limit))
        )

        # Log messages we are going to retry
        for orig_req in request_tries.keys():
            log.info('Retrying ProduceRequestPayload to %s:%d with msgs %s',
                     orig_req.topic, orig_req.partition,
                     orig_req.messages if log_messages_on_error
                                       else hash(orig_req.messages))

    if request_tries or not queue.empty():
        log.error('Stopped producer with %d unsent messages', len(request_tries) + queue.qsize())


class Producer(object):
    """
    Base class to be used by producers

    Arguments:
        client (kafka.SimpleClient): instance to use for broker
            communications. If async_send=True, the background thread will use
            :meth:`client.copy`, which is expected to return a thread-safe
            object.
        codec (kafka.protocol.ALL_CODECS): compression codec to use.
        req_acks (int, optional): A value indicating the acknowledgements that
            the server must receive before responding to the request,
            defaults to 1 (local ack).
        ack_timeout (int, optional): millisecond timeout to wait for the
            configured req_acks, defaults to 1000.
        sync_fail_on_error (bool, optional): whether sync producer should
            raise exceptions (True), or just return errors (False),
            defaults to True.
        async_send (bool, optional): send message using a background thread,
            defaults to False.
        batch_send_every_n (int, optional): If async_send is True, messages are
            sent in batches of this size, defaults to 20.
        batch_send_every_t (int or float, optional): If async_send is True,
            messages are sent immediately after this timeout in seconds, even
            if there are fewer than batch_send_every_n, defaults to 20.
        async_retry_limit (int, optional): number of retries for failed messages
            or None for unlimited, defaults to None / unlimited.
        async_retry_backoff_ms (int, optional): milliseconds to backoff on
            failed messages, defaults to 100.
        async_retry_on_timeouts (bool, optional): whether to retry on
            RequestTimedOutError, defaults to True.
        async_queue_maxsize (int, optional): limit to the size of the
            internal message queue in number of messages (not size), defaults
            to 0 (no limit).
        async_queue_put_timeout (int or float, optional): timeout seconds
            for queue.put in send_messages for async producers -- will only
            apply if async_queue_maxsize > 0 and the queue is Full,
            defaults to 0 (fail immediately on full queue).
        async_log_messages_on_error (bool, optional): set to False and the
            async producer will only log hash() contents on failed produce
            requests, defaults to True (log full messages). Hash logging
            will not allow you to identify the specific message that failed,
            but it will allow you to match failures with retries.
        async_stop_timeout (int or float, optional): seconds to continue
            attempting to send queued messages after :meth:`producer.stop`,
            defaults to 30.

    Deprecated Arguments:
        async (bool, optional): send message using a background thread,
            defaults to False. Deprecated, use 'async_send'
        batch_send (bool, optional): If True, messages are sent by a background
            thread in batches, defaults to False. Deprecated, use 'async_send'
    """
    ACK_NOT_REQUIRED = 0            # No ack is required
    ACK_AFTER_LOCAL_WRITE = 1       # Send response after it is written to log
    ACK_AFTER_CLUSTER_COMMIT = -1   # Send response after data is committed
    DEFAULT_ACK_TIMEOUT = 1000

    def __init__(self, client,
                 req_acks=ACK_AFTER_LOCAL_WRITE,
                 ack_timeout=DEFAULT_ACK_TIMEOUT,
                 codec=None,
                 codec_compresslevel=None,
                 sync_fail_on_error=SYNC_FAIL_ON_ERROR_DEFAULT,
                 async_send=False,
                 batch_send=False,  # deprecated, use async_send
                 batch_send_every_n=BATCH_SEND_MSG_COUNT,
                 batch_send_every_t=BATCH_SEND_DEFAULT_INTERVAL,
                 async_retry_limit=ASYNC_RETRY_LIMIT,
                 async_retry_backoff_ms=ASYNC_RETRY_BACKOFF_MS,
                 async_retry_on_timeouts=ASYNC_RETRY_ON_TIMEOUTS,
                 async_queue_maxsize=ASYNC_QUEUE_MAXSIZE,
                 async_queue_put_timeout=ASYNC_QUEUE_PUT_TIMEOUT,
                 async_log_messages_on_error=ASYNC_LOG_MESSAGES_ON_ERROR,
                 async_stop_timeout=ASYNC_STOP_TIMEOUT_SECS,
                 **kwargs):

        # async renamed async_send for python3.7 support
        if 'async' in kwargs:
            log.warning('Deprecated async option found -- use async_send')
            async_send = kwargs['async']

        if async_send:
            assert batch_send_every_n > 0
            assert batch_send_every_t > 0
            assert async_queue_maxsize >= 0

        self.client = client
        self.async_send = async_send
        self.req_acks = req_acks
        self.ack_timeout = ack_timeout
        self.stopped = False

        if codec is None:
            codec = CODEC_NONE
        elif codec not in ALL_CODECS:
            raise UnsupportedCodecError("Codec 0x%02x unsupported" % codec)

        self.codec = codec
        self.codec_compresslevel = codec_compresslevel

        if self.async_send:
            # Messages are sent through this queue
            self.queue = Queue(async_queue_maxsize)
            self.async_queue_put_timeout = async_queue_put_timeout
            async_retry_options = RetryOptions(
                limit=async_retry_limit,
                backoff_ms=async_retry_backoff_ms,
                retry_on_timeouts=async_retry_on_timeouts)
            self.thread_stop_event = Event()
            self.thread = Thread(
                target=_send_upstream,
                args=(self.queue, self.client.copy(), self.codec,
                      batch_send_every_t, batch_send_every_n,
                      self.req_acks, self.ack_timeout,
                      async_retry_options, self.thread_stop_event),
                kwargs={'log_messages_on_error': async_log_messages_on_error,
                        'stop_timeout': async_stop_timeout,
                        'codec_compresslevel': self.codec_compresslevel}
            )

            # Thread will die if main thread exits
            self.thread.daemon = True
            self.thread.start()

            def cleanup(obj):
                if not obj.stopped:
                    obj.stop()
            self._cleanup_func = cleanup
            atexit.register(cleanup, self)
        else:
            self.sync_fail_on_error = sync_fail_on_error

    def send_messages(self, topic, partition, *msg):
        """Helper method to send produce requests.

        Note that msg type *must* be encoded to bytes by user. Passing unicode
        message will not work, for example you should encode before calling
        send_messages via something like `unicode_message.encode('utf-8')`
        All messages will set the message 'key' to None.

        Arguments:
            topic (str): name of topic for produce request
            partition (int): partition number for produce request
            *msg (bytes): one or more message payloads

        Returns:
            ResponseRequest returned by server

        Raises:
            FailedPayloadsError: low-level connection error, can be caused by
                networking failures, or a malformed request.
            KafkaUnavailableError: all known brokers are down when attempting
                to refresh metadata.
            LeaderNotAvailableError: topic or partition is initializing or
                a broker failed and leadership election is in progress.
            NotLeaderForPartitionError: metadata is out of sync; the broker
                that the request was sent to is not the leader for the topic
                or partition.
            UnknownTopicOrPartitionError: the topic or partition has not
                been created yet and auto-creation is not available.
            AsyncProducerQueueFull: in async mode, if too many messages are
                unsent and remain in the internal queue.
        """
        return self._send_messages(topic, partition, *msg)

    def _send_messages(self, topic, partition, *msg, **kwargs):
        key = kwargs.pop('key', None)

        # Guarantee that msg is actually a list or tuple (should always be true)
        if not isinstance(msg, (list, tuple)):
            raise TypeError("msg is not a list or tuple!")

        for m in msg:
            # The protocol allows to have key & payload with null values both,
            # (https://goo.gl/o694yN) but having (null,null) pair doesn't make sense.
            if m is None:
                if key is None:
                    raise TypeError("key and payload can't be null in one")
            # Raise TypeError if any non-null message is not encoded as bytes
            elif not isinstance(m, six.binary_type):
                raise TypeError("all produce message payloads must be null or type bytes")

        # Raise TypeError if the key is not encoded as bytes
        if key is not None and not isinstance(key, six.binary_type):
            raise TypeError("the key must be type bytes")

        if self.async_send:
            for idx, m in enumerate(msg):
                try:
                    item = (TopicPartition(topic, partition), m, key)
                    if self.async_queue_put_timeout == 0:
                        self.queue.put_nowait(item)
                    else:
                        self.queue.put(item, True, self.async_queue_put_timeout)
                except Full:
                    raise AsyncProducerQueueFull(
                        msg[idx:],
                        'Producer async queue overfilled. '
                        'Current queue size %d.' % self.queue.qsize())
            resp = []
        else:
            messages = create_message_set([(m, key) for m in msg], self.codec, key, self.codec_compresslevel)
            req = ProduceRequestPayload(topic, partition, messages)
            try:
                resp = self.client.send_produce_request(
                    [req], acks=self.req_acks, timeout=self.ack_timeout,
                    fail_on_error=self.sync_fail_on_error
                )
            except Exception:
                log.exception("Unable to send messages")
                raise
        return resp

    def stop(self, timeout=None):
        """
        Stop the producer (async mode). Blocks until async thread completes.
        """
        if timeout is not None:
            log.warning('timeout argument to stop() is deprecated - '
                        'it will be removed in future release')

        if not self.async_send:
            log.warning('producer.stop() called, but producer is not async')
            return

        if self.stopped:
            log.warning('producer.stop() called, but producer is already stopped')
            return

        if self.async_send:
            self.queue.put((STOP_ASYNC_PRODUCER, None, None))
            self.thread_stop_event.set()
            self.thread.join()

        if hasattr(self, '_cleanup_func'):
            # Remove cleanup handler now that we've stopped

            # py3 supports unregistering
            if hasattr(atexit, 'unregister'):
                atexit.unregister(self._cleanup_func)  # pylint: disable=no-member

            # py2 requires removing from private attribute...
            else:

                # ValueError on list.remove() if the exithandler no longer exists
                # but that is fine here
                try:
                    atexit._exithandlers.remove(  # pylint: disable=no-member
                        (self._cleanup_func, (self,), {}))
                except ValueError:
                    pass

            del self._cleanup_func

        self.stopped = True

    def __del__(self):
        if self.async_send and not self.stopped:
            self.stop()