|
| 1 | +#!/usr/bin/env python |
| 2 | +# -*- coding: utf-8 -*- |
| 3 | +# |
| 4 | +# Copyright 2020 Confluent Inc. |
| 5 | +# |
| 6 | +# Licensed under the Apache License, Version 2.0 (the "License"); |
| 7 | +# you may not use this file except in compliance with the License. |
| 8 | +# You may obtain a copy of the License at |
| 9 | +# |
| 10 | +# http://www.apache.org/licenses/LICENSE-2.0 |
| 11 | +# |
| 12 | +# Unless required by applicable law or agreed to in writing, software |
| 13 | +# distributed under the License is distributed on an "AS IS" BASIS, |
| 14 | +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 15 | +# See the License for the specific language governing permissions and |
| 16 | +# limitations under the License. |
| 17 | +# |
| 18 | + |
| 19 | +from confluent_kafka.cimpl import (KafkaError, |
| 20 | + Consumer as _ConsumerImpl) |
| 21 | +from .error import ConsumeError |
| 22 | +from .serialization import (SerializationError, |
| 23 | + SerializationContext, |
| 24 | + MessageField) |
| 25 | + |
| 26 | + |
| 27 | +class DeserializingConsumer(_ConsumerImpl): |
| 28 | + """ |
| 29 | + A client that consumes records from a Kafka cluster. With deserialization |
| 30 | + capabilities. |
| 31 | +
|
| 32 | + Note: |
| 33 | +
|
| 34 | + The DeserializingConsumer is an experimental API and subject to change. |
| 35 | +
|
| 36 | + .. versionadded:: 1.4.0 |
| 37 | +
|
| 38 | + The ``key.deserializer`` and ``value.deserializer`` classes instruct the |
| 39 | + DeserializingConsumer on how to convert the message payload bytes to objects. |
| 40 | +
|
| 41 | + Note: |
| 42 | +
|
| 43 | + All configured callbacks are served from the application queue upon |
| 44 | + calling :py:func:`DeserializingConsumer.poll` |
| 45 | +
|
| 46 | + DeserializingConsumer configuration properties(* indicates required field) |
| 47 | +
|
| 48 | + +--------------------+-----------------+-----------------------------------------------------+ |
| 49 | + | Property Name | Type | Description | |
| 50 | + +====================+=================+=====================================================+ |
| 51 | + | bootstrap.servers* | str | Comma-separated list of brokers. | |
| 52 | + +--------------------+-----------------+-----------------------------------------------------+ |
| 53 | + | | | Client group id string. | |
| 54 | + | group.id* | str | All clients sharing the same group.id belong to the | |
| 55 | + | | | same group. | |
| 56 | + +--------------------+-----------------+-----------------------------------------------------+ |
| 57 | + | | | Callable(SerializationContext, bytes) -> obj | |
| 58 | + | key.deserializer | callable | | |
| 59 | + | | | Deserializer used for message keys. | |
| 60 | + +--------------------+-----------------+-----------------------------------------------------+ |
| 61 | + | | | Callable(SerializationContext, bytes) -> obj | |
| 62 | + | value.deserializer | callable | | |
| 63 | + | | | Deserializer used for message values. | |
| 64 | + +--------------------+-----------------+-----------------------------------------------------+ |
| 65 | + | | | Callable(KafkaError) | |
| 66 | + | | | | |
| 67 | + | error_cb | callable | Callback for generic/global error events. These | |
| 68 | + | | | errors are typically to be considered informational | |
| 69 | + | | | since the client will automatically try to recover. | |
| 70 | + +--------------------+-----------------+-----------------------------------------------------+ |
| 71 | + | log_cb | logging.Handler | Logging handler to forward logs | |
| 72 | + +--------------------+-----------------+-----------------------------------------------------+ |
| 73 | + | | | Callable(str) | |
| 74 | + | | | | |
| 75 | + | | | Callback for statistics. This callback is | |
| 76 | + | stats_cb | callable | added to the application queue every | |
| 77 | + | | | ``statistics.interval.ms`` (configured separately). | |
| 78 | + | | | The function argument is a JSON formatted str | |
| 79 | + | | | containing statistics data. | |
| 80 | + +--------------------+-----------------+-----------------------------------------------------+ |
| 81 | + | | | Callable(ThrottleEvent) | |
| 82 | + | throttle_cb | callable | | |
| 83 | + | | | Callback for throttled request reporting. | |
| 84 | + +--------------------+-----------------+-----------------------------------------------------+ |
| 85 | +
|
| 86 | + .. _See Client CONFIGURATION.md for a complete list of configuration properties: |
| 87 | + https://github.com/edenhill/librdkafka/blob/master/CONFIGURATION.md |
| 88 | +
|
| 89 | + Args: |
| 90 | + conf (dict): DeserializingConsumer configuration. |
| 91 | +
|
| 92 | + Raises: |
| 93 | + ValueError: if configuration validation fails |
| 94 | +
|
| 95 | + .. _Statistics: |
| 96 | + https://github.com/edenhill/librdkafka/blob/master/STATISTICS.md |
| 97 | +
|
| 98 | + """ |
| 99 | + |
| 100 | + def __init__(self, conf): |
| 101 | + conf_copy = conf.copy() |
| 102 | + self._key_deserializer = conf_copy.pop('key.deserializer', None) |
| 103 | + self._value_deserializer = conf_copy.pop('value.deserializer', None) |
| 104 | + |
| 105 | + super(DeserializingConsumer, self).__init__(conf_copy) |
| 106 | + |
| 107 | + def poll(self, timeout=-1): |
| 108 | + """ |
| 109 | + Consume messages and calls callbacks. |
| 110 | +
|
| 111 | + Args: |
| 112 | + timeout (float): Maximum time to block waiting for message(Seconds). |
| 113 | +
|
| 114 | + Returns: |
| 115 | + :py:class:`Message` or None on timeout |
| 116 | +
|
| 117 | + Raises: |
| 118 | + ConsumeError if an error was encountered while polling. |
| 119 | +
|
| 120 | + """ |
| 121 | + msg = super(DeserializingConsumer, self).poll(timeout) |
| 122 | + |
| 123 | + if msg is None: |
| 124 | + return None |
| 125 | + |
| 126 | + if msg.error() is not None: |
| 127 | + raise ConsumeError(msg.error(), message=msg) |
| 128 | + |
| 129 | + ctx = SerializationContext(msg.topic(), MessageField.VALUE) |
| 130 | + value = msg.value() |
| 131 | + if self._value_deserializer is not None: |
| 132 | + try: |
| 133 | + value = self._value_deserializer(ctx, value) |
| 134 | + except SerializationError as se: |
| 135 | + raise ConsumeError(KafkaError._VALUE_DESERIALIZATION, |
| 136 | + reason=se.message, |
| 137 | + message=msg) |
| 138 | + |
| 139 | + key = msg.key() |
| 140 | + ctx.field = MessageField.KEY |
| 141 | + if self._key_deserializer is not None: |
| 142 | + try: |
| 143 | + key = self._key_deserializer(ctx, key) |
| 144 | + except SerializationError as se: |
| 145 | + raise ConsumeError(KafkaError._KEY_DESERIALIZATION, |
| 146 | + reason=se.message, |
| 147 | + message=msg) |
| 148 | + |
| 149 | + msg.set_key(key) |
| 150 | + msg.set_value(value) |
| 151 | + return msg |
| 152 | + |
| 153 | + def consume(self, num_messages=1, timeout=-1): |
| 154 | + """ |
| 155 | + :py:func:`Consumer.consume` not implemented, |
| 156 | + :py:func:`DeserializingConsumer.poll` instead |
| 157 | + """ |
| 158 | + raise NotImplementedError |
0 commit comments