pax_global_header00006660000000000000000000000064122762001010014501gustar00rootroot0000000000000052 comment=67bd93cc6c035a56f70af9938e38e1845dc72010 librdkafka-0.8.3/000077500000000000000000000000001227620010100136035ustar00rootroot00000000000000librdkafka-0.8.3/.dir-locals.el000066400000000000000000000000521227620010100162310ustar00rootroot00000000000000( (c-mode . ((c-file-style . "linux"))) ) librdkafka-0.8.3/.gitignore000066400000000000000000000001031227620010100155650ustar00rootroot00000000000000*~ \#* *.o *.so *.so.? *.a *.d librdkafka.lds core *dSYM/ *.offset librdkafka-0.8.3/.travis.yml000066400000000000000000000001111227620010100157050ustar00rootroot00000000000000language: c compiler: - gcc - clang script: make all examples check librdkafka-0.8.3/CONFIGURATION.md000066400000000000000000000163041227620010100161400ustar00rootroot00000000000000## Global configuration properties Property | Default | Description -----------------------------------------|--------------:|-------------------------- client.id | rdkafka | Client identifier. metadata.broker.list | | Initial list of brokers. The application may also use `rd_kafka_brokers_add()` to add brokers during runtime. message.max.bytes | 4000000 | Maximum transmit message size. receive.message.max.bytes | 100000000 | Maximum receive message size. This is a safety precaution to avoid memory exhaustion in case of protocol hickups. The value should be at least fetch.message.max.bytes * number of partitions consumed from. metadata.request.timeout.ms | 60000 | Non-topic request timeout in milliseconds. This is for metadata requests, etc. topic.metadata.refresh.interval.ms | 10000 | Topic metadata refresh interval in milliseconds. The metadata is automatically refreshed on error and connect. Use -1 to disable the intervalled refresh. topic.metadata.refresh.fast.cnt | 10 | When a topic looses its leader this number of metadata requests are sent with `topic.metadata.refresh.fast.interval.ms` interval disregarding the `topic.metadata.refresh.interval.ms` value. This is used to recover quickly from transitioning leader brokers. topic.metadata.refresh.fast.interval.ms | 250 | See `topic.metadata.refresh.fast.cnt` description debug | | A comma-separated list of debug contexts to enable: all,generic,broker,topic,metadata,producer,queue,msg socket.timeout.ms | 60000 | Timeout for network requests. socket.send.buffer.bytes | 0 | Broker socket send buffer size. System default is used if 0. socket.receive.buffer.bytes | 0 | Broker socket receive buffer size. System default is used if 0. broker.address.ttl | 300000 | How long to cache the broker address resolving results. statistics.interval.ms | 0 | librdkafka statistics emit interval. The application also needs to register a stats callback using `rd_kafka_conf_set_stats_cb()`. The granularity is 1000ms. A value of 0 disables statistics. error_cb | | Error callback (set with rd_kafka_conf_set_error_cb()) stats_cb | | Statistics callback (set with rd_kafka_conf_set_stats_cb()) opaque | | Application opaque (set with rd_kafka_conf_set_opaque()) queued.min.messages | 100000 | Minimum number of messages that should to be available for consumption by application. fetch.wait.max.ms | 100 | Maximum time the broker may wait to fill the response with fetch.min.bytes. fetch.message.max.bytes | 1048576 | Maximum number of bytes per topic+partition to request when fetching messages from the broker. fetch.min.bytes | 1 | Minimum number of bytes the broker responds with. If fetch.wait.max.ms expires the accumulated data will be sent to the client regardless of this setting. fetch.error.backoff.ms | 500 | How long to postpone the next fetch request for a topic+partition in case of a fetch error. queue.buffering.max.messages | 100000 | Maximum number of messages allowed on the producer queue. queue.buffering.max.ms | 1000 | Maximum time, in milliseconds, for buffering data on the producer queue. message.send.max.retries | 2 | How many times to retry sending a failing MessageSet. **Note:** retrying may cause reordering. retry.backoff.ms | 100 | The backoff time in milliseconds before retrying a message send. compression.codec | none | Compression codec to use for compressing message sets: none, gzip or snappy batch.num.messages | 1000 | Maximum number of messages batched in one MessageSet. dr_cb | | Delivery report callback (set with rd_kafka_conf_set_dr_cb()) ## Topic configuration properties Property | Default | Description -----------------------------------------|--------------:|-------------------------- request.required.acks | 1 | This field indicates how many acknowledgements the leader broker must receive from ISR brokers before responding to the request: *0*=broker does not send any response, *1*=broker will wait until the data is written to local log before sending a response, *-1*=broker will block until message is committed by all in sync replicas (ISRs) before sending response. *>1*=for any number > 1 the broker will block waiting for this number of acknowledgements to be received (but the broker will never wait for more acknowledgements than there are ISRs). request.timeout.ms | 5000 | The ack timeout of the producer request in milliseconds. This value is only enforced by the broker and relies on `request.required.acks` being > 0. message.timeout.ms | 300000 | Local message timeout. This value is only enforced locally and limits the time a produced message waits for successful delivery. partitioner | | Partitioner callback (set with rd_kafka_topic_conf_set_partitioner_cb()) opaque | | Application opaque (set with rd_kafka_topic_conf_set_opaque()) auto.commit.enable | true | If true, periodically commit offset of the last message handed to the application. This commited offset will be used when the process restarts to pick up where it left off. If false, the application will have to call `rd_kafka_offset_store()` to store an offset (optional). **NOTE:** There is currently no zookeeper integration, offsets will be written to local file according to offset.store.path. auto.commit.interval.ms | 60000 | The frequency in milliseconds that the consumer offsets are commited (written) to offset storage. auto.offset.reset | largest | Action to take when there is no initial offset in offset store or the desired offset is out of range: 'smallest' - automatically reset the offset to the smallest offset, 'largest' - automatically reset the offset to the largest offset, 'error' - trigger an error which is retrieved by consuming messages and checking 'message->err'. offset.store.path | . | Path to local file for storing offsets. If the path is a directory a filename will be automatically generated in that directory based on the topic and partition. offset.store.sync.interval.ms | -1 | fsync() interval for the offset file, in milliseconds. Use -1 to disable syncing, and 0 for immediate sync after each write. librdkafka-0.8.3/INTRODUCTION.md000066400000000000000000000443011227620010100160500ustar00rootroot00000000000000# Introduction to librdkafka - the Apache Kafka C client library librdkafka is a high performance C implementation of the Apache Kafka client, providing a reliable and performant client for production use. ## Contents The following chapters are available in this document * Performance * Performance numbers * High throughput * Low latency * Compression * Message reliability * Usage * Documentation * Initialization * Configuration * Threads and callbacks * Brokers * Producer API * Consumer API * Appendix * Test detailts ## Performance librdkafka is a multi-threaded library designed for use on modern hardware and it attempts to keep memory copying at a minimal. The payload of produced or consumed messages may pass through without any copying (if so desired by the application) putting no limit on message sizes. > *"You can have high throughput or low latency, but you cant have both"* librdkafka allows you to decide if high throughput is the name of the game, or if a low latency service is required, all through the configuration property interface. The two most important configuration properties for performance tuning are: * batch.num.messages - the minimum number of messages to wait for to accumulate in the local queue before sending off a message set. * queue.buffering.max.ms - how long to wait for batch.num.messages to fill up in the local queue. ### Performance numbers The following performance numbers stem from tests using the following setup: * Intel Quad Core i7 at 3.4GHz, 8GB of memory * Disk performance has been shortcut by setting the brokers' flush configuration properties as so: * `log.flush.interval.messages=10000000` * `log.flush.interval.ms=100000` * Two brokers running on the same machine as librdkafka. * One topic with two partitions. * Each broker is leader for one partition each. * Using `rdkafka_performance` program available in the `examples` subdir. **Test results** * **Test1**: 2 brokers, 2 partitions, required.acks=2, 100 byte messages: **850000 messages/second**, **85 MB/second** * **Test2**: 1 broker, 1 partition, required.acks=0, 100 byte messages: **710000 messages/second**, **71 MB/second** * **Test3**: 2 broker2, 2 partitions, required.acks=2, 100 byte messages, snappy compression: **300000 messages/second**, **30 MB/second** * **Test4**: 2 broker2, 2 partitions, required.acks=2, 100 byte messages, gzip compression: **230000 messages/second**, **23 MB/second** **Note**: See the *Test details* chapter at the end of this document for information about the commands executed, etc. **Note**: Consumer performance tests will be announced soon. ### High throughput The key to high throughput is message batching - waiting for a certain amount of messages to accumulate in the local queue before sending them off in one large message set or batch to the peer. This amortizes the messaging overhead and eliminates the adverse effect of the round trip time (rtt). The default settings, batch.num.messages=1000 and queue.buffering.max.ms=1000, are suitable for high throughput. This allows librdkafka to wait up to 1000 ms for up to 1000 messages to accumulate in the local queue before sending the accumulate messages to the broker. These setting are set globally (`rd_kafka_conf_t`) but applies on a per topic+partition basis. ### Low latency When low latency messaging is required the "queue.buffering.max.ms" should be tuned to the maximum permitted producer-side latency. Setting queue.buffering.max.ms to 0 will make sure messages are sent as soon as possible. ### Compression Producer message compression is enabled through the "compression.codec" configuration property. Compression is performed on the batch of messages in the local queue, the larger the batch the higher likelyhood of a higher compression ratio. The local batch queue size is controlled through the "batch.num.messages" and "queue.buffering.max.ms" configuration properties as described in the **High throughput** chapter above. ## Message reliability Message reliability is an important factor of librdkafka - an application can rely fully on librdkafka to deliver a message according to the specified configuration ("request.required.acks" and "message.send.max.retries", etc). If the topic configuration property "request.required.acks" is set to wait for message commit acknowledgements from brokers (any value but 0, see [`CONFIGURATION.md`](https://github.com/edenhill/librdkafka/blob/master/CONFIGURATION.md) for specifics) then librdkafka will hold on to the message until all expected acks have been received, gracefully handling the following events: * Broker connection failure * Topic leader change * Produce errors signaled by the broker This is handled automatically by librdkafka and the application does not need to take any action at any of the above events. The message will be resent up to "message.send.max.retries" times before reporting a failure back to the application. The delivery report callback is used by librdkafka to signal the status of a message back to the application, it will be called once for each message to report the status of message delivery: * If `error_code` is non-zero the message delivery failed and the error_code indicates the nature of the failure (`rd_kafka_resp_err_t` enum). * If `error_code` is zero the message has been successfully delivered. See Producer API chapter for more details on delivery report callback usage. The delivery report callback is optional. ## Usage ### Documentation The librdkafka API is documented in the [`rdkafka.h`](https://github.com/edenhill/librdkafka/blob/master/rdkafka.h) header file, the configuration properties are documented in [`CONFIGURATION.md`](https://github.com/edenhill/librdkafka/blob/master/CONFIGURATION.md) ### Initialization The application needs to instantiate a top-level object `rd_kafka_t` which is the base container, providing global configuration and shared state. It is created by calling `rd_kafka_new()`. It also needs to instantiate one or more topics (`rd_kafka_topic_t`) to be used for producing to or consuming from. The topic object holds topic-specific configuration and will be internally populated with a mapping of all available partitions and their leader brokers. It is created by calling `rd_kafka_topic_new()`. Both `rd_kafka_t` and `rd_kafka_topic_t` comes with a configuration API which is optional. Not using the API will cause librdkafka to use its default values which are documented *`CONFIGURATION.md`*. **Note**: An application may create multiple `rd_kafka_t` objects and they share no state. **Note**: An `rd_kafka_topic_t` object may only be used with the `rd_kafka_t` object it was created from. ### Configuration To ease integration with the official Apache Kafka software and lower the learning curve, librdkafka implements identical configuration properties as found in the official clients of Apache Kafka. Configuration is applied prior to object creation using the `rd_kafka_conf_set()` and `rd_kafka_topic_conf_set()` APIs. **Note**: The `rd_kafka.._conf_t` objects are not reusable after have been passed to `rd_kafka.._new()`. The application does not need to free any config resources after a `rd_kafka.._new()` call. #### Example rd_kafka_conf_t *conf; char errstr[512]; conf = rd_kafka_conf_new(); rd_kafka_conf_set(conf, "compression.codec", "snappy", errstr, sizeof(errstr)); rd_kafka_conf_set(conf, "batch.num.messages", "100", errstr, sizeof(errstr)); rd_kafka_new(RD_KAFKA_PRODUCER, conf); ### Threads and callbacks librdkafka uses multiple threads internally to fully utilize modern hardware. The API is completely thread-safe and the calling application may call any of the API functions from any of its own threads at any time. A poll-based API is used to provide signaling back to the application, the application should call rd_kafka_poll() at regular intervals. The poll API will call the following configured callbacks (optional): * message delivery report callback - signals that a message has been delivered or failed delivery, allowing the application to take action and to release any application resources used in the message. * error callback - signals an error. These errors are usually of an informational nature, i.e., failure to connect to a broker, and the application usually does not need to take any action. The type of error is passed as a rd_kafka_resp_err_t enum value, including both remote broker errors as well as local failures. Optional callbacks not triggered by poll, these may be called from any thread: * Logging callback - allows the application to output log messages generated by librdkafka. * partitioner callback - application provided message partitioner. The partitioner may be called in any thread at any time, it may be called multiple times for the same key. Partitioner function contraints: * MUST NOT call any rd_kafka_*() functions * MUST NOT block or execute for prolonged periods of time. * MUST return a value between 0 and partition_cnt-1, or the special RD_KAFKA_PARTITION_UA value if partitioning could not be performed. ### Brokers librdkafka only needs an initial list of brokers (at least one), called the bootstrap brokers. It will connect to all the bootstrap brokers, specified by the "metadata.broker.list" configuration property or by `rd_kafka_brokers_add()`, and query each one for Metadata information which contains the full list of brokers, topic, partitions and their leaders in the Kafka cluster. Broker names are specified as "host[:port]" where the port is optional (default 9092) and the host is either a resolvable hostname or an IPv4 or IPv6 address. If host resolves to multiple addresses librdkafka will round-robin the addresses for each connection attempt. A DNS record containing all broker address can thus be used to provide a reliable bootstrap broker. ### Producer API After setting up the `rd_kafka_t` object with type `RD_KAFKA_PRODUCER` and one or more `rd_kafka_topic_t` objects librdkafka is ready for accepting messages to be produced and sent to brokers. The `rd_kafka_produce()` function takes the following arguments: * `rkt` - the topic to produce to, previously created with `rd_kafka_topic_new()` * `partition` - partition to produce to. If this is set to `RD_KAFKA_PARTITION_UA` (UnAssigned) then the configured partitioner function will be used to select a target partition. * `payload`,`len` - the message payload * `msgflags` - 0, or one of: * `RD_KAFKA_MSG_F_COPY` - librdkafka will immediately make a copy of the payload. Use this when the payload is in non-persistent memory, such as the stack. * `RD_KAFKA_MSG_F_FREE` - let librdkafka free the payload using `free(3)` when it is done with it. These two flags are mutually exclusive and neither need to be set in which case the payload is neither copied nor freed by librdkafka. If `RD_KAFKA_MSG_F_COPY` flag is not set no data copying will be performed and librdkafka will hold on the payload pointer until the message has been delivered or fails. The delivery report callback will be called when librdkafka is done with the message to let the application regain ownership of the payload memory. The application must not free the payload in the delivery report callback if `RD_KAFKA_MSG_F_FREE is set`. * `key`,`keylen` - an optional message key which can be used for partitioning. It will be passed to the topic partitioner callback, if any, and will be attached to the message when sending to the broker. * `msg_opaque` - an optional application-provided per-message opaque pointer that will be provided in the message delivery callback to let the application reference a specific message. `rd_kafka_produce()` is a non-blocking API, it will enqueue the message on an internal queue and return immediately. If the number of queued messages would exceed the "queue.buffering.max.messages" configuration property then `rd_kafka_produce()` returns -1 and sets errno to `ENOBUFS`, thus providing a backpressure mechanism. **Note**: See `examples/rdkafka_performance.c` for a producer implementation. ### Consumer API The consumer API is a bit more stateful than the producer API. After creating `rd_kafka_t` with type `RD_KAFKA_CONSUMER` and `rd_kafka_topic_t` instances the application must also start the consumer for a given partition by calling `rd_kafka_consume_start()`. `rd_kafka_consume_start()` arguments: * `rkt` - the topic to start consuming from, previously created with `rd_kafka_topic_new()`. * `partition` - partition to consume from. * `offset` - message offset to start consuming from. This may either be an absolute message offset or one of the two special offsets: `RD_KAFKA_OFFSET_BEGINNING` to start consuming from the beginning of the partition's queue (oldest message), or `RD_KAFKA_OFFSET_END` to start consuming at the next message to be produced to the partition, or `RD_KAFKA_OFFSET_STORED` to use the offset store. After a topic+partition consumer has been started librdkafka will attempt to keep "queued.min.messages" messages in the local queue by repeatedly fetching batches of messages from the broker. This local message queue is then served to the application through three different consume APIs: * `rd_kafka_consume()` - consumes a single message * `rd_kafka_consume_batch()` - consumes one or more messages * `rd_kafka_consume_callback()` - consumes all messages in the local queue and calls a callback function for each one. These three APIs are listed above the ascending order of performance, `rd_kafka_consume()` being the slowest and `rd_kafka_consume_callback()` being the fastest. The different consume variants are provided to cater for different application needs. A consumed message, as provided or returned by each of the consume functions, is represented by the `rd_kafka_message_t` type. `rd_kafka_message_t` members: * `err` - Error signaling back to the application. If this field is non-zero the `payload` field should be considered an error message and `err` is an error code (`rd_kafka_resp_err_t`). If `err` is zero then the message is a proper fetched message and `payload` et.al contains message payload data. * `rkt`,`partition` - Topic and partition for this message or error. * `payload`,`len` - Message payload data or error message (err!=0). * `key`,`key_len` - Optional message key as specified by the producer * `offset` - Message offset Both the `payload` and `key` memory, as well as the message as a whole, is owned by librdkafka and must not be used after an `rd_kafka_message_destroy()` call. librdkafka will share the same messageset receive buffer memory for all message payloads of that messageset to avoid excessive copying which means that if the application decides to hang on to a single `rd_kafka_message_t` it will hinder the backing memory to be released for all other messages from the same messageset. When the application is done consuming messages from a topic+partition it should call `rd_kafka_consume_stop()` to stop the consumer. This will also purge any messages currently in the local queue. **Note**: See `examples/rdkafka_performance.c` for a consumer implementation. #### Offset management Offset management is available through a local offset file store, where the offset is periodically written to a local file for each topic+partition according to the following topic configuration properties: * `auto.commit.enable` * `auto.commit.interval.ms` * `offset.store.path` * `offset.store.sync.interval.ms` There is currently no support for offset management with ZooKeeper. #### Consumer groups There is currently no support for consumer groups, the librdkafka consumer API resembles the official scala Simple Consumer. The application should provide its own consumer group management until librdkafka adds support for it. ### Topics #### Topic auto creation Topic auto creation is supported by librdkafka. The broker needs to be configured with "auto.create.topics.enable=true". ## Appendix ### Test details #### Test1: Produce to two brokers, two partitions, required.acks=2, 100 byte messages Each broker is leader for one of the two partitions. The random partitioner is used (default) and each broker and partition is assigned approximately 250000 messages each. **Command:** # examples/rdkafka_performance -P -t test2 -s 100 -c 500000 -m "_____________Test1:TwoBrokers:500kmsgs:100bytes" -S 1 -a 2 .... % 500000 messages and 50000000 bytes sent in 587ms: 851531 msgs/s and 85.15 Mb/s, 0 messages failed, no compression **Result:** Message transfer rate is approximately **850000 messages per second**, **85 megabytes per second**. #### Test2: Produce to one broker, one partition, required.acks=0, 100 byte messages **Command:** # examples/rdkafka_performance -P -t test2 -s 100 -c 500000 -m "_____________Test2:OneBrokers:500kmsgs:100bytes" -S 1 -a 0 -p 1 .... % 500000 messages and 50000000 bytes sent in 698ms: 715994 msgs/s and 71.60 Mb/s, 0 messages failed, no compression **Result:** Message transfer rate is approximately **710000 messages per second**, **71 megabytes per second**. #### Test3: Produce to two brokers, two partitions, required.acks=2, 100 byte messages, snappy compression **Command:** # examples/rdkafka_performance -P -t test2 -s 100 -c 500000 -m "_____________Test3:TwoBrokers:500kmsgs:100bytes:snappy" -S 1 -a 2 -z snappy .... % 500000 messages and 50000000 bytes sent in 1672ms: 298915 msgs/s and 29.89 Mb/s, 0 messages failed, snappy compression **Result:** Message transfer rate is approximately **300000 messages per second**, **30 megabytes per second**. #### Test4: Produce to two brokers, two partitions, required.acks=2, 100 byte messages, gzip compression **Command:** # examples/rdkafka_performance -P -t test2 -s 100 -c 500000 -m "_____________Test3:TwoBrokers:500kmsgs:100bytes:gzip" -S 1 -a 2 -z gzip .... % 500000 messages and 50000000 bytes sent in 2111ms: 236812 msgs/s and 23.68 Mb/s, 0 messages failed, gzip compression **Result:** Message transfer rate is approximately **230000 messages per second**, **23 megabytes per second**. librdkafka-0.8.3/LICENSE000066400000000000000000000025131227620010100146110ustar00rootroot00000000000000librdkafka - Apache Kafka C driver library Copyright (c) 2012, Magnus Edenhill All rights reserved. Redistribution and use in source and binary forms, with or without modification, are permitted provided that the following conditions are met: 1. Redistributions of source code must retain the above copyright notice, this list of conditions and the following disclaimer. 2. Redistributions in binary form must reproduce the above copyright notice, this list of conditions and the following disclaimer in the documentation and/or other materials provided with the distribution. THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. librdkafka-0.8.3/LICENSE.pycrc000066400000000000000000000023671227620010100157370ustar00rootroot00000000000000The following license applies to the files rdcrc32.c and rdcrc32.h which have been generated by the pycrc tool. ============================================================================ Copyright (c) 2006-2012, Thomas Pircher Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated documentation files (the "Software"), to deal in the Software without restriction, including without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit persons to whom the Software is furnished to do so, subject to the following conditions: The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software. THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. librdkafka-0.8.3/LICENSE.snappy000066400000000000000000000035761227620010100161340ustar00rootroot00000000000000###################################################################### # LICENSE.snappy covers files: snappy.c, snappy.h, snappy_compat.h # # originally retrieved from http://github.com/andikleen/snappy-c # # git revision 711c52b7ef94c8e5c600571987fbe5769070b884 # ###################################################################### The snappy-c code is under the same license as the original snappy source Copyright 2011 Intel Corporation All Rights Reserved. Redistribution and use in source and binary forms, with or without modification, are permitted provided that the following conditions are met: * Redistributions of source code must retain the above copyright notice, this list of conditions and the following disclaimer. * Redistributions in binary form must reproduce the above copyright notice, this list of conditions and the following disclaimer in the documentation and/or other materials provided with the distribution. * Neither the name of Intel Corporation nor the names of its contributors may be used to endorse or promote products derived from this software without specific prior written permission. THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. librdkafka-0.8.3/Makefile000077500000000000000000000055431227620010100152550ustar00rootroot00000000000000LIBNAME=librdkafka LIBVER=1 DESTDIR?=/usr/local SRCS= rdkafka.c rdkafka_broker.c rdkafka_msg.c rdkafka_topic.c \ rdkafka_defaultconf.c rdkafka_timer.c rdkafka_offset.c SRCS+= rdcrc32.c rdgz.c rdaddr.c rdrand.c rdthread.c rdqueue.c rdlog.c SRCS+= snappy.c HDRS= rdkafka.h OBJS= $(SRCS:.c=.o) DEPS= ${OBJS:%.o=%.d} CFLAGS+=-O2 -Wall -Werror -Wfloat-equal -Wpointer-arith -I. CFLAGS+=-g # Clang warnings to ignore ifeq ($(CC),clang) CFLAGS+=-Wno-gnu-designator endif # Enable iovecs in snappy CFLAGS+=-DSG # Profiling #CFLAGS+=-O0 #CFLAGS += -pg #LDFLAGS += -pg LDFLAGS+= -g UNAME_S = $(shell uname -s) ifneq ($(findstring CYGWIN,$(UNAME_S)),CYGWIN) LDFLAGS+=-fPIC CFLAGS+=-fPIC endif .PHONY: all: libs check libs: $(LIBNAME).so.$(LIBVER) $(LIBNAME).a CONFIGURATION.md %.o: %.c $(CC) -MD -MP $(CFLAGS) -c $< librdkafka.lds: rdkafka.h @echo "\033[33mGenerating linker script $@\033[0m" @./lds-gen.pl > $@ $(LIBNAME).so.$(LIBVER): $(OBJS) librdkafka.lds @echo "\033[33mCreating shared library $@\033[0m" @(if [ $(UNAME_S) = "Darwin" ]; then \ $(CC) $(LDFLAGS) \ $(OBJS) -dynamiclib -o $@ -lpthread -lz -lc ; \ else \ $(CC) $(LDFLAGS) \ -shared -Wl,-soname,$@ \ -Wl,--version-script=librdkafka.lds \ $(OBJS) -o $@ -lpthread -lrt -lz -lc ; \ fi) $(LIBNAME).a: $(OBJS) @echo "\033[33mCreating static library $@\033[0m" $(AR) rcs $@ $(OBJS) CONFIGURATION.md: rdkafka.h examples @echo "\033[33mUpdating $@\033[0m" @(examples/rdkafka_performance -X list > CONFIGURATION.md.tmp; \ cmp CONFIGURATION.md CONFIGURATION.md.tmp || \ mv CONFIGURATION.md.tmp CONFIGURATION.md; \ rm -f CONFIGURATION.md.tmp) examples: .PHONY make -C $@ install: @echo "\033[33mInstall to root $(DESTDIR)\033[0m" if [ "$(DESTDIR)" != "/usr/local" ]; then \ DESTDIR="$(DESTDIR)/usr"; \ else \ DESTDIR="$(DESTDIR)" ; \ fi ; \ install -d $$DESTDIR/include/librdkafka $$DESTDIR/lib ; \ install $(HDRS) $$DESTDIR/include/$(LIBNAME) ; \ install $(LIBNAME).a $$DESTDIR/lib ; \ install $(LIBNAME).so.$(LIBVER) $$DESTDIR/lib ; \ (cd $$DESTDIR/lib && ln -sf $(LIBNAME).so.$(LIBVER) $(LIBNAME).so) tests: .PHONY check make -C tests check: @echo "\033[33mChecking integrity\033[0m" @(RET=true ; \ for f in librdkafka.so.1 librdkafka.a CONFIGURATION.md \ examples/rdkafka_example examples/rdkafka_performance ; do \ printf "%-30s " $$f ; \ if [ -f "$$f" ]; then \ echo "\033[32mOK\033[0m"; \ else \ echo "\033[31mMISSING\033[0m"; \ RET=false ; \ fi; \ done ; \ $$($$RET)) @(printf "%-30s " "Symbol visibility" ; \ ((nm -D librdkafka.so.1 | grep -q rd_kafka_new) && \ (nm -D librdkafka.so.1 | grep -vq rd_kafka_destroy) && \ echo "\033[32mOK\033[0m") || \ echo "\033[31mFAILED\033[0m") clean: rm -f $(OBJS) $(DEPS) \ $(LIBNAME)*.a $(LIBNAME)*.so $(LIBNAME)*.so.$(LIBVER) \ librdkafka.lds make -C tests clean make -C examples clean -include $(DEPS) librdkafka-0.8.3/README.md000066400000000000000000000064761227620010100150770ustar00rootroot00000000000000librdkafka - Apache Kafka C client library ========================================== Copyright (c) 2012-2013, [Magnus Edenhill](http://www.edenhill.se/). [https://github.com/edenhill/librdkafka](https://github.com/edenhill/librdkafka) **librdkafka** is a C library implementation of the [Apache Kafka](http://kafka.apache.org/) protocol, containing both Producer and Consumer support. It was designed with message delivery reliability and high performance in mind, current figures exceed 800000 msgs/second for the producer and 3 million msgs/second for the consumer. **librdkafka** is licensed under the 2-clause BSD license. For an introduction to the performance and usage of librdkafka, see [INTRODUCTION.md](https://github.com/edenhill/librdkafka/blob/master/INTRODUCTION.md) **NOTE**: A C++ interface is currently in testing, see the `c++` branch. **Apache Kafka 0.8 support:** * Branch: master * Producer: supported * Consumer: supported * Compression: snappy and gzip * Debian package: librdkafka1 and librdkafka-dev in Debian testing and unstable * ZooKeeper: not supported * API: Stable, not backwards compatible * Tests: Regression tests in `tests/` directory. * Statistics: JSON formatted, see `rd_kafka_conf_set_stats_cb` in `rdkafka.h`. * Status: Stable **Apache Kafka 0.7 support:** * Branch: 0.7 * Producer: supported * Consumer: supported * Compression: not supported * ZooKeeper: not supported * API: backwards compatible with 0.6 * Status: Stable **Apache Kafka 0.6 support:** * Branch: 0.6 * Producer: supported * Consumer: supported * Compression: not supported * ZooKeeper: not supported * Status: Testing #Users of librdkafka# * [Wikimedia's varnishkafka](https://github.com/wikimedia/varnishkafka) * [redBorder](http://www.redborder.net) * [Headweb](http://www.headweb.com/) * [node-kafka](https://github.com/sutoiku/node-kafka) * [Produban's log2kafka](https://github.com/Produban/log2kafka) * [phpkafka](https://github.com/salebab/phpkafka) * *Let [me](mailto:rdkafka@edenhill.se) know if you are using librdkafka* # Usage ## Requirements The GNU toolchain pthreads zlib ## Instructions ### Building make all sudo make install # or to install in another location than /usr/local, set DESTDIR env # to the filesystem root of your choice. sudo make DESTDIR=/usr make install ### Usage in code See [examples/rdkafka_example.c](https://github.com/edenhill/librdkafka/blob/master/examples/rdkafka_example.c) for an example producer and consumer. Link your program with `-lrdkafka -lz -lpthread -lrt`. ## Documentation The API is documented in `rdkafka.h` Configuration properties are documented in [CONFIGURATION.md](https://github.com/edenhill/librdkafka/blob/master/CONFIGURATION.md) For a librdkafka introduction, see [INTRODUCTION.md](https://github.com/edenhill/librdkafka/blob/master/INTRODUCTION.md) ## Examples See the `examples/`sub-directory. ## Tests See the `tests/`sub-directory. ## Support File bug reports, feature requests and questions using [GitHub Issues](https://github.com/edenhill/librdkafka/issues) Questions and discussions are also welcome on irc.freenode.org, #apache-kafka, nickname Snaps. ### Commercial support Commercial support is available from [Edenhill services](http://www.edenhill.se) librdkafka-0.8.3/endian_compat.h000077500000000000000000000052511227620010100165630ustar00rootroot00000000000000/* * librdkafka - Apache Kafka C library * * Copyright (c) 2012 - 2014 Magnus Edenhill * All rights reserved. * * Redistribution and use in source and binary forms, with or without * modification, are permitted provided that the following conditions are met: * * 1. Redistributions of source code must retain the above copyright notice, * this list of conditions and the following disclaimer. * 2. Redistributions in binary form must reproduce the above copyright notice, * this list of conditions and the following disclaimer in the documentation * and/or other materials provided with the distribution. * * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE * POSSIBILITY OF SUCH DAMAGE. */ #ifdef __FreeBSD__ #include #elif defined __linux__ #include #elif defined __CYGWIN__ #include #elif defined __BSD__ #include #elif defined __APPLE__ #include #include #define __bswap_64(x) OSSwapInt64(x) #define __bswap_32(x) OSSwapInt32(x) #define __bswap_16(x) OSSwapInt16(x) #if __DARWIN_BYTE_ORDER == __DARWIN_BIG_ENDIAN #define htobe16(x) (x) #define htole16(x) __bswap_16 (x) #define be16toh(x) (x) #define le16toh(x) __bswap_16 (x) #define htobe32(x) (x) #define htole32(x) __bswap_32 (x) #define be32toh(x) (x) #define le32toh(x) __bswap_32 (x) #define htobe64(x) (x) #define htole64(x) __bswap_64 (x) #define be64toh(x) (x) #define le64toh(x) __bswap_64 (x) #else #define htobe16(x) __bswap_16 (x) #define htole16(x) (x) #define be16toh(x) __bswap_16 (x) #define le16toh(x) (x) #define htobe32(x) __bswap_32 (x) #define htole32(x) (x) #define be32toh(x) __bswap_32 (x) #define le32toh(x) (x) #define htobe64(x) __bswap_64 (x) #define htole64(x) (x) #define be64toh(x) __bswap_64 (x) #define le64toh(x) (x) #endif #else #error Unknown location for endian.h #endif librdkafka-0.8.3/examples/000077500000000000000000000000001227620010100154215ustar00rootroot00000000000000librdkafka-0.8.3/examples/.gitignore000066400000000000000000000000711227620010100174070ustar00rootroot00000000000000rdkafka_example rdkafka_performance rdkafka_example_cpp librdkafka-0.8.3/examples/Makefile000066400000000000000000000023211227620010100170570ustar00rootroot00000000000000EXAMPLES ?= rdkafka_example rdkafka_performance CC ?= cc CXX ?= g++ CFLAGS += -g -Wall -Werror -Wfloat-equal -Wpointer-arith -O2 -I../ CXXFLAGS += $(CFLAGS) LDFLAGS += ../librdkafka.a LDFLAGS += -lpthread -lz ifeq ($(shell uname -s), Linux) LDFLAGS += -lrt endif # Profiling #CFLAGS += -O0 -pg #LDFLAGS += -pg all: $(EXAMPLES) rdkafka_example: ../librdkafka.a rdkafka_example.c @(test $@ -nt $< || \ $(CC) $(CFLAGS) rdkafka_example.c -o $@ $(LDFLAGS)) @echo "# $@ is ready" @echo "#" @echo "# Run producer (write messages on stdin)" @echo "./rdkafka_example -P -t -p " @echo "" @echo "# or consumer" @echo "./rdkafka_example -C -t -p " @echo "" @echo "#" @echo "# More usage options:" @echo "./rdkafka_example --help" rdkafka_performance: ../librdkafka.a rdkafka_performance.c @(test $@ -nt $< || \ $(CC) $(CFLAGS) rdkafka_performance.c -o $@ $(LDFLAGS)) @echo "# $@ is ready" @echo "#" @echo "# Run producer" @echo "./$@ -P -t -p -s " @echo "" @echo "# or consumer" @echo "./$@ -C -t -p " @echo "" @echo "#" @echo "# More usage options:" @echo "./$@ --help" clean: rm -f rdkafka_example rdkafka_performance librdkafka-0.8.3/examples/rdkafka_example.c000077500000000000000000000314261227620010100207140ustar00rootroot00000000000000/* * librdkafka - Apache Kafka C library * * Copyright (c) 2012, Magnus Edenhill * All rights reserved. * * Redistribution and use in source and binary forms, with or without * modification, are permitted provided that the following conditions are met: * * 1. Redistributions of source code must retain the above copyright notice, * this list of conditions and the following disclaimer. * 2. Redistributions in binary form must reproduce the above copyright notice, * this list of conditions and the following disclaimer in the documentation * and/or other materials provided with the distribution. * * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE * POSSIBILITY OF SUCH DAMAGE. */ /** * Apache Kafka consumer & producer example programs * using the Kafka driver from librdkafka * (https://github.com/edenhill/librdkafka) */ #include #include #include #include #include #include #include #include /* Typical include path would be , but this program * is builtin from within the librdkafka source tree and thus differs. */ #include "rdkafka.h" /* for Kafka driver */ static int run = 1; static rd_kafka_t *rk; static int exit_eof = 0; static int quiet = 0; static enum { OUTPUT_HEXDUMP, OUTPUT_RAW, } output = OUTPUT_HEXDUMP; static void stop (int sig) { run = 0; fclose(stdin); /* abort fgets() */ } static void hexdump (FILE *fp, const char *name, const void *ptr, size_t len) { const char *p = (const char *)ptr; int of = 0; if (name) fprintf(fp, "%s hexdump (%zd bytes):\n", name, len); for (of = 0 ; of < len ; of += 16) { char hexen[16*3+1]; char charen[16+1]; int hof = 0; int cof = 0; int i; for (i = of ; i < of + 16 && i < len ; i++) { hof += sprintf(hexen+hof, "%02x ", p[i] & 0xff); cof += sprintf(charen+cof, "%c", isprint((int)p[i]) ? p[i] : '.'); } fprintf(fp, "%08x: %-48s %-16s\n", of, hexen, charen); } } /** * Kafka logger callback (optional) */ static void logger (const rd_kafka_t *rk, int level, const char *fac, const char *buf) { struct timeval tv; gettimeofday(&tv, NULL); fprintf(stderr, "%u.%03u RDKAFKA-%i-%s: %s: %s\n", (int)tv.tv_sec, (int)(tv.tv_usec / 1000), level, fac, rd_kafka_name(rk), buf); } /** * Message delivery report callback. * Called once for each message. * See rdkafka.h for more information. */ static void msg_delivered (rd_kafka_t *rk, void *payload, size_t len, int error_code, void *opaque, void *msg_opaque) { if (error_code) fprintf(stderr, "%% Message delivery failed: %s\n", rd_kafka_err2str(error_code)); else if (!quiet) fprintf(stderr, "%% Message delivered (%zd bytes)\n", len); } static void msg_consume (rd_kafka_message_t *rkmessage, void *opaque) { if (rkmessage->err) { if (rkmessage->err == RD_KAFKA_RESP_ERR__PARTITION_EOF) { fprintf(stderr, "%% Consumer reached end of %s [%"PRId32"] " "message queue at offset %"PRId64"\n", rd_kafka_topic_name(rkmessage->rkt), rkmessage->partition, rkmessage->offset); if (exit_eof) run = 0; return; } fprintf(stderr, "%% Consume error for topic \"%s\" [%"PRId32"] " "offset %"PRId64": %s\n", rd_kafka_topic_name(rkmessage->rkt), rkmessage->partition, rkmessage->offset, rd_kafka_message_errstr(rkmessage)); return; } if (!quiet) fprintf(stdout, "%% Message (offset %"PRId64", %zd bytes):\n", rkmessage->offset, rkmessage->len); if (rkmessage->key_len) { if (output == OUTPUT_HEXDUMP) hexdump(stdout, "Message Key", rkmessage->key, rkmessage->key_len); else printf("Key: %.*s\n", (int)rkmessage->key_len, (char *)rkmessage->key); } if (output == OUTPUT_HEXDUMP) hexdump(stdout, "Message Payload", rkmessage->payload, rkmessage->len); else printf("%.*s\n", (int)rkmessage->len, (char *)rkmessage->payload); } static void sig_usr1 (int sig) { rd_kafka_dump(stdout, rk); } int main (int argc, char **argv) { rd_kafka_topic_t *rkt; char *brokers = "localhost:9092"; char mode = 'C'; char *topic = NULL; int partition = RD_KAFKA_PARTITION_UA; int opt; rd_kafka_conf_t *conf; rd_kafka_topic_conf_t *topic_conf; char errstr[512]; const char *debug = NULL; int64_t start_offset = 0; int do_conf_dump = 0; quiet = !isatty(STDIN_FILENO); /* Kafka configuration */ conf = rd_kafka_conf_new(); /* Topic configuration */ topic_conf = rd_kafka_topic_conf_new(); while ((opt = getopt(argc, argv, "PCt:p:b:z:qd:o:eX:A")) != -1) { switch (opt) { case 'P': case 'C': mode = opt; break; case 't': topic = optarg; break; case 'p': partition = atoi(optarg); break; case 'b': brokers = optarg; break; case 'z': if (rd_kafka_conf_set(conf, "compression.codec", optarg, errstr, sizeof(errstr)) != RD_KAFKA_CONF_OK) { fprintf(stderr, "%% %s\n", errstr); exit(1); } break; case 'o': if (!strcmp(optarg, "end")) start_offset = RD_KAFKA_OFFSET_END; else if (!strcmp(optarg, "beginning")) start_offset = RD_KAFKA_OFFSET_BEGINNING; else if (!strcmp(optarg, "stored")) start_offset = RD_KAFKA_OFFSET_STORED; else start_offset = strtoll(optarg, NULL, 10); break; case 'e': exit_eof = 1; break; case 'd': debug = optarg; break; case 'q': quiet = 1; break; case 'A': output = OUTPUT_RAW; break; case 'X': { char *name, *val; rd_kafka_conf_res_t res; if (!strcmp(optarg, "list") || !strcmp(optarg, "help")) { rd_kafka_conf_properties_show(stdout); exit(0); } if (!strcmp(optarg, "dump")) { do_conf_dump = 1; continue; } name = optarg; if (!(val = strchr(name, '='))) { fprintf(stderr, "%% Expected " "-X property=value, not %s\n", name); exit(1); } *val = '\0'; val++; res = RD_KAFKA_CONF_UNKNOWN; /* Try "topic." prefixed properties on topic * conf first, and then fall through to global if * it didnt match a topic configuration property. */ if (!strncmp(name, "topic.", strlen("topic."))) res = rd_kafka_topic_conf_set(topic_conf, name+ strlen("topic."), val, errstr, sizeof(errstr)); if (res == RD_KAFKA_CONF_UNKNOWN) res = rd_kafka_conf_set(conf, name, val, errstr, sizeof(errstr)); if (res != RD_KAFKA_CONF_OK) { fprintf(stderr, "%% %s\n", errstr); exit(1); } } break; default: goto usage; } } if (do_conf_dump) { const char **arr; size_t cnt; int pass; for (pass = 0 ; pass < 2 ; pass++) { int i; if (pass == 0) { arr = rd_kafka_conf_dump(conf, &cnt); printf("# Global config\n"); } else { printf("# Topic config\n"); arr = rd_kafka_topic_conf_dump(topic_conf, &cnt); } for (i = 0 ; i < cnt ; i += 2) printf("%s = %s\n", arr[i], arr[i+1]); printf("\n"); rd_kafka_conf_dump_free(arr, cnt); } exit(0); } if (!topic || optind != argc) { usage: fprintf(stderr, "Usage: %s [-C|-P] -t " "[-p ] [-b ]\n" "\n" "librdkafka version %s (0x%08x)\n" "\n" " Options:\n" " -C | -P Consumer or Producer mode\n" " -t Topic to fetch / produce\n" " -p Partition (random partitioner)\n" " -b Broker address (localhost:9092)\n" " -z Enable compression:\n" " none|gzip|snappy\n" " -o Start offset (consumer)\n" " -e Exit consumer when last message\n" " in partition has been received.\n" " -d [facs..] Enable debugging contexts:\n" " -q Be quiet\n" " -A Raw payload output (consumer)\n" " %s\n" " -X Set arbitrary librdkafka " "configuration property\n" " Properties prefixed with \"topic.\" " "will be set on topic object.\n" " Use '-X list' to see the full list\n" " of supported properties.\n" "\n" " In Consumer mode:\n" " writes fetched messages to stdout\n" " In Producer mode:\n" " reads messages from stdin and sends to broker\n" "\n" "\n" "\n", argv[0], rd_kafka_version_str(), rd_kafka_version(), RD_KAFKA_DEBUG_CONTEXTS); exit(1); } signal(SIGINT, stop); signal(SIGUSR1, sig_usr1); if (debug && rd_kafka_conf_set(conf, "debug", debug, errstr, sizeof(errstr)) != RD_KAFKA_CONF_OK) { fprintf(stderr, "%% Debug configuration failed: %s: %s\n", errstr, debug); exit(1); } if (mode == 'P') { /* * Producer */ char buf[2048]; int sendcnt = 0; /* Set up a message delivery report callback. * It will be called once for each message, either on successful * delivery to broker, or upon failure to deliver to broker. */ rd_kafka_conf_set_dr_cb(conf, msg_delivered); /* Create Kafka handle */ if (!(rk = rd_kafka_new(RD_KAFKA_PRODUCER, conf, errstr, sizeof(errstr)))) { fprintf(stderr, "%% Failed to create new producer: %s\n", errstr); exit(1); } /* Set logger */ rd_kafka_set_logger(rk, logger); rd_kafka_set_log_level(rk, LOG_DEBUG); /* Add brokers */ if (rd_kafka_brokers_add(rk, brokers) == 0) { fprintf(stderr, "%% No valid brokers specified\n"); exit(1); } /* Create topic */ rkt = rd_kafka_topic_new(rk, topic, topic_conf); if (!quiet) fprintf(stderr, "%% Type stuff and hit enter to send\n"); while (run && fgets(buf, sizeof(buf), stdin)) { size_t len = strlen(buf); if (buf[len-1] == '\n') buf[--len] = '\0'; /* Send/Produce message. */ if (rd_kafka_produce(rkt, partition, RD_KAFKA_MSG_F_COPY, /* Payload and length */ buf, len, /* Optional key and its length */ NULL, 0, /* Message opaque, provided in * delivery report callback as * msg_opaque. */ NULL) == -1) { fprintf(stderr, "%% Failed to produce to topic %s " "partition %i: %s\n", rd_kafka_topic_name(rkt), partition, rd_kafka_err2str( rd_kafka_errno2err(errno))); /* Poll to handle delivery reports */ rd_kafka_poll(rk, 0); continue; } if (!quiet) fprintf(stderr, "%% Sent %zd bytes to topic " "%s partition %i\n", len, rd_kafka_topic_name(rkt), partition); sendcnt++; /* Poll to handle delivery reports */ rd_kafka_poll(rk, 0); } /* Poll to handle delivery reports */ rd_kafka_poll(rk, 0); /* Wait for messages to be delivered */ while (run && rd_kafka_outq_len(rk) > 0) rd_kafka_poll(rk, 100); /* Destroy the handle */ rd_kafka_destroy(rk); } else if (mode == 'C') { /* * Consumer */ /* Create Kafka handle */ if (!(rk = rd_kafka_new(RD_KAFKA_CONSUMER, conf, errstr, sizeof(errstr)))) { fprintf(stderr, "%% Failed to create new consumer: %s\n", errstr); exit(1); } /* Set logger */ rd_kafka_set_logger(rk, logger); rd_kafka_set_log_level(rk, LOG_DEBUG); /* Add brokers */ if (rd_kafka_brokers_add(rk, brokers) == 0) { fprintf(stderr, "%% No valid brokers specified\n"); exit(1); } /* Create topic */ rkt = rd_kafka_topic_new(rk, topic, topic_conf); /* Start consuming */ if (rd_kafka_consume_start(rkt, partition, start_offset) == -1){ fprintf(stderr, "%% Failed to start consuming: %s\n", rd_kafka_err2str(rd_kafka_errno2err(errno))); exit(1); } while (run) { rd_kafka_message_t *rkmessage; /* Consume single message. * See rdkafka_performance.c for high speed * consuming of messages. */ rkmessage = rd_kafka_consume(rkt, partition, 1000); if (!rkmessage) /* timeout */ continue; msg_consume(rkmessage, NULL); /* Return message to rdkafka */ rd_kafka_message_destroy(rkmessage); } /* Stop consuming */ rd_kafka_consume_stop(rkt, partition); rd_kafka_topic_destroy(rkt); rd_kafka_destroy(rk); } /* Let background threads clean up and terminate cleanly. */ rd_kafka_wait_destroyed(2000); return 0; } librdkafka-0.8.3/examples/rdkafka_performance.c000066400000000000000000000464271227620010100215660ustar00rootroot00000000000000/* * librdkafka - Apache Kafka C library * * Copyright (c) 2012, Magnus Edenhill * All rights reserved. * * Redistribution and use in source and binary forms, with or without * modification, are permitted provided that the following conditions are met: * * 1. Redistributions of source code must retain the above copyright notice, * this list of conditions and the following disclaimer. * 2. Redistributions in binary form must reproduce the above copyright notice, * this list of conditions and the following disclaimer in the documentation * and/or other materials provided with the distribution. * * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE * POSSIBILITY OF SUCH DAMAGE. */ /** * Apache Kafka consumer & producer performance tester * using the Kafka driver from librdkafka * (https://github.com/edenhill/librdkafka) */ #include #include #include #include /* Typical include path would be , but this program * is built from within the librdkafka source tree and thus differs. */ #include "rdkafka.h" /* for Kafka driver */ /* Do not include these defines from your program, they will not be * provided by librdkafka. */ #include "rd.h" #include "rdtime.h" static int run = 1; static int forever = 1; static int dispintvl = 1000; static int do_seq = 0; static int exit_after = 0; static int exit_eof = 0; static int quiet = 0; static void stop (int sig) { run = 0; } static long int msgs_wait_cnt = 0; static int msgs_failed = 0; static rd_ts_t t_end; static rd_kafka_t *rk; static struct { rd_ts_t t_start; rd_ts_t t_end; rd_ts_t t_end_send; uint64_t msgs; uint64_t bytes; uint64_t tx; uint64_t tx_err; rd_ts_t t_latency; rd_ts_t t_last; rd_ts_t t_total; } cnt = {}; static void err_cb (rd_kafka_t *rk, int err, const char *reason, void *opaque) { printf("ERROR CALLBACK: %s: %s: %s\n", rd_kafka_name(rk), rd_kafka_err2str(err), reason); } static void msg_delivered (rd_kafka_t *rk, void *payload, size_t len, int error_code, void *opaque, void *msg_opaque) { static rd_ts_t last; rd_ts_t now = rd_clock(); static int msgs; msgs++; msgs_wait_cnt--; if (error_code) msgs_failed++; if ((error_code && (msgs_failed < 50 || !(msgs_failed % (dispintvl / 1000)))) || !last || msgs_wait_cnt < 5 || !(msgs_wait_cnt % (dispintvl / 1000)) || (now - last) >= dispintvl * 1000) { if (error_code) printf("Message delivery failed: %s (%li remain)\n", rd_kafka_err2str(error_code), msgs_wait_cnt); else if (!quiet) printf("Message delivered: %li remain\n", msgs_wait_cnt); if (!quiet && do_seq) printf(" --> \"%.*s\"\n", (int)len, (char *)payload); last = now; } if (msgs_wait_cnt == 0 && !forever) { if (!quiet) printf("All messages delivered!\n"); t_end = rd_clock(); run = 0; } if (exit_after && exit_after <= msgs) { printf("%% Hard exit after %i messages, as requested\n", exit_after); exit(0); } } static void msg_consume (rd_kafka_message_t *rkmessage, void *opaque) { if (rkmessage->err) { if (rkmessage->err == RD_KAFKA_RESP_ERR__PARTITION_EOF) { printf("%% Consumer reached end of %s [%"PRId32"] " "message queue at offset %"PRId64"\n", rd_kafka_topic_name(rkmessage->rkt), rkmessage->partition, rkmessage->offset); if (exit_eof) run = 0; return; } printf("%% Consume error for topic \"%s\" [%"PRId32"] " "offset %"PRId64": %s\n", rd_kafka_topic_name(rkmessage->rkt), rkmessage->partition, rkmessage->offset, rd_kafka_message_errstr(rkmessage)); msgs_failed++; return; } cnt.msgs++; cnt.bytes += rkmessage->len; if (!(cnt.msgs % 1000000)) printf("@%"PRId64": %.*s\n", rkmessage->offset, (int)rkmessage->len, (char *)rkmessage->payload); #if 0 /* Future API */ /* We store offset when we're done processing * the current message. */ rd_kafka_offset_store(rkmessage->rkt, rkmessage->partition, rd_kafka_offset_next(rkmessage)); #endif } static int stats_cb (rd_kafka_t *rk, char *json, size_t json_len, void *opaque) { printf("%s\n", json); return 0; } static void print_stats (int mode, int force_show, const char *compression) { rd_ts_t now = rd_clock(); rd_ts_t t_total; if (!force_show && cnt.t_last + dispintvl > now) return; if (cnt.t_end_send) t_total = cnt.t_end_send - cnt.t_start; else if (cnt.t_end) t_total = cnt.t_end - cnt.t_start; else t_total = now - cnt.t_start; printf("%% %"PRIu64" messages and %"PRIu64" bytes " "%s in %"PRIu64"ms: %"PRIu64" msgs/s and %.02f Mb/s, " "%i messages failed, %s compression\n", cnt.msgs, cnt.bytes, mode == 'P' ? "produced" : "consumed", t_total / 1000, ((cnt.msgs * 1000000) / t_total), (float)((cnt.bytes) / (float)t_total), msgs_failed, compression); cnt.t_last = now; } static void sig_usr1 (int sig) { rd_kafka_dump(stdout, rk); } int main (int argc, char **argv) { char *brokers = "localhost"; char mode = 'C'; char *topic = NULL; const char *key = NULL; int partition = RD_KAFKA_PARTITION_UA; /* random */ int opt; int msgcnt = -1; int sendflags = 0; char *msgpattern = "librdkafka_performance testing!"; int msgsize = strlen(msgpattern); const char *debug = NULL; rd_ts_t now; char errstr[512]; uint64_t seq = 0; int seed = time(NULL); rd_kafka_topic_t *rkt; rd_kafka_conf_t *conf; rd_kafka_topic_conf_t *topic_conf; const char *compression = "no"; int64_t start_offset = 0; int batch_size = 0; int idle = 0; /* Kafka configuration */ conf = rd_kafka_conf_new(); rd_kafka_conf_set_error_cb(conf, err_cb); rd_kafka_conf_set_dr_cb(conf, msg_delivered); /* Producer config */ rd_kafka_conf_set(conf, "queue.buffering.max.messages", "500000", NULL, 0); rd_kafka_conf_set(conf, "message.send.max.retries", "3", NULL, 0); rd_kafka_conf_set(conf, "retry.backoff.ms", "500", NULL, 0); /* Consumer config */ /* Tell rdkafka to (try to) maintain 1M messages * in its internal receive buffers. This is to avoid * application -> rdkafka -> broker per-message ping-pong * latency. * The larger the local queue, the higher the performance. * Try other values with: ... -X queued.min.messages=1000 */ rd_kafka_conf_set(conf, "queued.min.messages", "1000000", NULL, 0); /* Kafka topic configuration */ topic_conf = rd_kafka_topic_conf_new(); rd_kafka_topic_conf_set(topic_conf, "message.timeout.ms", "5000", NULL, 0); while ((opt = getopt(argc, argv, "PCt:p:b:s:k:c:fi:Dd:m:S:x:R:a:z:o:X:B:eT:qI")) != -1) { switch (opt) { case 'P': case 'C': mode = opt; break; case 't': topic = optarg; break; case 'p': partition = atoi(optarg); break; case 'b': brokers = optarg; break; case 's': msgsize = atoi(optarg); break; case 'k': key = optarg; break; case 'c': msgcnt = atoi(optarg); break; case 'D': sendflags |= RD_KAFKA_MSG_F_FREE; break; case 'i': dispintvl = atoi(optarg); break; case 'm': msgpattern = optarg; break; case 'S': seq = strtoull(optarg, NULL, 10); do_seq = 1; break; case 'x': exit_after = atoi(optarg); break; case 'R': seed = atoi(optarg); break; case 'a': if (rd_kafka_topic_conf_set(topic_conf, "request.required.acks", optarg, errstr, sizeof(errstr)) != RD_KAFKA_CONF_OK) { fprintf(stderr, "%% %s\n", errstr); exit(1); } break; case 'B': batch_size = atoi(optarg); break; case 'z': if (rd_kafka_conf_set(conf, "compression.codec", optarg, errstr, sizeof(errstr)) != RD_KAFKA_CONF_OK) { fprintf(stderr, "%% %s\n", errstr); exit(1); } compression = optarg; break; case 'o': if (!strcmp(optarg, "end")) start_offset = RD_KAFKA_OFFSET_END; else if (!strcmp(optarg, "beginning")) start_offset = RD_KAFKA_OFFSET_BEGINNING; else if (!strcmp(optarg, "stored")) start_offset = RD_KAFKA_OFFSET_STORED; else start_offset = strtoll(optarg, NULL, 10); break; case 'e': exit_eof = 1; break; case 'd': debug = optarg; break; case 'X': { char *name, *val; rd_kafka_conf_res_t res; if (!strcmp(optarg, "list") || !strcmp(optarg, "help")) { rd_kafka_conf_properties_show(stdout); exit(0); } name = optarg; if (!(val = strchr(name, '='))) { fprintf(stderr, "%% Expected " "-X property=value, not %s\n", name); exit(1); } *val = '\0'; val++; res = RD_KAFKA_CONF_UNKNOWN; /* Try "topic." prefixed properties on topic * conf first, and then fall through to global if * it didnt match a topic configuration property. */ if (!strncmp(name, "topic.", strlen("topic."))) res = rd_kafka_topic_conf_set(topic_conf, name+ strlen("topic."), val, errstr, sizeof(errstr)); if (res == RD_KAFKA_CONF_UNKNOWN) res = rd_kafka_conf_set(conf, name, val, errstr, sizeof(errstr)); if (res != RD_KAFKA_CONF_OK) { fprintf(stderr, "%% %s\n", errstr); exit(1); } } break; case 'T': if (rd_kafka_conf_set(conf, "statistics.interval.ms", optarg, errstr, sizeof(errstr)) != RD_KAFKA_CONF_OK) { fprintf(stderr, "%% %s\n", errstr); exit(1); } rd_kafka_conf_set_stats_cb(conf, stats_cb); break; case 'q': quiet = 1; break; case 'I': idle = 1; break; default: goto usage; } } if (!topic || optind != argc) { usage: fprintf(stderr, "Usage: %s [-C|-P] -t " "[-p ] [-b ] [options..]\n" "\n" "librdkafka version %s (0x%08x)\n" "\n" " Options:\n" " -C | -P Consumer or Producer mode\n" " -t Topic to fetch / produce\n" " -p Partition (defaults to random)\n" " -b Broker address list (host[:port],..)\n" " -s Message size (producer)\n" " -k Message key (producer)\n" " -c Messages to transmit/receive\n" " -D Copy/Duplicate data buffer (producer)\n" " -i Display interval\n" " -m Message payload pattern\n" " -S Send a sequence number starting at " " as payload\n" " -R Random seed value (defaults to time)\n" " -a Required acks (producer): " "-1, 0, 1, >1\n" " -B Consume batch size (# of msgs)\n" " -z Enable compression:\n" " none|gzip|snappy\n" " -o Start offset (consumer)\n" " -d [facs..] Enable debugging contexts:\n" " %s\n" " -X Set arbitrary librdkafka " "configuration property\n" " Properties prefixed with \"topic.\" " "will be set on topic object.\n" " Use '-X list' to see the full list\n" " of supported properties.\n" " -T Enable statistics from librdkafka at " "specified interval (ms)\n" " -q Be more quiet\n" " -I Idle: dont produce any messages\n" "\n" " In Consumer mode:\n" " consumes messages and prints thruput\n" " If -B <..> is supplied the batch consumer\n" " mode is used, else the callback mode is used.\n" "\n" " In Producer mode:\n" " writes messages of size -s <..> and prints thruput\n" "\n", argv[0], rd_kafka_version_str(), rd_kafka_version(), RD_KAFKA_DEBUG_CONTEXTS); exit(1); } dispintvl *= 1000; /* us */ printf("%% Using random seed %i\n", seed); srand(seed); signal(SIGINT, stop); signal(SIGUSR1, sig_usr1); if (debug && rd_kafka_conf_set(conf, "debug", debug, errstr, sizeof(errstr)) != RD_KAFKA_CONF_OK) { printf("%% Debug configuration failed: %s: %s\n", errstr, debug); exit(1); } if (msgcnt != -1) forever = 0; if (mode == 'P') { /* * Producer */ char *sbuf; char *pbuf; int outq; int i; int keylen = key ? strlen(key) : 0; off_t rof = 0; size_t plen = strlen(msgpattern); if (do_seq) { if (msgsize < strlen("18446744073709551615: ")+1) msgsize = strlen("18446744073709551615: ")+1; /* Force duplication of payload */ sendflags |= RD_KAFKA_MSG_F_FREE; } sbuf = malloc(msgsize); /* Copy payload content to new buffer */ while (rof < msgsize) { size_t xlen = RD_MIN(msgsize-rof, plen); memcpy(sbuf+rof, msgpattern, xlen); rof += xlen; } if (msgcnt == -1) printf("%% Sending messages of size %i bytes\n", msgsize); else printf("%% Sending %i messages of size %i bytes\n", msgcnt, msgsize); /* Create Kafka handle */ if (!(rk = rd_kafka_new(RD_KAFKA_PRODUCER, conf, errstr, sizeof(errstr)))) { fprintf(stderr, "%% Failed to create Kafka producer: %s\n", errstr); exit(1); } if (debug) rd_kafka_set_log_level(rk, 7); /* Add broker(s) */ if (rd_kafka_brokers_add(rk, brokers) < 1) { fprintf(stderr, "%% No valid brokers specified\n"); exit(1); } /* Explicitly create topic to avoid per-msg lookups. */ rkt = rd_kafka_topic_new(rk, topic, topic_conf); cnt.t_start = rd_clock(); while (run && (msgcnt == -1 || cnt.msgs < msgcnt)) { /* Send/Produce message. */ if (idle) { rd_kafka_poll(rk, 1000); continue; } if (do_seq) { snprintf(sbuf, msgsize-1, "%"PRIu64": ", seq); seq++; } if (sendflags & RD_KAFKA_MSG_F_FREE) { /* Duplicate memory */ pbuf = malloc(msgsize); memcpy(pbuf, sbuf, msgsize); } else pbuf = sbuf; cnt.tx++; while (run && rd_kafka_produce(rkt, partition, sendflags, pbuf, msgsize, key, keylen, NULL) == -1) { if (errno == ESRCH) printf("No such partition: %"PRId32"\n", partition); else if (!quiet || errno != ENOBUFS) printf("produce error: %s%s\n", rd_kafka_err2str( rd_kafka_errno2err( errno)), errno == ENOBUFS ? " (backpressure)":""); msgs_failed++; cnt.tx_err++; if (errno != ENOBUFS) { run = 0; break; } now = rd_clock(); if (cnt.t_last + dispintvl <= now) { printf("%% Backpressure %i " "(tx %"PRIu64", " "txerr %"PRIu64")\n", rd_kafka_outq_len(rk), cnt.tx, cnt.tx_err); cnt.t_last = now; } /* Poll to handle delivery reports */ rd_kafka_poll(rk, 10); } msgs_wait_cnt++; cnt.msgs++; cnt.bytes += msgsize; print_stats(mode, 0, compression); /* Must poll to handle delivery reports */ rd_kafka_poll(rk, 0); } forever = 0; printf("All messages produced, " "now waiting for %li deliveries\n", msgs_wait_cnt); if (debug) rd_kafka_dump(stdout, rk); /* Wait for messages to be delivered */ i = 0; while (run && rd_kafka_poll(rk, 1000) != -1) { if (!(i++ % (dispintvl/1000))) printf("%% Waiting for %li, " "%i messages in outq " "to be sent. Abort with Ctrl-c\n", msgs_wait_cnt, rd_kafka_outq_len(rk)); } outq = rd_kafka_outq_len(rk); printf("%% %i messages in outq\n", outq); cnt.msgs -= outq; cnt.bytes -= msgsize * outq; cnt.t_end = t_end; if (cnt.tx_err > 0) printf("%% %"PRIu64" backpressures for %"PRIu64 " produce calls: %.3f%% backpressure rate\n", cnt.tx_err, cnt.tx, ((double)cnt.tx_err / (double)cnt.tx) * 100.0); if (debug) rd_kafka_dump(stdout, rk); /* Destroy the handle */ rd_kafka_destroy(rk); } else if (mode == 'C') { /* * Consumer */ rd_kafka_message_t **rkmessages = NULL; #if 0 /* Future API */ /* The offset storage file is optional but its presence * avoids starting all over from offset 0 again when * the program restarts. * ZooKeeper functionality will be implemented in future * versions and then the offset will be stored there instead. */ conf.consumer.offset_file = "."; /* current directory */ /* Indicate to rdkafka that the application is responsible * for storing the offset. This allows the application to * successfully handle a message before storing the offset. * If this flag is not set rdkafka will store the offset * just prior to returning the message from rd_kafka_consume(). */ conf.flags |= RD_KAFKA_CONF_F_APP_OFFSET_STORE; #endif /* Create Kafka handle */ if (!(rk = rd_kafka_new(RD_KAFKA_CONSUMER, conf, errstr, sizeof(errstr)))) { fprintf(stderr, "%% Failed to create Kafka consumer: %s\n", errstr); exit(1); } if (debug) rd_kafka_set_log_level(rk, 7); /* Add broker(s) */ if (rd_kafka_brokers_add(rk, brokers) < 1) { fprintf(stderr, "%% No valid brokers specified\n"); exit(1); } /* Create topic to consume from */ rkt = rd_kafka_topic_new(rk, topic, topic_conf); /* Batch consumer */ if (batch_size) rkmessages = malloc(sizeof(*rkmessages) * batch_size); /* Start consuming */ if (rd_kafka_consume_start(rkt, partition, start_offset) == -1){ fprintf(stderr, "%% Failed to start consuming: %s\n", rd_kafka_err2str(rd_kafka_errno2err(errno))); exit(1); } cnt.t_start = rd_clock(); while (run && (msgcnt == -1 || msgcnt > cnt.msgs)) { /* Consume messages. * A message may either be a real message, or * an error signaling (if rkmessage->err is set). */ uint64_t latency; int r; latency = rd_clock(); if (batch_size) { int i; /* Batch fetch mode */ r = rd_kafka_consume_batch(rkt, partition, 1000, rkmessages, batch_size); if (r != -1) { for (i = 0 ; i < r ; i++) { msg_consume(rkmessages[i],NULL); rd_kafka_message_destroy( rkmessages[i]); } } } else { /* Callback mode */ r = rd_kafka_consume_callback(rkt, partition, 1000/*timeout*/, msg_consume, NULL); } cnt.t_latency += rd_clock() - latency; if (r == -1) fprintf(stderr, "%% Error: %s\n", rd_kafka_err2str( rd_kafka_errno2err(errno))); print_stats(mode, 0, compression); /* Poll to handle stats callbacks */ rd_kafka_poll(rk, 0); } cnt.t_end = rd_clock(); /* Stop consuming */ rd_kafka_consume_stop(rkt, partition); /* Destroy topic */ rd_kafka_topic_destroy(rkt); if (batch_size) free(rkmessages); /* Destroy the handle */ rd_kafka_destroy(rk); } print_stats(mode, 1, compression); if (cnt.t_latency && cnt.msgs) printf("%% Average application fetch latency: %"PRIu64"us\n", cnt.t_latency / cnt.msgs); /* Let background threads clean up and terminate cleanly. */ rd_kafka_wait_destroyed(2000); return 0; } librdkafka-0.8.3/lds-gen.pl000077500000000000000000000006671227620010100155050ustar00rootroot00000000000000#!/usr/bin/env perl # # # Generate linker script to only expose symbols of the public API # open(IN, ") { push(@funcs, $2) if /^(\S+.*\s+\**)?(rd_kafka_\S+)\s+\(/; } close(IN); print "# Automatically generated by lds-gen.pl - DO NOT EDIT\n"; print "{\n global:\n"; foreach my $f (sort @funcs) { print " $f;\n"; } print " local:\n *;\n};\n"; librdkafka-0.8.3/rd.c000066400000000000000000000030511227620010100143530ustar00rootroot00000000000000/* * librd - Rapid Development C library * * Copyright (c) 2012, Magnus Edenhill * All rights reserved. * * Redistribution and use in source and binary forms, with or without * modification, are permitted provided that the following conditions are met: * * 1. Redistributions of source code must retain the above copyright notice, * this list of conditions and the following disclaimer. * 2. Redistributions in binary form must reproduce the above copyright notice, * this list of conditions and the following disclaimer in the documentation * and/or other materials provided with the distribution. * * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE * POSSIBILITY OF SUCH DAMAGE. */ #include "rd.h" void rd_init (void) { extern void rd_thread_init (void); extern void rd_timers_init(); rd_thread_init(); rd_timers_init(); } librdkafka-0.8.3/rd.h000066400000000000000000000063641227620010100143720ustar00rootroot00000000000000/* * librd - Rapid Development C library * * Copyright (c) 2012, Magnus Edenhill * All rights reserved. * * Redistribution and use in source and binary forms, with or without * modification, are permitted provided that the following conditions are met: * * 1. Redistributions of source code must retain the above copyright notice, * this list of conditions and the following disclaimer. * 2. Redistributions in binary form must reproduce the above copyright notice, * this list of conditions and the following disclaimer in the documentation * and/or other materials provided with the distribution. * * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE * POSSIBILITY OF SUCH DAMAGE. */ #pragma once #include #include #include #include #include #include #include #include #include #include #include "rdtypes.h" #ifndef likely #define likely(x) __builtin_expect((x),1) #endif #ifndef unlikely #define unlikely(x) __builtin_expect((x),0) #endif #define RD_UNUSED __attribute__((unused)) #define RD_PACKED __attribute__((packed)) #define RD_ARRAY_SIZE(A) (sizeof((A)) / sizeof(*(A))) #define RD_ARRAYSIZE(A) RD_ARRAY_SIZE(A) #define RD_SIZEOF(TYPE,MEMBER) sizeof(((TYPE *)NULL)->MEMBER) #define RD_OFFSETOF(TYPE,MEMBER) ((size_t) &(((TYPE *)NULL)->MEMBER)) /** * Returns the 'I'th array element from static sized array 'A' * or NULL if 'I' is out of range. * 'PFX' is an optional prefix to provide the correct return type. */ #define RD_ARRAY_ELEM(A,I,PFX...) \ ((unsigned int)(I) < RD_ARRAY_SIZE(A) ? PFX (A)[(I)] : NULL) #define RD_STRINGIFY(X) # X #define RD_MIN(a,b) ((a) < (b) ? (a) : (b)) #define RD_MAX(a,b) ((a) > (b) ? (a) : (b)) /** * Cap an integer (of any type) to reside within the defined limit. */ #define RD_INT_CAP(val,low,hi) \ ((val) < (low) ? low : ((val) > (hi) ? (hi) : (val))) #define rd_atomic_add(PTR,VAL) __sync_add_and_fetch(PTR,VAL) #define rd_atomic_sub(PTR,VAL) __sync_sub_and_fetch(PTR,VAL) #define rd_atomic_add_prev(PTR,VAL) __sync_fetch_and_add(PTR,VAL) #define rd_atomic_sub_prev(PTR,VAL) __sync_fetch_and_sub(PTR,VAL) #define rd_atomic_set(PTR,VAL) __sync_lock_test_and_set(PTR,VAL) #ifndef be64toh #ifndef __APPLE__ #include #if __BYTE_ORDER == __BIG_ENDIAN #define be64toh(x) (x) #else # if __BYTE_ORDER == __LITTLE_ENDIAN #define be64toh(x) bswap_64(x) # endif #endif #define htobe64(x) be64toh(x) #endif #endif void rd_init (void); librdkafka-0.8.3/rdaddr.c000066400000000000000000000121511227620010100152070ustar00rootroot00000000000000/* * librd - Rapid Development C library * * Copyright (c) 2012, Magnus Edenhill * All rights reserved. * * Redistribution and use in source and binary forms, with or without * modification, are permitted provided that the following conditions are met: * * 1. Redistributions of source code must retain the above copyright notice, * this list of conditions and the following disclaimer. * 2. Redistributions in binary form must reproduce the above copyright notice, * this list of conditions and the following disclaimer in the documentation * and/or other materials provided with the distribution. * * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE * POSSIBILITY OF SUCH DAMAGE. */ #include "rd.h" #include "rdaddr.h" #include "rdrand.h" const char *rd_sockaddr2str (const void *addr, int flags) { const rd_sockaddr_inx_t *a = (const rd_sockaddr_inx_t *)addr; static __thread char ret[32][INET6_ADDRSTRLEN + 16]; static __thread int reti = 0; char portstr[64]; int of = 0; int niflags = NI_NUMERICSERV; reti = (reti + 1) % 32; switch (a->sinx_family) { case AF_INET: case AF_INET6: if (flags & RD_SOCKADDR2STR_F_FAMILY) of += sprintf(&ret[reti][of], "ipv%i#", a->sinx_family == AF_INET ? 4 : 6); if ((flags & RD_SOCKADDR2STR_F_PORT) && a->sinx_family == AF_INET6) ret[reti][of++] = '['; if (!(flags & RD_SOCKADDR2STR_F_RESOLVE)) niflags |= NI_NUMERICHOST; if (getnameinfo((const struct sockaddr *)a, RD_SOCKADDR_INX_LEN(a), ret[reti]+of, sizeof(ret[reti])-of, (flags & RD_SOCKADDR2STR_F_PORT) ? portstr : NULL, (flags & RD_SOCKADDR2STR_F_PORT) ? sizeof(portstr) : 0, niflags)) break; if (flags & RD_SOCKADDR2STR_F_PORT) { int len = strlen(ret[reti]); snprintf(ret[reti]+len, sizeof(ret[reti])-len, "%s:%s", a->sinx_family == AF_INET6 ? "]" : "", portstr); } return ret[reti]; } /* Error-case */ snprintf(ret[reti], sizeof(ret[reti]), "", rd_family2str(a->sinx_family)); return ret[reti]; } const char *rd_addrinfo_prepare (const char *nodesvc, char **node, char **svc) { static __thread char snode[256]; static __thread char ssvc[64]; const char *t; const char *svct = NULL; int nodelen = 0; *snode = '\0'; *ssvc = '\0'; if (*nodesvc == '[') { /* "[host]".. (enveloped node name) */ if (!(t = strchr(nodesvc, ']'))) return "Missing close-']'"; nodesvc++; nodelen = (int)(t-nodesvc); svct = t+1; } else if (*nodesvc == ':' && *(nodesvc+1) != ':') { /* ":".. (port only) */ nodelen = 0; svct = nodesvc; } if ((svct = strrchr(svct ? : nodesvc, ':')) && (*(svct-1) != ':') && *(++svct)) { /* Optional ":service" definition. */ if (strlen(svct) >= sizeof(ssvc)) return "Service name too long"; strcpy(ssvc, svct); if (!nodelen) nodelen = (int)(svct - nodesvc)-1; } else if (!nodelen) nodelen = strlen(nodesvc); if (nodelen) { strncpy(snode, nodesvc, nodelen); snode[nodelen] = '\0'; } *node = snode; *svc = ssvc; return NULL; } rd_sockaddr_list_t *rd_getaddrinfo (const char *nodesvc, const char *defsvc, int flags, int family, int socktype, int protocol, const char **errstr) { struct addrinfo hints = { .ai_family = family, .ai_socktype = socktype, .ai_protocol = protocol, .ai_flags = flags }; struct addrinfo *ais, *ai; char *node, *svc; int r; int cnt = 0; rd_sockaddr_list_t *rsal; if ((*errstr = rd_addrinfo_prepare(nodesvc, &node, &svc))) { errno = EINVAL; return NULL; } if (*svc) defsvc = svc; if ((r = getaddrinfo(node, defsvc, &hints, &ais))) { if (r == EAI_SYSTEM) *errstr = strerror(errno); else { *errstr = gai_strerror(r); errno = EFAULT; } return NULL; } /* Count number of addresses */ for (ai = ais ; ai != NULL ; ai = ai->ai_next) cnt++; if (cnt == 0) { /* unlikely? */ freeaddrinfo(ais); errno = ENOENT; *errstr = "No addresses"; return NULL; } rsal = calloc(1, sizeof(*rsal) + (sizeof(*rsal->rsal_addr) * cnt)); for (ai = ais ; ai != NULL ; ai = ai->ai_next) memcpy(&rsal->rsal_addr[rsal->rsal_cnt++], ai->ai_addr, ai->ai_addrlen); freeaddrinfo(ais); /* Shuffle address list for proper round-robin */ if (!(flags & RD_AI_NOSHUFFLE)) rd_array_shuffle(rsal->rsal_addr, rsal->rsal_cnt, sizeof(*rsal->rsal_addr)); return rsal; } void rd_sockaddr_list_destroy (rd_sockaddr_list_t *rsal) { free(rsal); } librdkafka-0.8.3/rdaddr.h000066400000000000000000000133711227620010100152210ustar00rootroot00000000000000/* * librd - Rapid Development C library * * Copyright (c) 2012, Magnus Edenhill * All rights reserved. * * Redistribution and use in source and binary forms, with or without * modification, are permitted provided that the following conditions are met: * * 1. Redistributions of source code must retain the above copyright notice, * this list of conditions and the following disclaimer. * 2. Redistributions in binary form must reproduce the above copyright notice, * this list of conditions and the following disclaimer in the documentation * and/or other materials provided with the distribution. * * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE * POSSIBILITY OF SUCH DAMAGE. */ #pragma once #include #include #include /** * rd_sockaddr_inx_t is a union for either ipv4 or ipv6 sockaddrs. * It provides conveniant abstraction of AF_INET* agnostic operations. */ typedef union { struct sockaddr_in in; struct sockaddr_in6 in6; } rd_sockaddr_inx_t; #define sinx_family in.sin_family #define sinx_addr in.sin_addr #define RD_SOCKADDR_INX_LEN(sinx) \ ((sinx)->sinx_family == AF_INET ? sizeof(struct sockaddr_in) : \ (sinx)->sinx_family == AF_INET6 ? sizeof(struct sockaddr_in6): \ sizeof(rd_sockaddr_inx_t)) #define RD_SOCKADDR_INX_PORT(sinx) \ ((sinx)->sinx_family == AF_INET ? (sinx)->in.sin_port : \ (sinx)->sinx_family == AF_INET6 ? (sinx)->in6.sin6_port : 0) #define RD_SOCKADDR_INX_PORT_SET(sinx,port) do { \ if ((sinx)->sinx_family == AF_INET) \ (sinx)->in.sin_port = port; \ else if ((sinx)->sinx_family == AF_INET6) \ (sinx)->in6.sin6_port = port; \ } while (0) /** * Returns a thread-local temporary string (may be called up to 32 times * without buffer wrapping) containing the human string representation * of the sockaddr (which should be AF_INET or AF_INET6 at this point). * If the RD_SOCKADDR2STR_F_PORT is provided the port number will be * appended to the string. * IPv6 address enveloping ("[addr]:port") will also be performed * if .._F_PORT is set. */ #define RD_SOCKADDR2STR_F_PORT 0x1 /* Append the port. */ #define RD_SOCKADDR2STR_F_RESOLVE 0x2 /* Try to resolve address to hostname. */ #define RD_SOCKADDR2STR_F_FAMILY 0x4 /* Prepend address family. */ #define RD_SOCKADDR2STR_F_NICE /* Nice and friendly output */ \ (RD_SOCKADDR2STR_F_PORT | RD_SOCKADDR2STR_F_RESOLVE) const char *rd_sockaddr2str (const void *addr, int flags); /** * Splits a node:service definition up into their node and svc counterparts * suitable for passing to getaddrinfo(). * Returns NULL on success (and temporarily available pointers in '*node' * and '*svc') or error string on failure. * * Thread-safe but returned buffers in '*node' and '*svc' are only * usable until the next call to rd_addrinfo_prepare() in the same thread. */ const char *rd_addrinfo_prepare (const char *nodesvc, char **node, char **svc); typedef struct rd_sockaddr_list_s { int rsal_cnt; int rsal_curr; rd_sockaddr_inx_t rsal_addr[0]; } rd_sockaddr_list_t; /** * Returns the next address from a sockaddr list and updates * the current-index to point to it. * * Typical usage is for round-robin connection attempts or similar: * while (1) { * rd_sockaddr_inx_t *sinx = rd_sockaddr_list_next(my_server_list); * if (do_connect((struct sockaddr *)sinx) == -1) { * sleep(1); * continue; * } * ... * } * */ static inline rd_sockaddr_inx_t * rd_sockaddr_list_next (rd_sockaddr_list_t *rsal) RD_UNUSED; static inline rd_sockaddr_inx_t * rd_sockaddr_list_next (rd_sockaddr_list_t *rsal) { rsal->rsal_curr = (rsal->rsal_curr + 1) % rsal->rsal_cnt; return &rsal->rsal_addr[rsal->rsal_curr]; } #define RD_SOCKADDR_LIST_FOREACH(sinx, rsal) \ for ((sinx) = &(rsal)->rsal_addr[0] ; \ (sinx) < &(rsal)->rsal_addr[(rsal)->rsal_len] ; \ (sinx)++) /** * Wrapper for getaddrinfo(3) that performs these additional tasks: * - Input is a combined "[:]" string, with support for * IPv6 enveloping ("[addr]:port"). * - Returns a rd_sockaddr_list_t which must be freed with * rd_sockaddr_list_destroy() when done with it. * - Automatically shuffles the returned address list to provide * round-robin (unless RD_AI_NOSHUFFLE is provided in 'flags'). * * Thread-safe. */ #define RD_AI_NOSHUFFLE 0x10000000 /* Dont shuffle returned address list. * FIXME: Guessing non-used bits like this * is a bad idea. */ rd_sockaddr_list_t *rd_getaddrinfo (const char *nodesvc, const char *defsvc, int flags, int family, int socktype, int protocol, const char **errstr); /** * Frees a sockaddr list. * * Thread-safe. */ void rd_sockaddr_list_destroy (rd_sockaddr_list_t *rsal); /** * Returns the human readable name of a socket family. */ static const char *rd_family2str (int af) RD_UNUSED; static const char *rd_family2str (int af) { switch(af){ case AF_LOCAL: return "local"; case AF_INET: return "inet"; case AF_INET6: return "inet6"; default: return "af?"; }; } librdkafka-0.8.3/rdcrc32.c000066400000000000000000000122651227620010100152170ustar00rootroot00000000000000/** * \file rdcrc32.c * Functions and types for CRC checks. * * Generated on Tue May 8 17:37:04 2012, * by pycrc v0.7.10, http://www.tty1.net/pycrc/ * using the configuration: * Width = 32 * Poly = 0x04c11db7 * XorIn = 0xffffffff * ReflectIn = True * XorOut = 0xffffffff * ReflectOut = True * Algorithm = table-driven *****************************************************************************/ #include "rdcrc32.h" /* include the header file generated with pycrc */ #include #include /** * Static table used for the table_driven implementation. *****************************************************************************/ static const rd_crc32_t crc_table[256] = { 0x00000000, 0x77073096, 0xee0e612c, 0x990951ba, 0x076dc419, 0x706af48f, 0xe963a535, 0x9e6495a3, 0x0edb8832, 0x79dcb8a4, 0xe0d5e91e, 0x97d2d988, 0x09b64c2b, 0x7eb17cbd, 0xe7b82d07, 0x90bf1d91, 0x1db71064, 0x6ab020f2, 0xf3b97148, 0x84be41de, 0x1adad47d, 0x6ddde4eb, 0xf4d4b551, 0x83d385c7, 0x136c9856, 0x646ba8c0, 0xfd62f97a, 0x8a65c9ec, 0x14015c4f, 0x63066cd9, 0xfa0f3d63, 0x8d080df5, 0x3b6e20c8, 0x4c69105e, 0xd56041e4, 0xa2677172, 0x3c03e4d1, 0x4b04d447, 0xd20d85fd, 0xa50ab56b, 0x35b5a8fa, 0x42b2986c, 0xdbbbc9d6, 0xacbcf940, 0x32d86ce3, 0x45df5c75, 0xdcd60dcf, 0xabd13d59, 0x26d930ac, 0x51de003a, 0xc8d75180, 0xbfd06116, 0x21b4f4b5, 0x56b3c423, 0xcfba9599, 0xb8bda50f, 0x2802b89e, 0x5f058808, 0xc60cd9b2, 0xb10be924, 0x2f6f7c87, 0x58684c11, 0xc1611dab, 0xb6662d3d, 0x76dc4190, 0x01db7106, 0x98d220bc, 0xefd5102a, 0x71b18589, 0x06b6b51f, 0x9fbfe4a5, 0xe8b8d433, 0x7807c9a2, 0x0f00f934, 0x9609a88e, 0xe10e9818, 0x7f6a0dbb, 0x086d3d2d, 0x91646c97, 0xe6635c01, 0x6b6b51f4, 0x1c6c6162, 0x856530d8, 0xf262004e, 0x6c0695ed, 0x1b01a57b, 0x8208f4c1, 0xf50fc457, 0x65b0d9c6, 0x12b7e950, 0x8bbeb8ea, 0xfcb9887c, 0x62dd1ddf, 0x15da2d49, 0x8cd37cf3, 0xfbd44c65, 0x4db26158, 0x3ab551ce, 0xa3bc0074, 0xd4bb30e2, 0x4adfa541, 0x3dd895d7, 0xa4d1c46d, 0xd3d6f4fb, 0x4369e96a, 0x346ed9fc, 0xad678846, 0xda60b8d0, 0x44042d73, 0x33031de5, 0xaa0a4c5f, 0xdd0d7cc9, 0x5005713c, 0x270241aa, 0xbe0b1010, 0xc90c2086, 0x5768b525, 0x206f85b3, 0xb966d409, 0xce61e49f, 0x5edef90e, 0x29d9c998, 0xb0d09822, 0xc7d7a8b4, 0x59b33d17, 0x2eb40d81, 0xb7bd5c3b, 0xc0ba6cad, 0xedb88320, 0x9abfb3b6, 0x03b6e20c, 0x74b1d29a, 0xead54739, 0x9dd277af, 0x04db2615, 0x73dc1683, 0xe3630b12, 0x94643b84, 0x0d6d6a3e, 0x7a6a5aa8, 0xe40ecf0b, 0x9309ff9d, 0x0a00ae27, 0x7d079eb1, 0xf00f9344, 0x8708a3d2, 0x1e01f268, 0x6906c2fe, 0xf762575d, 0x806567cb, 0x196c3671, 0x6e6b06e7, 0xfed41b76, 0x89d32be0, 0x10da7a5a, 0x67dd4acc, 0xf9b9df6f, 0x8ebeeff9, 0x17b7be43, 0x60b08ed5, 0xd6d6a3e8, 0xa1d1937e, 0x38d8c2c4, 0x4fdff252, 0xd1bb67f1, 0xa6bc5767, 0x3fb506dd, 0x48b2364b, 0xd80d2bda, 0xaf0a1b4c, 0x36034af6, 0x41047a60, 0xdf60efc3, 0xa867df55, 0x316e8eef, 0x4669be79, 0xcb61b38c, 0xbc66831a, 0x256fd2a0, 0x5268e236, 0xcc0c7795, 0xbb0b4703, 0x220216b9, 0x5505262f, 0xc5ba3bbe, 0xb2bd0b28, 0x2bb45a92, 0x5cb36a04, 0xc2d7ffa7, 0xb5d0cf31, 0x2cd99e8b, 0x5bdeae1d, 0x9b64c2b0, 0xec63f226, 0x756aa39c, 0x026d930a, 0x9c0906a9, 0xeb0e363f, 0x72076785, 0x05005713, 0x95bf4a82, 0xe2b87a14, 0x7bb12bae, 0x0cb61b38, 0x92d28e9b, 0xe5d5be0d, 0x7cdcefb7, 0x0bdbdf21, 0x86d3d2d4, 0xf1d4e242, 0x68ddb3f8, 0x1fda836e, 0x81be16cd, 0xf6b9265b, 0x6fb077e1, 0x18b74777, 0x88085ae6, 0xff0f6a70, 0x66063bca, 0x11010b5c, 0x8f659eff, 0xf862ae69, 0x616bffd3, 0x166ccf45, 0xa00ae278, 0xd70dd2ee, 0x4e048354, 0x3903b3c2, 0xa7672661, 0xd06016f7, 0x4969474d, 0x3e6e77db, 0xaed16a4a, 0xd9d65adc, 0x40df0b66, 0x37d83bf0, 0xa9bcae53, 0xdebb9ec5, 0x47b2cf7f, 0x30b5ffe9, 0xbdbdf21c, 0xcabac28a, 0x53b39330, 0x24b4a3a6, 0xbad03605, 0xcdd70693, 0x54de5729, 0x23d967bf, 0xb3667a2e, 0xc4614ab8, 0x5d681b02, 0x2a6f2b94, 0xb40bbe37, 0xc30c8ea1, 0x5a05df1b, 0x2d02ef8d }; /** * Reflect all bits of a \a data word of \a data_len bytes. * * \param data The data word to be reflected. * \param data_len The width of \a data expressed in number of bits. * \return The reflected data. *****************************************************************************/ rd_crc32_t rd_crc32_reflect(rd_crc32_t data, size_t data_len) { unsigned int i; rd_crc32_t ret; ret = data & 0x01; for (i = 1; i < data_len; i++) { data >>= 1; ret = (ret << 1) | (data & 0x01); } return ret; } /** * Update the crc value with new data. * * \param crc The current crc value. * \param data Pointer to a buffer of \a data_len bytes. * \param data_len Number of bytes in the \a data buffer. * \return The updated crc value. *****************************************************************************/ rd_crc32_t rd_crc32_update(rd_crc32_t crc, const unsigned char *data, size_t data_len) { unsigned int tbl_idx; while (data_len--) { tbl_idx = (crc ^ *data) & 0xff; crc = (crc_table[tbl_idx] ^ (crc >> 8)) & 0xffffffff; data++; } return crc & 0xffffffff; } librdkafka-0.8.3/rdcrc32.h000066400000000000000000000053511227620010100152220ustar00rootroot00000000000000/** * \file rdcrc32.h * Functions and types for CRC checks. * * Generated on Tue May 8 17:36:59 2012, * by pycrc v0.7.10, http://www.tty1.net/pycrc/ * * NOTE: Contains librd modifications: * - rd_crc32() helper. * - __RDCRC32___H__ define (was missing the '32' part). * * using the configuration: * Width = 32 * Poly = 0x04c11db7 * XorIn = 0xffffffff * ReflectIn = True * XorOut = 0xffffffff * ReflectOut = True * Algorithm = table-driven *****************************************************************************/ #ifndef __RDCRC32___H__ #define __RDCRC32___H__ #include #include #ifdef __cplusplus extern "C" { #endif /** * The definition of the used algorithm. *****************************************************************************/ #define CRC_ALGO_TABLE_DRIVEN 1 /** * The type of the CRC values. * * This type must be big enough to contain at least 32 bits. *****************************************************************************/ typedef uint32_t rd_crc32_t; /** * Reflect all bits of a \a data word of \a data_len bytes. * * \param data The data word to be reflected. * \param data_len The width of \a data expressed in number of bits. * \return The reflected data. *****************************************************************************/ rd_crc32_t rd_crc32_reflect(rd_crc32_t data, size_t data_len); /** * Calculate the initial crc value. * * \return The initial crc value. *****************************************************************************/ static inline rd_crc32_t rd_crc32_init(void) { return 0xffffffff; } /** * Update the crc value with new data. * * \param crc The current crc value. * \param data Pointer to a buffer of \a data_len bytes. * \param data_len Number of bytes in the \a data buffer. * \return The updated crc value. *****************************************************************************/ rd_crc32_t rd_crc32_update(rd_crc32_t crc, const unsigned char *data, size_t data_len); /** * Calculate the final crc value. * * \param crc The current crc value. * \return The final crc value. *****************************************************************************/ static inline rd_crc32_t rd_crc32_finalize(rd_crc32_t crc) { return crc ^ 0xffffffff; } /** * Wrapper for performing CRC32 on the provided buffer. */ static inline rd_crc32_t rd_crc32 (const char *data, size_t data_len) { return rd_crc32_finalize(rd_crc32_update(rd_crc32_init(), (const unsigned char *)data, data_len)); } #ifdef __cplusplus } /* closing brace for extern "C" */ #endif #endif /* __RDCRC32___H__ */ librdkafka-0.8.3/rdevent.h000066400000000000000000000066721227620010100154360ustar00rootroot00000000000000/* * librd - Rapid Development C library * * Copyright (c) 2012-2013, Magnus Edenhill * All rights reserved. * * Redistribution and use in source and binary forms, with or without * modification, are permitted provided that the following conditions are met: * * 1. Redistributions of source code must retain the above copyright notice, * this list of conditions and the following disclaimer. * 2. Redistributions in binary form must reproduce the above copyright notice, * this list of conditions and the following disclaimer in the documentation * and/or other materials provided with the distribution. * * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE * POSSIBILITY OF SUCH DAMAGE. */ #pragma once #include "rdthread.h" #define rd_thread_event_f(F) void (F) (void *ptr) typedef struct rd_thread_event_s { rd_thread_event_f(*rte_callback); void *rte_ptr; } rd_thread_event_t; /** * Enqueue event (callback call) on thread 'rdt'. * Requires 'rdt' to call rd_thread_dispatch(). */ static void rd_thread_event_add (rd_thread_t *rdt, rd_thread_event_f(*callback), void *ptr) RD_UNUSED; static void rd_thread_event_add (rd_thread_t *rdt, rd_thread_event_f(*callback), void *ptr) { rd_thread_event_t *rte = malloc(sizeof(*rte)); rte->rte_callback = callback; rte->rte_ptr = ptr; rd_fifoq_add(&rdt->rdt_eventq, rte); } /** * Convenience function to enqueue a call to function 'cb' on thread 'rdt'. * Depending on the value of 'argcnt' (0..4) 'cb' may be one of: * * argcnt | prototype * -------+-------------------------------------------------------------- * 0 | void (*cb) (void) * 1 | void (*cb) (void *arg1) * 2 | void (*cb) (void *arg1, void *arg2) * 3 | void (*cb) (void *arg1, void *arg2, void *arg3) * 4 | void (*cb) (void *arg1, void *arg2, void *arg3, void *arg4) */ void rd_thread_func_call (rd_thread_t *rdt, void *cb, int argcnt, void **args); #define rd_thread_func_call0(rdt,cb) \ rd_thread_func_call(rdt,cb,0,NULL) #define rd_thread_func_call1(rdt,cb,arg1) \ rd_thread_func_call(rdt, cb, 1, ((void *[]){ arg1 })) #define rd_thread_func_call2(rdt,cb,arg1,arg2) \ rd_thread_func_call(rdt, cb, 2, ((void *[]){ arg1, arg2 })) #define rd_thread_func_call3(rdt,cb,arg1,arg2,arg3) \ rd_thread_func_call(rdt, cb, 3, ((void *[]){ arg1, arg2, arg3 })) #define rd_thread_func_call4(rdt,cb,arg1,arg2,arg3,arg4) \ rd_thread_func_call(rdt, cb, 4, ((void *[]){ arg1, arg2, arg3, arg4 })) /** * Calls the callback and destroys the event. */ static void rd_thread_event_call (rd_thread_event_t *rte) RD_UNUSED; static void rd_thread_event_call (rd_thread_event_t *rte) { rte->rte_callback(rte->rte_ptr); free(rte); } librdkafka-0.8.3/rdgz.c000066400000000000000000000064121227620010100147200ustar00rootroot00000000000000/* * librd - Rapid Development C library * * Copyright (c) 2012, Magnus Edenhill * All rights reserved. * * Redistribution and use in source and binary forms, with or without * modification, are permitted provided that the following conditions are met: * * 1. Redistributions of source code must retain the above copyright notice, * this list of conditions and the following disclaimer. * 2. Redistributions in binary form must reproduce the above copyright notice, * this list of conditions and the following disclaimer in the documentation * and/or other materials provided with the distribution. * * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE * POSSIBILITY OF SUCH DAMAGE. */ #include "rd.h" #include "rdgz.h" #include #define RD_GZ_CHUNK 262144 void *rd_gz_decompress (void *compressed, int compressed_len, uint64_t *decompressed_lenp) { int pass = 1; char *decompressed = NULL; /* First pass (1): calculate decompressed size. * (pass-1 is skipped if *decompressed_lenp is * non-zero). * Second pass (2): perform actual decompression. */ if (*decompressed_lenp != 0LLU) pass++; for (; pass <= 2 ; pass++) { z_stream strm = {}; gz_header hdr; char buf[512]; char *p; int len; int r; if ((r = inflateInit2(&strm, 15+32)) != Z_OK) goto fail; strm.next_in = compressed; strm.avail_in = compressed_len; if ((r = inflateGetHeader(&strm, &hdr)) != Z_OK) { inflateEnd(&strm); goto fail; } if (pass == 1) { /* Use dummy output buffer */ p = buf; len = sizeof(buf); } else { /* Use real output buffer */ p = decompressed; len = *decompressed_lenp; } do { strm.next_out = (unsigned char *)p; strm.avail_out = len; r = inflate(&strm, Z_NO_FLUSH); switch (r) { case Z_STREAM_ERROR: case Z_NEED_DICT: case Z_DATA_ERROR: case Z_MEM_ERROR: inflateEnd(&strm); goto fail; } if (pass == 2) { /* Advance output pointer (in pass 2). */ p += len - strm.avail_out; len -= len - strm.avail_out; } } while (strm.avail_out == 0 && r != Z_STREAM_END); if (pass == 1) { *decompressed_lenp = strm.total_out; if (!(decompressed = malloc(*decompressed_lenp+1))) { inflateEnd(&strm); return NULL; } /* For convenience of the caller we nul-terminate * the buffer. If it happens to be a string there * is no need for extra copies. */ decompressed[*decompressed_lenp] = '\0'; } inflateEnd(&strm); } return decompressed; fail: if (decompressed) free(decompressed); return NULL; } librdkafka-0.8.3/rdgz.h000066400000000000000000000036021227620010100147230ustar00rootroot00000000000000/* * librd - Rapid Development C library * * Copyright (c) 2012, Magnus Edenhill * All rights reserved. * * Redistribution and use in source and binary forms, with or without * modification, are permitted provided that the following conditions are met: * * 1. Redistributions of source code must retain the above copyright notice, * this list of conditions and the following disclaimer. * 2. Redistributions in binary form must reproduce the above copyright notice, * this list of conditions and the following disclaimer in the documentation * and/or other materials provided with the distribution. * * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE * POSSIBILITY OF SUCH DAMAGE. */ #pragma once /** * Simple gzip decompression returning the inflated data * in a malloced buffer. * '*decompressed_lenp' must be 0 if the length of the uncompressed data * is not known in which case it will be calculated. * The returned buffer is nul-terminated (the actual allocated length * is '*decompressed_lenp'+1. * * The decompressed length is returned in '*decompressed_lenp'. */ void *rd_gz_decompress (void *compressed, int compressed_len, uint64_t *decompressed_lenp); librdkafka-0.8.3/rdkafka.c000066400000000000000000001052211227620010100153530ustar00rootroot00000000000000/* * librdkafka - Apache Kafka C library * * Copyright (c) 2012-2013, Magnus Edenhill * All rights reserved. * * Redistribution and use in source and binary forms, with or without * modification, are permitted provided that the following conditions are met: * * 1. Redistributions of source code must retain the above copyright notice, * this list of conditions and the following disclaimer. * 2. Redistributions in binary form must reproduce the above copyright notice, * this list of conditions and the following disclaimer in the documentation * and/or other materials provided with the distribution. * * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE * POSSIBILITY OF SUCH DAMAGE. */ #define _GNU_SOURCE #include #include #include #include #include #include "rdkafka_int.h" #include "rdkafka_msg.h" #include "rdkafka_broker.h" #include "rdkafka_topic.h" #include "rdkafka_offset.h" #include "rdtime.h" static pthread_once_t rd_kafka_global_init_once = PTHREAD_ONCE_INIT; /** * Current number of threads created by rdkafka. * This is used in regression tests. */ int rd_kafka_thread_cnt_curr = 0; int rd_kafka_thread_cnt (void) { return rd_kafka_thread_cnt_curr; } /** * Current number of live rd_kafka_t handles. * This is used by rd_kafka_wait_destroyed() to know when the library * has fully cleaned up after itself. */ static int rd_kafka_handle_cnt_curr = 0; /* atomic */ /** * Wait for all rd_kafka_t objects to be destroyed. * Returns 0 if all kafka objects are now destroyed, or -1 if the * timeout was reached. */ int rd_kafka_wait_destroyed (int timeout_ms) { rd_ts_t timeout = rd_clock() + (timeout_ms * 1000); while (rd_kafka_thread_cnt() > 0 || rd_kafka_handle_cnt_curr > 0) { if (rd_clock() >= timeout) { errno = ETIMEDOUT; return -1; } usleep(25000); /* 25ms */ } return 0; } /** * Wrapper for pthread_cond_timedwait() that makes it simpler to use * for delta timeouts. * `timeout_ms' is the delta timeout in milliseconds. */ int pthread_cond_timedwait_ms (pthread_cond_t *cond, pthread_mutex_t *mutex, int timeout_ms) { struct timeval tv; struct timespec ts; gettimeofday(&tv, NULL); TIMEVAL_TO_TIMESPEC(&tv, &ts); ts.tv_sec += timeout_ms / 1000; ts.tv_nsec += (timeout_ms % 1000) * 1000000; if (ts.tv_nsec > 1000000000) { ts.tv_sec++; ts.tv_nsec -= 1000000000; } return pthread_cond_timedwait(cond, mutex, &ts); } void rd_kafka_log_buf (const rd_kafka_t *rk, int level, const char *fac, const char *buf) { if (!rk->rk_log_cb || level > rk->rk_log_level) return; rk->rk_log_cb(rk, level, fac, buf); } void rd_kafka_log0 (const rd_kafka_t *rk, const char *extra, int level, const char *fac, const char *fmt, ...) { char buf[2048]; va_list ap; int elen = 0; if (!rk->rk_log_cb || level > rk->rk_log_level) return; if (extra) { elen = snprintf(buf, sizeof(buf), "%s: ", extra); if (unlikely(elen >= sizeof(buf))) elen = sizeof(buf); } va_start(ap, fmt); vsnprintf(buf+elen, sizeof(buf)-elen, fmt, ap); va_end(ap); rk->rk_log_cb(rk, level, fac, buf); } void rd_kafka_log_print (const rd_kafka_t *rk, int level, const char *fac, const char *buf) { struct timeval tv; gettimeofday(&tv, NULL); fprintf(stderr, "%%%i|%u.%03u|%s|%s| %s\n", level, (int)tv.tv_sec, (int)(tv.tv_usec / 1000), fac, rk ? rk->rk_name : "", buf); } void rd_kafka_log_syslog (const rd_kafka_t *rk, int level, const char *fac, const char *buf) { static int initialized = 0; if (!initialized) openlog("rdkafka", LOG_PID|LOG_CONS, LOG_USER); syslog(level, "%s: %s: %s", fac, rk ? rk->rk_name : "", buf); } void rd_kafka_set_logger (rd_kafka_t *rk, void (*func) (const rd_kafka_t *rk, int level, const char *fac, const char *buf)) { rk->rk_log_cb = func; } void rd_kafka_set_log_level (rd_kafka_t *rk, int level) { rk->rk_log_level = level; } rd_kafka_op_t *rd_kafka_op_new (rd_kafka_op_type_t type) { rd_kafka_op_t *rko; rko = calloc(1, sizeof(*rko)); rko->rko_type = type; return rko; } void rd_kafka_op_destroy (rd_kafka_op_t *rko) { /* Decrease refcount on rkbuf to eventually free the shared buffer */ if (rko->rko_rkbuf) rd_kafka_buf_destroy(rko->rko_rkbuf); else if (rko->rko_payload && rko->rko_flags & RD_KAFKA_OP_F_FREE) free(rko->rko_payload); free(rko); } /** * Destroy a queue. The queue must be empty. */ void rd_kafka_q_destroy (rd_kafka_q_t *rkq) { assert(TAILQ_EMPTY(&rkq->rkq_q)); pthread_mutex_destroy(&rkq->rkq_lock); pthread_cond_destroy(&rkq->rkq_cond); } /** * Initialize a queue. */ void rd_kafka_q_init (rd_kafka_q_t *rkq) { TAILQ_INIT(&rkq->rkq_q); rkq->rkq_qlen = 0; pthread_mutex_init(&rkq->rkq_lock, NULL); pthread_cond_init(&rkq->rkq_cond, NULL); } /** * Purge all entries from a queue. */ void rd_kafka_q_purge (rd_kafka_q_t *rkq) { rd_kafka_op_t *rko, *next; pthread_mutex_lock(&rkq->rkq_lock); next = TAILQ_FIRST(&rkq->rkq_q); while ((rko = next)) { next = TAILQ_NEXT(next, rko_link); rd_kafka_op_destroy(rko); } TAILQ_INIT(&rkq->rkq_q); (void)rd_atomic_set(&rkq->rkq_qlen, 0); pthread_mutex_unlock(&rkq->rkq_lock); } /** * Move 'cnt' entries from 'srcq' to 'dstq'. * Returns the number of entries moved. */ size_t rd_kafka_q_move_cnt (rd_kafka_q_t *dstq, rd_kafka_q_t *srcq, size_t cnt) { rd_kafka_op_t *rko; size_t mcnt = 0; pthread_mutex_lock(&srcq->rkq_lock); pthread_mutex_lock(&dstq->rkq_lock); /* Optimization, if 'cnt' is equal/larger than all items of 'srcq' * we can move the entire queue. */ if (cnt >= srcq->rkq_qlen) { mcnt = srcq->rkq_qlen; TAILQ_CONCAT(&dstq->rkq_q, &srcq->rkq_q, rko_link); TAILQ_INIT(&srcq->rkq_q); (void)rd_atomic_set(&srcq->rkq_qlen, 0); (void)rd_atomic_add(&dstq->rkq_qlen, mcnt); } else { while (mcnt < cnt && (rko = TAILQ_FIRST(&srcq->rkq_q))) { TAILQ_REMOVE(&srcq->rkq_q, rko, rko_link); TAILQ_INSERT_TAIL(&dstq->rkq_q, rko, rko_link); (void)rd_atomic_sub(&dstq->rkq_qlen, 1); (void)rd_atomic_add(&dstq->rkq_qlen, 1); mcnt++; } } pthread_mutex_unlock(&dstq->rkq_lock); pthread_mutex_unlock(&srcq->rkq_lock); return mcnt; } /** * Pop an op from a queue. * * Locality: any thread. */ rd_kafka_op_t *rd_kafka_q_pop (rd_kafka_q_t *rkq, int timeout_ms) { rd_kafka_op_t *rko; pthread_mutex_lock(&rkq->rkq_lock); while (!(rko = TAILQ_FIRST(&rkq->rkq_q)) && timeout_ms != RD_POLL_NOWAIT) { if (timeout_ms != RD_POLL_INFINITE) { if (pthread_cond_timedwait_ms(&rkq->rkq_cond, &rkq->rkq_lock, timeout_ms) == ETIMEDOUT) { pthread_mutex_unlock(&rkq->rkq_lock); return NULL; } timeout_ms = 0; } else pthread_cond_wait(&rkq->rkq_cond, &rkq->rkq_lock); } if (rko) { TAILQ_REMOVE(&rkq->rkq_q, rko, rko_link); (void)rd_atomic_sub(&rkq->rkq_qlen, 1); } pthread_mutex_unlock(&rkq->rkq_lock); return rko; } /** * Pop all available ops from a queue and call the provided * callback for each op. * * Returns the number of ops served. * * Locality: any thread. */ int rd_kafka_q_serve (rd_kafka_t *rk, rd_kafka_q_t *rkq, int timeout_ms, void (*callback) (rd_kafka_op_t *rko, void *opaque), void *opaque) { rd_kafka_op_t *rko, *tmp; rd_kafka_q_t localq; TAILQ_INIT(&localq.rkq_q); pthread_mutex_lock(&rkq->rkq_lock); /* Wait for op */ while (!(rko = TAILQ_FIRST(&rkq->rkq_q)) && timeout_ms != 0) { if (timeout_ms != RD_POLL_INFINITE) { if (pthread_cond_timedwait_ms(&rkq->rkq_cond, &rkq->rkq_lock, timeout_ms) == ETIMEDOUT) break; timeout_ms = 0; } else pthread_cond_wait(&rkq->rkq_cond, &rkq->rkq_lock); } if (!rko) { pthread_mutex_unlock(&rkq->rkq_lock); return 0; } /* Move all ops to local queue */ TAILQ_CONCAT(&localq.rkq_q, &rkq->rkq_q, rko_link); localq.rkq_qlen = rkq->rkq_qlen; /* Reset real queue */ TAILQ_INIT(&rkq->rkq_q); (void)rd_atomic_set(&rkq->rkq_qlen, 0); pthread_mutex_unlock(&rkq->rkq_lock); rd_kafka_dbg(rk, QUEUE, "QSERVE", "Serving %i ops", localq.rkq_qlen); /* Call callback for each op */ TAILQ_FOREACH_SAFE(rko, &localq.rkq_q, rko_link, tmp) { callback(rko, opaque); rd_kafka_op_destroy(rko); } return localq.rkq_qlen; } /** * Send an op back to the application. * * Locality: Kafka threads */ void rd_kafka_op_reply0 (rd_kafka_t *rk, rd_kafka_op_t *rko, rd_kafka_op_type_t type, rd_kafka_resp_err_t err, void *payload, int len) { rko->rko_type = type; rko->rko_flags |= RD_KAFKA_OP_F_FREE; rko->rko_payload = payload; rko->rko_len = len; rko->rko_err = err; } /** * Send an op back to the application. * * Locality: Kafka thread */ void rd_kafka_op_reply (rd_kafka_t *rk, rd_kafka_op_type_t type, rd_kafka_resp_err_t err, void *payload, int len) { rd_kafka_op_t *rko; rko = calloc(1, sizeof(*rko)); if (err && !payload) { /* Provide human readable error string if not provided. */ /* Provide more info for some errors. */ if (err == RD_KAFKA_RESP_ERR_OFFSET_OUT_OF_RANGE) { char tmp[512]; snprintf(tmp, sizeof(tmp), "%s (%"PRIu64")", rd_kafka_err2str(err), rk->rk_consumer.offset); payload = strdup(tmp); } else payload = strdup(rd_kafka_err2str(err)); len = strlen(payload); } rd_kafka_op_reply0(rk, rko, type, err, payload, len); rd_kafka_q_enq(&rk->rk_rep, rko); } void rd_kafka_op_reply2 (rd_kafka_t *rk, rd_kafka_op_t *rko) { rd_kafka_q_enq(&rk->rk_rep, rko); } /** * Propogate an error event to the application. * If no error_cb has been set by the application the error will * be logged instead. */ void rd_kafka_op_err (rd_kafka_t *rk, rd_kafka_resp_err_t err, const char *fmt, ...) { va_list ap; char buf[2048]; va_start(ap, fmt); vsnprintf(buf, sizeof(buf), fmt, ap); va_end(ap); if (rk->rk_conf.error_cb) rd_kafka_op_reply(rk, RD_KAFKA_OP_ERR, err, strdup(buf), strlen(buf)); else rd_kafka_log_buf(rk, LOG_ERR, "ERROR", buf); } static const char *rd_kafka_type2str (rd_kafka_type_t type) { static const char *types[] = { [RD_KAFKA_PRODUCER] = "producer", [RD_KAFKA_CONSUMER] = "consumer", }; return types[type]; } const char *rd_kafka_err2str (rd_kafka_resp_err_t err) { static __thread char ret[32]; switch (err) { case RD_KAFKA_RESP_ERR__BAD_MSG: return "Local: Bad message format"; case RD_KAFKA_RESP_ERR__BAD_COMPRESSION: return "Local: Invalid compressed data"; case RD_KAFKA_RESP_ERR__DESTROY: return "Local: Broker handle destroyed"; case RD_KAFKA_RESP_ERR__FAIL: return "Local: Communication failure with broker"; case RD_KAFKA_RESP_ERR__TRANSPORT: return "Local: Broker transport failure"; case RD_KAFKA_RESP_ERR__CRIT_SYS_RESOURCE: return "Local: Critical system resource failure"; case RD_KAFKA_RESP_ERR__RESOLVE: return "Local: Host resolution failure"; case RD_KAFKA_RESP_ERR__MSG_TIMED_OUT: return "Local: Message timed out"; case RD_KAFKA_RESP_ERR__PARTITION_EOF: return "Broker: No more messages"; case RD_KAFKA_RESP_ERR__UNKNOWN_PARTITION: return "Local: Unknown partition"; case RD_KAFKA_RESP_ERR__FS: return "Local: File or filesystem error"; case RD_KAFKA_RESP_ERR__UNKNOWN_TOPIC: return "Local: Unknown topic"; case RD_KAFKA_RESP_ERR__ALL_BROKERS_DOWN: return "Local: All broker connections are down"; case RD_KAFKA_RESP_ERR__INVALID_ARG: return "Local: Invalid argument or configuration"; case RD_KAFKA_RESP_ERR__TIMED_OUT: return "Local: Timed out"; case RD_KAFKA_RESP_ERR__QUEUE_FULL: return "Local: Queue full"; case RD_KAFKA_RESP_ERR_UNKNOWN: return "Unknown error"; case RD_KAFKA_RESP_ERR_NO_ERROR: return "Success"; case RD_KAFKA_RESP_ERR_OFFSET_OUT_OF_RANGE: return "Broker: Offset out of range"; case RD_KAFKA_RESP_ERR_INVALID_MSG: return "Broker: Invalid message"; case RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_OR_PART: return "Broker: Unknown topic or partition"; case RD_KAFKA_RESP_ERR_INVALID_MSG_SIZE: return "Broker: Invalid message size"; case RD_KAFKA_RESP_ERR_LEADER_NOT_AVAILABLE: return "Broker: Leader not available"; case RD_KAFKA_RESP_ERR_NOT_LEADER_FOR_PARTITION: return "Broker: Not leader for partition"; case RD_KAFKA_RESP_ERR_REQUEST_TIMED_OUT: return "Broker: Request timed out"; case RD_KAFKA_RESP_ERR_BROKER_NOT_AVAILABLE: return "Broker: Broker not available"; case RD_KAFKA_RESP_ERR_REPLICA_NOT_AVAILABLE: return "Broker: Replica not available"; case RD_KAFKA_RESP_ERR_MSG_SIZE_TOO_LARGE: return "Broker: Message size too large"; case RD_KAFKA_RESP_ERR_STALE_CTRL_EPOCH: return "Broker: StaleControllerEpochCode"; case RD_KAFKA_RESP_ERR_OFFSET_METADATA_TOO_LARGE: return "Broker: Offset metadata string too large"; default: snprintf(ret, sizeof(ret), "Err-%i?", err); return ret; } } rd_kafka_resp_err_t rd_kafka_errno2err (int errnox) { switch (errnox) { case EINVAL: return RD_KAFKA_RESP_ERR__INVALID_ARG; case ENOENT: return RD_KAFKA_RESP_ERR__UNKNOWN_TOPIC; case ESRCH: return RD_KAFKA_RESP_ERR__UNKNOWN_PARTITION; case ETIMEDOUT: return RD_KAFKA_RESP_ERR__TIMED_OUT; case EMSGSIZE: return RD_KAFKA_RESP_ERR_MSG_SIZE_TOO_LARGE; case ENOBUFS: return RD_KAFKA_RESP_ERR__QUEUE_FULL; default: return RD_KAFKA_RESP_ERR__FAIL; } } void rd_kafka_destroy0 (rd_kafka_t *rk) { if (rd_atomic_sub(&rk->rk_refcnt, 1) > 0) return; /* Purge op-queue */ rd_kafka_q_purge(&rk->rk_rep); rd_kafkap_str_destroy(rk->rk_clientid); rd_kafka_anyconf_destroy(_RK_GLOBAL, &rk->rk_conf); pthread_mutex_destroy(&rk->rk_lock); free(rk); rd_atomic_sub(&rd_kafka_handle_cnt_curr, 1); } /* NOTE: Must only be called by application. * librdkafka itself must use rd_kafka_destroy0(). */ void rd_kafka_destroy (rd_kafka_t *rk) { rd_kafka_topic_t *rkt, *rkt_tmp; rd_kafka_dbg(rk, GENERIC, "DESTROY", "Terminating instance"); (void)rd_atomic_add(&rk->rk_terminate, 1); /* Decommission all topics */ rd_kafka_lock(rk); TAILQ_FOREACH_SAFE(rkt, &rk->rk_topics, rkt_link, rkt_tmp) { rd_kafka_unlock(rk); rd_kafka_topic_partitions_remove(rkt); rd_kafka_lock(rk); } rd_kafka_unlock(rk); /* Brokers pick up on rk_terminate automatically. */ rd_kafka_destroy0(rk); } /* Stats buffer printf */ #define _st_printf(fmt...) do { \ ssize_t r; \ ssize_t rem = size-of; \ r = snprintf(buf+of, rem, fmt); \ if (r > rem) { \ size *= 2; \ buf = realloc(buf, size); \ r = snprintf(buf+of, size-of, fmt); \ } \ of += r; \ } while (0) /** * Emit stats for toppar */ static inline void rd_kafka_stats_emit_toppar (char **bufp, size_t *sizep, int *ofp, rd_kafka_toppar_t *rktp, int first) { char *buf = *bufp; size_t size = *sizep; int of = *ofp; _st_printf("%s\"%"PRId32"\": { " "\"partition\":%"PRId32", " "\"leader\":%"PRId32", " "\"desired\":%s, " "\"unknown\":%s, " "\"msgq_cnt\":%i, " "\"msgq_bytes\":%"PRIu64", " "\"xmit_msgq_cnt\":%i, " "\"xmit_msgq_bytes\":%"PRIu64", " "\"fetchq_cnt\":%i, " "\"fetch_state\":\"%s\", " "\"query_offset\":%"PRId64", " "\"next_offset\":%"PRId64", " "\"app_offset\":%"PRId64", " "\"commited_offset\":%"PRId64", " "\"eof_offset\":%"PRId64", " "\"txmsgs\":%"PRIu64", " "\"txbytes\":%"PRIu64" " "} ", first ? "" : ", ", rktp->rktp_partition, rktp->rktp_partition, rktp->rktp_leader ? rktp->rktp_leader->rkb_nodeid : -1, (rktp->rktp_flags&RD_KAFKA_TOPPAR_F_DESIRED)?"true":"false", (rktp->rktp_flags&RD_KAFKA_TOPPAR_F_UNKNOWN)?"true":"false", rktp->rktp_msgq.rkmq_msg_cnt, rktp->rktp_msgq.rkmq_msg_bytes, rktp->rktp_xmit_msgq.rkmq_msg_cnt, rktp->rktp_xmit_msgq.rkmq_msg_bytes, rktp->rktp_fetchq.rkq_qlen, rd_kafka_fetch_states[rktp->rktp_fetch_state], rktp->rktp_query_offset, rktp->rktp_next_offset, rktp->rktp_app_offset, rktp->rktp_commited_offset, rktp->rktp_eof_offset, rktp->rktp_c.tx_msgs, rktp->rktp_c.tx_bytes); *bufp = buf; *sizep = size; *ofp = of; } /** * Emit all statistics */ static void rd_kafka_stats_emit_all (rd_kafka_t *rk) { char *buf; size_t size = 1024*rk->rk_refcnt; int of = 0; rd_kafka_broker_t *rkb; rd_kafka_topic_t *rkt; rd_kafka_toppar_t *rktp; rd_ts_t now; buf = malloc(size); rd_kafka_lock(rk); now = rd_clock(); _st_printf("{ " "\"ts\":%"PRIu64", " "\"time\":%lli, " "\"replyq\":%i, " "\"brokers\":{ "/*open brokers*/, now, (signed long long)time(NULL), rk->rk_rep.rkq_qlen); TAILQ_FOREACH(rkb, &rk->rk_brokers, rkb_link) { rd_kafka_broker_lock(rkb); rd_kafka_avg_rollover(&rkb->rkb_rtt_last, &rkb->rkb_rtt_curr); _st_printf("%s\"%s\": { "/*open broker*/ "\"name\":\"%s\", " "\"nodeid\":%"PRId32", " "\"state\":\"%s\", " "\"outbuf_cnt\":%i, " "\"waitresp_cnt\":%i, " "\"tx\":%"PRIu64", " "\"txbytes\":%"PRIu64", " "\"txerrs\":%"PRIu64", " "\"txretries\":%"PRIu64", " "\"rx\":%"PRIu64", " "\"rxbytes\":%"PRIu64", " "\"rxerrs\":%"PRIu64", " "\"rxcorriderrs\":%"PRIu64", " "\"rtt\": {" " \"min\":%"PRIu64"," " \"max\":%"PRIu64"," " \"avg\":%"PRIu64"," " \"cnt\":%i " "}, " "\"toppars\":{ "/*open toppars*/, rkb == TAILQ_FIRST(&rk->rk_brokers) ? "" : ", ", rkb->rkb_name, rkb->rkb_name, rkb->rkb_nodeid, rd_kafka_broker_state_names[rkb->rkb_state], rkb->rkb_outbufs.rkbq_cnt, rkb->rkb_waitresps.rkbq_cnt, rkb->rkb_c.tx, rkb->rkb_c.tx_bytes, rkb->rkb_c.tx_err, rkb->rkb_c.tx_retries, rkb->rkb_c.rx, rkb->rkb_c.rx_bytes, rkb->rkb_c.rx_err, rkb->rkb_c.rx_corrid_err, rkb->rkb_rtt_last.ra_min, rkb->rkb_rtt_last.ra_max, rkb->rkb_rtt_last.ra_avg, rkb->rkb_rtt_last.ra_cnt); rd_kafka_broker_toppars_rdlock(rkb); TAILQ_FOREACH(rktp, &rkb->rkb_toppars, rktp_rkblink) { _st_printf("%s\"%.*s\": { " "\"topic\":\"%.*s\", " "\"partition\":%"PRId32"} ", rktp==TAILQ_FIRST(&rkb->rkb_toppars)?"":", ", RD_KAFKAP_STR_PR(rktp->rktp_rkt->rkt_topic), RD_KAFKAP_STR_PR(rktp->rktp_rkt->rkt_topic), rktp->rktp_partition); } rd_kafka_broker_toppars_unlock(rkb); rd_kafka_broker_unlock(rkb); _st_printf("} "/*close toppars*/ "} "/*close broker*/); } _st_printf("}, " /* close "brokers" array */ "\"topics\":{ "); TAILQ_FOREACH(rkt, &rk->rk_topics, rkt_link) { int i; rd_kafka_topic_rdlock(rkt); _st_printf("%s\"%.*s\": { " "\"topic\":\"%.*s\", " "\"partitions\":{ " /*open partitions*/, rkt==TAILQ_FIRST(&rk->rk_topics)?"":", ", RD_KAFKAP_STR_PR(rkt->rkt_topic), RD_KAFKAP_STR_PR(rkt->rkt_topic)); for (i = 0 ; i < rkt->rkt_partition_cnt ; i++) rd_kafka_stats_emit_toppar(&buf, &size, &of, rkt->rkt_p[i], i == 0); TAILQ_FOREACH(rktp, &rkt->rkt_desp, rktp_rktlink) rd_kafka_stats_emit_toppar(&buf, &size, &of, rktp, i++ == 0); if (rkt->rkt_ua) rd_kafka_stats_emit_toppar(&buf, &size, &of, rkt->rkt_ua, i++ == 0); rd_kafka_topic_unlock(rkt); _st_printf("} "/*close partitions*/ "} "/*close topic*/); } rd_kafka_unlock(rk); _st_printf("} "/*close topics*/ "}"/*close object*/); /* Enqueue op for application */ rd_kafka_op_reply(rk, RD_KAFKA_OP_STATS, 0, buf, of); } static void rd_kafka_topic_scan_tmr_cb (rd_kafka_t *rk, void *arg) { rd_kafka_topic_scan_all(rk, rd_clock()); } static void rd_kafka_stats_emit_tmr_cb (rd_kafka_t *rk, void *arg) { rd_kafka_stats_emit_all(rk); } /** * Main loop for Kafka handler thread. */ static void *rd_kafka_thread_main (void *arg) { rd_kafka_t *rk = arg; rd_kafka_timer_t tmr_topic_scan = {}; rd_kafka_timer_t tmr_stats_emit = {}; (void)rd_atomic_add(&rd_kafka_thread_cnt_curr, 1); rd_kafka_timer_start(rk, &tmr_topic_scan, 1000000, rd_kafka_topic_scan_tmr_cb, NULL); rd_kafka_timer_start(rk, &tmr_stats_emit, rk->rk_conf.stats_interval_ms * 1000, rd_kafka_stats_emit_tmr_cb, NULL); while (likely(rk->rk_terminate == 0)) { rd_kafka_timers_run(rk, 1000000); } rd_kafka_destroy0(rk); /* destroy handler thread's refcnt */ (void)rd_atomic_sub(&rd_kafka_thread_cnt_curr, 1); return NULL; } static void rd_kafka_global_init (void) { } rd_kafka_t *rd_kafka_new (rd_kafka_type_t type, rd_kafka_conf_t *conf, char *errstr, size_t errstr_size) { rd_kafka_t *rk; static int rkid = 0; pthread_attr_t attr; int err; pthread_once(&rd_kafka_global_init_once, rd_kafka_global_init); /* * Set up the handle. */ rk = calloc(1, sizeof(*rk)); rk->rk_type = type; if (!conf) conf = rd_kafka_conf_new(); rk->rk_conf = *conf; free(conf); rd_kafka_keep(rk); /* application refcnt */ pthread_mutex_init(&rk->rk_lock, NULL); rd_kafka_q_init(&rk->rk_rep); TAILQ_INIT(&rk->rk_brokers); TAILQ_INIT(&rk->rk_topics); TAILQ_INIT(&rk->rk_timers); pthread_mutex_init(&rk->rk_timers_lock, NULL); pthread_cond_init(&rk->rk_timers_cond, NULL); rk->rk_log_cb = rd_kafka_log_print; if (rk->rk_conf.debug) rd_kafka_set_log_level(rk, LOG_DEBUG); else rk->rk_log_level = LOG_INFO; /* Construct a client id if none is given. */ if (!rk->rk_conf.clientid) rk->rk_conf.clientid = strdup("rdkafka"); snprintf(rk->rk_name, sizeof(rk->rk_name), "%s#%s-%i", rk->rk_conf.clientid, rd_kafka_type2str(rk->rk_type), rkid++); /* Construct clientid kafka string */ rk->rk_clientid = rd_kafkap_str_new(rk->rk_conf.clientid); if (rk->rk_type == RD_KAFKA_CONSUMER) { /* Pre-build RequestHeader */ rk->rk_conf.FetchRequest.ReplicaId = htonl(-1); rk->rk_conf.FetchRequest.MaxWaitTime = htonl(rk->rk_conf.fetch_wait_max_ms); rk->rk_conf.FetchRequest.MinBytes = htonl(rk->rk_conf.fetch_min_bytes); } /* Create handler thread */ pthread_attr_init(&attr); pthread_attr_setdetachstate(&attr, PTHREAD_CREATE_DETACHED); rd_kafka_keep(rk); /* one refcnt for handler thread */ if ((err = pthread_create(&rk->rk_thread, &attr, rd_kafka_thread_main, rk))) { if (errstr) snprintf(errstr, errstr_size, "Failed to create thread: %s", strerror(err)); rd_kafka_destroy0(rk); /* handler thread */ rd_kafka_destroy0(rk); /* application refcnt */ errno = err; return NULL; } /* Add initial list of brokers from configuration */ if (rk->rk_conf.brokerlist) rd_kafka_brokers_add(rk, rk->rk_conf.brokerlist); rd_atomic_add(&rd_kafka_handle_cnt_curr, 1); return rk; } /** * Produce a single message. * Locality: any application thread */ int rd_kafka_produce (rd_kafka_topic_t *rkt, int32_t partition, int msgflags, void *payload, size_t len, const void *key, size_t keylen, void *msg_opaque) { return rd_kafka_msg_new(rkt, partition, msgflags, payload, len, key, keylen, msg_opaque); } int rd_kafka_consume_start (rd_kafka_topic_t *rkt, int32_t partition, int64_t offset) { rd_kafka_toppar_t *rktp; if (partition < 0) { errno = ESRCH; return -1; } rd_kafka_topic_wrlock(rkt); rktp = rd_kafka_toppar_desired_add(rkt, partition); rd_kafka_topic_unlock(rkt); rd_kafka_toppar_lock(rktp); switch (offset) { case RD_KAFKA_OFFSET_BEGINNING: case RD_KAFKA_OFFSET_END: rktp->rktp_query_offset = offset; rktp->rktp_fetch_state = RD_KAFKA_TOPPAR_FETCH_OFFSET_QUERY; break; case RD_KAFKA_OFFSET_STORED: if (!rkt->rkt_conf.auto_commit) { rd_kafka_toppar_unlock(rktp); rd_kafka_toppar_destroy(rktp); errno = EINVAL; return -1; } rd_kafka_offset_store_init(rktp); break; default: rktp->rktp_next_offset = offset; rktp->rktp_fetch_state = RD_KAFKA_TOPPAR_FETCH_ACTIVE; } rd_kafka_toppar_unlock(rktp); rd_kafka_dbg(rkt->rkt_rk, TOPIC, "CONSUMER", "Start consuming %.*s [%"PRId32"] at " "offset %"PRId64, RD_KAFKAP_STR_PR(rktp->rktp_rkt->rkt_topic), rktp->rktp_partition, offset); return 0; } int rd_kafka_consume_stop (rd_kafka_topic_t *rkt, int32_t partition) { rd_kafka_toppar_t *rktp; if (partition == RD_KAFKA_PARTITION_UA) { errno = EINVAL; return -1; } rd_kafka_topic_wrlock(rkt); if (!(rktp = rd_kafka_toppar_get(rkt, partition, 0)) && !(rktp = rd_kafka_toppar_desired_get(rkt, partition))) { rd_kafka_topic_unlock(rkt); errno = ESRCH; return -1; } rd_kafka_toppar_desired_del(rktp); rd_kafka_topic_unlock(rkt); rd_kafka_toppar_lock(rktp); rktp->rktp_fetch_state = RD_KAFKA_TOPPAR_FETCH_NONE; if (rktp->rktp_offset_path) rd_kafka_offset_store_term(rktp); /* Purge receive queue. */ rd_kafka_q_purge(&rktp->rktp_fetchq); rd_kafka_dbg(rkt->rkt_rk, TOPIC, "CONSUMER", "Stop consuming %.*s [%"PRId32"] currently at offset " "%"PRId64, RD_KAFKAP_STR_PR(rktp->rktp_rkt->rkt_topic), rktp->rktp_partition, rktp->rktp_next_offset); rd_kafka_toppar_unlock(rktp); rd_kafka_toppar_destroy(rktp); /* .._get() */ return 0; } void rd_kafka_message_destroy (rd_kafka_message_t *rkmessage) { rd_kafka_op_t *rko; if (likely((rko = (rd_kafka_op_t *)rkmessage->_private) != NULL)) rd_kafka_op_destroy(rko); else free(rkmessage); } static rd_kafka_message_t *rd_kafka_message_get (rd_kafka_op_t *rko) { rd_kafka_message_t *rkmessage; if (rko) { rkmessage = &rko->rko_rkmessage; rkmessage->_private = rko; } else rkmessage = calloc(1, sizeof(*rkmessage)); return rkmessage; } ssize_t rd_kafka_consume_batch (rd_kafka_topic_t *rkt, int32_t partition, int timeout_ms, rd_kafka_message_t **rkmessages, size_t rkmessages_size) { rd_kafka_toppar_t *rktp; struct timeval tv; struct timespec ts; ssize_t cnt = 0; gettimeofday(&tv, NULL); TIMEVAL_TO_TIMESPEC(&tv, &ts); ts.tv_sec += timeout_ms / 1000; ts.tv_nsec += (timeout_ms % 1000) * 1000000; if (ts.tv_nsec > 1000000000) { ts.tv_sec++; ts.tv_nsec -= 1000000000; } /* Get toppar */ rd_kafka_topic_rdlock(rkt); rktp = rd_kafka_toppar_get(rkt, partition, 0/*no ua on miss*/); if (unlikely(!rktp)) rktp = rd_kafka_toppar_desired_get(rkt, partition); rd_kafka_topic_unlock(rkt); if (unlikely(!rktp)) { /* No such toppar known */ errno = ESRCH; return -1; } /* Populate application's rkmessages array. */ while (cnt < rkmessages_size) { rd_kafka_op_t *rko; pthread_mutex_lock(&rktp->rktp_fetchq.rkq_lock); while (!(rko = TAILQ_FIRST(&rktp->rktp_fetchq.rkq_q))) { if (pthread_cond_timedwait(&rktp->rktp_fetchq.rkq_cond, &rktp->rktp_fetchq.rkq_lock, &ts) == ETIMEDOUT) break; } if (!rko) { /* Timed out */ pthread_mutex_unlock(&rktp->rktp_fetchq.rkq_lock); break; } TAILQ_REMOVE(&rktp->rktp_fetchq.rkq_q, rko, rko_link); (void)rd_atomic_sub(&rktp->rktp_fetchq.rkq_qlen, 1); pthread_mutex_unlock(&rktp->rktp_fetchq.rkq_lock); /* Get rkmessage from rko and append to array. */ rkmessages[cnt++] = rd_kafka_message_get(rko); } /* Auto store offset of last message in batch, if enabled */ if (cnt > 0 && rkt->rkt_conf.auto_commit) rd_kafka_offset_store0(rktp, rkmessages[cnt-1]->offset, 1/*lock*/); rd_kafka_toppar_destroy(rktp); /* refcnt from .._get() */ return cnt; } struct consume_ctx { void (*consume_cb) (rd_kafka_message_t *rkmessage, void *opaque); void *opaque; rd_kafka_toppar_t *rktp; }; /** * Trampoline for application's consume_cb() */ static void rd_kafka_consume_cb (rd_kafka_op_t *rko, void *opaque) { struct consume_ctx *ctx = opaque; rd_kafka_message_t *rkmessage; rkmessage = rd_kafka_message_get(rko); if (ctx->rktp->rktp_rkt->rkt_conf.auto_commit) rd_kafka_offset_store0(ctx->rktp, rkmessage->offset, 1/*lock*/); ctx->consume_cb(rkmessage, ctx->opaque); } int rd_kafka_consume_callback (rd_kafka_topic_t *rkt, int32_t partition, int timeout_ms, void (*consume_cb) (rd_kafka_message_t *rkmessage, void *opaque), void *opaque) { rd_kafka_toppar_t *rktp; struct consume_ctx ctx = { .consume_cb = consume_cb, .opaque = opaque }; int r; /* Get toppar */ rd_kafka_topic_rdlock(rkt); rktp = rd_kafka_toppar_get(rkt, partition, 0/*no ua on miss*/); if (unlikely(!rktp)) rktp = rd_kafka_toppar_desired_get(rkt, partition); rd_kafka_topic_unlock(rkt); if (unlikely(!rktp)) { /* No such toppar known */ errno = ESRCH; return -1; } ctx.rktp = rktp; r = rd_kafka_q_serve(rkt->rkt_rk, &rktp->rktp_fetchq, timeout_ms, rd_kafka_consume_cb, &ctx); rd_kafka_toppar_destroy(rktp); return r; } rd_kafka_message_t *rd_kafka_consume (rd_kafka_topic_t *rkt, int32_t partition, int timeout_ms) { rd_kafka_op_t *rko; rd_kafka_toppar_t *rktp; rd_kafka_message_t *rkmessage; rd_kafka_topic_rdlock(rkt); rktp = rd_kafka_toppar_get(rkt, partition, 0/*no ua on miss*/); if (unlikely(!rktp)) rktp = rd_kafka_toppar_desired_get(rkt, partition); rd_kafka_topic_unlock(rkt); if (unlikely(!rktp)) { /* No such toppar known */ errno = ESRCH; return NULL; } /* Pop op from queue. May either be an error or a message. */ rko = rd_kafka_q_pop(&rktp->rktp_fetchq, timeout_ms); if (!rko) { /* Timeout reached with no op returned. */ rd_kafka_toppar_destroy(rktp); /* refcnt from .._get() */ errno = ETIMEDOUT; return NULL; } /* Get rkmessage from rko */ rkmessage = rd_kafka_message_get(rko); /* Store offset */ if (rktp->rktp_rkt->rkt_conf.auto_commit) rd_kafka_offset_store0(rktp, rkmessage->offset, 1/*lock*/); rd_kafka_toppar_destroy(rktp); /* refcnt from .._get() */ return rkmessage; } static void rd_kafka_poll_cb (rd_kafka_op_t *rko, void *opaque) { rd_kafka_t *rk = opaque; rd_kafka_msg_t *rkm; static int dcnt = 0; switch (rko->rko_type) { case RD_KAFKA_OP_FETCH: /* FIXME */ break; case RD_KAFKA_OP_ERR: if (rk->rk_conf.error_cb) rk->rk_conf.error_cb(rk, rko->rko_err, strndupa(rko->rko_payload, rko->rko_len), rk->rk_conf.opaque); else rd_kafka_log(rk, LOG_ERR, "ERROR", "%s: %s: %.*s", rk->rk_name, rd_kafka_err2str(rko->rko_err), (int)rko->rko_len, (char *)rko->rko_payload); break; case RD_KAFKA_OP_DR: /* Delivery report: * call application DR callback for each message. */ while ((rkm = TAILQ_FIRST(&rko->rko_msgq.rkmq_msgs))) { TAILQ_REMOVE(&rko->rko_msgq.rkmq_msgs, rkm, rkm_link); dcnt++; rk->rk_conf.dr_cb(rk, rkm->rkm_payload, rkm->rkm_len, rko->rko_err, rk->rk_conf.opaque, rkm->rkm_opaque); rd_kafka_msg_destroy(rk, rkm); } rd_kafka_msgq_init(&rko->rko_msgq); if (!(dcnt % 1000)) rd_kafka_dbg(rk, MSG, "POLL", "Now %i messages delivered to app", dcnt); break; case RD_KAFKA_OP_STATS: /* Statistics */ if (rk->rk_conf.stats_cb && rk->rk_conf.stats_cb(rk, rko->rko_json, rko->rko_json_len, rk->rk_conf.opaque) == 1) rko->rko_json = NULL; /* Application wanted json ptr */ break; default: rd_kafka_dbg(rk, ALL, "POLLCB", "cant handle op %i here", rko->rko_type); assert(!*"cant handle op type"); break; } } int rd_kafka_poll (rd_kafka_t *rk, int timeout_ms) { return rd_kafka_q_serve(rk, &rk->rk_rep, timeout_ms, rd_kafka_poll_cb, rk); } static void rd_kafka_toppar_dump (FILE *fp, const char *indent, rd_kafka_toppar_t *rktp) { fprintf(fp, "%s%.*s [%"PRId32"] leader %s\n", indent, RD_KAFKAP_STR_PR(rktp->rktp_rkt->rkt_topic), rktp->rktp_partition, rktp->rktp_leader ? rktp->rktp_leader->rkb_name : "none"); fprintf(fp, "%s refcnt %i\n" "%s msgq: %i messages\n" "%s xmit_msgq: %i messages\n" "%s total: %"PRIu64" messages, %"PRIu64" bytes\n", indent, rktp->rktp_refcnt, indent, rktp->rktp_msgq.rkmq_msg_cnt, indent, rktp->rktp_xmit_msgq.rkmq_msg_cnt, indent, rktp->rktp_c.tx_msgs, rktp->rktp_c.tx_bytes); } void rd_kafka_dump (FILE *fp, rd_kafka_t *rk) { rd_kafka_broker_t *rkb; rd_kafka_topic_t *rkt; rd_kafka_toppar_t *rktp; rd_kafka_lock(rk); fprintf(fp, "rd_kafka_t %p: %s\n", rk, rk->rk_name); fprintf(fp, " refcnt %i\n", rk->rk_refcnt); fprintf(fp, " rk_rep reply queue: %i ops\n", rk->rk_rep.rkq_qlen); fprintf(fp, " brokers:\n"); TAILQ_FOREACH(rkb, &rk->rk_brokers, rkb_link) { rd_kafka_broker_lock(rkb); fprintf(fp, " rd_kafka_broker_t %p: %s NodeId %"PRId32 " in state %s\n", rkb, rkb->rkb_name, rkb->rkb_nodeid, rd_kafka_broker_state_names[rkb->rkb_state]); fprintf(fp, " refcnt %i\n", rkb->rkb_refcnt); fprintf(fp, " outbuf_cnt: %i waitresp_cnt: %i\n", rkb->rkb_outbufs.rkbq_cnt, rkb->rkb_waitresps.rkbq_cnt); fprintf(fp, " %"PRIu64 " messages sent, %"PRIu64" bytes, " "%"PRIu64" errors\n" " %"PRIu64 " messages received, %"PRIu64" bytes, " "%"PRIu64" errors\n" " %"PRIu64 " messageset transmissions were retried\n", rkb->rkb_c.tx, rkb->rkb_c.tx_bytes, rkb->rkb_c.tx_err, rkb->rkb_c.rx, rkb->rkb_c.rx_bytes, rkb->rkb_c.rx_err, rkb->rkb_c.tx_retries); fprintf(fp, " %i toppars:\n", rkb->rkb_toppar_cnt); rd_kafka_broker_toppars_rdlock(rkb); TAILQ_FOREACH(rktp, &rkb->rkb_toppars, rktp_rkblink) rd_kafka_toppar_dump(fp, " ", rktp); rd_kafka_broker_toppars_unlock(rkb); rd_kafka_broker_unlock(rkb); } fprintf(fp, " topics:\n"); TAILQ_FOREACH(rkt, &rk->rk_topics, rkt_link) { fprintf(fp, " %.*s with %"PRId32" partitions, refcnt %i\n", RD_KAFKAP_STR_PR(rkt->rkt_topic), rkt->rkt_partition_cnt, rkt->rkt_refcnt); if (rkt->rkt_ua) rd_kafka_toppar_dump(fp, " ", rkt->rkt_ua); if (!TAILQ_EMPTY(&rkt->rkt_desp)) { fprintf(fp, " desired partitions:"); TAILQ_FOREACH(rktp, &rkt->rkt_desp, rktp_rktlink) fprintf(fp, " %"PRId32, rktp->rktp_partition); fprintf(fp, "\n"); } } rd_kafka_unlock(rk); } const char *rd_kafka_name (const rd_kafka_t *rk) { return rk->rk_name; } int rd_kafka_outq_len (rd_kafka_t *rk) { return rk->rk_producer.msg_cnt; } int rd_kafka_version (void) { return RD_KAFKA_VERSION; } const char *rd_kafka_version_str (void) { static char ret[64]; int ver = rd_kafka_version(); if (!*ret) snprintf(ret, sizeof(ret), "%i.%i.%i", (ver >> 24) & 0xff, (ver >> 16) & 0xff, (ver >> 8) & 0xff); return ret; } librdkafka-0.8.3/rdkafka.h000066400000000000000000000671741227620010100153760ustar00rootroot00000000000000/* * librdkafka - Apache Kafka C library * * Copyright (c) 2012-2013 Magnus Edenhill * All rights reserved. * * Redistribution and use in source and binary forms, with or without * modification, are permitted provided that the following conditions are met: * * 1. Redistributions of source code must retain the above copyright notice, * this list of conditions and the following disclaimer. * 2. Redistributions in binary form must reproduce the above copyright notice, * this list of conditions and the following disclaimer in the documentation * and/or other materials provided with the distribution. * * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE * POSSIBILITY OF SUCH DAMAGE. */ /** * Apache Kafka consumer & producer * * rdkafka.h contains the public API for librdkafka. * The API isdocumented in this file as comments prefixing the function, type, * enum, define, etc. */ #pragma once #include #include #include /** * librdkafka version * * Interpreted as hex MM.mm.rr.xx: * MM = Major * mm = minor * rr = revision * xx = currently unused * * I.e.: 0x00080100 = 0.8.1 */ #define RD_KAFKA_VERSION 0x00080300 /** * Returns the librdkafka version as integer. */ int rd_kafka_version (void); /** * Returns the librdkafka version as string. */ const char *rd_kafka_version_str (void); /** * rd_kafka_t handle type */ typedef enum { RD_KAFKA_PRODUCER, RD_KAFKA_CONSUMER, } rd_kafka_type_t; /** * Supported debug contexts (CSV "debug" configuration property) */ #define RD_KAFKA_DEBUG_CONTEXTS \ "all,generic,broker,topic,metadata,producer,queue,msg" /* Private types to provide ABI compatibility */ typedef struct rd_kafka_s rd_kafka_t; typedef struct rd_kafka_topic_s rd_kafka_topic_t; typedef struct rd_kafka_conf_s rd_kafka_conf_t; typedef struct rd_kafka_topic_conf_s rd_kafka_topic_conf_t; /** * Kafka protocol error codes (version 0.8) */ typedef enum { /* Internal errors to rdkafka: */ RD_KAFKA_RESP_ERR__BEGIN = -200, /* begin internal error codes */ RD_KAFKA_RESP_ERR__BAD_MSG = -199, /* Received message is incorrect */ RD_KAFKA_RESP_ERR__BAD_COMPRESSION = -198, /* Bad/unknown compression */ RD_KAFKA_RESP_ERR__DESTROY = -197, /* Broker is going away */ RD_KAFKA_RESP_ERR__FAIL = -196, /* Generic failure */ RD_KAFKA_RESP_ERR__TRANSPORT = -195, /* Broker transport error */ RD_KAFKA_RESP_ERR__CRIT_SYS_RESOURCE = -194, /* Critical system resource * failure */ RD_KAFKA_RESP_ERR__RESOLVE = -193, /* Failed to resolve broker */ RD_KAFKA_RESP_ERR__MSG_TIMED_OUT = -192, /* Produced message timed out*/ RD_KAFKA_RESP_ERR__PARTITION_EOF = -191, /* Reached the end of the * topic+partition queue on * the broker. * Not really an error. */ RD_KAFKA_RESP_ERR__UNKNOWN_PARTITION = -190, /* Permanent: * Partition does not * exist in cluster. */ RD_KAFKA_RESP_ERR__FS = -189, /* File or filesystem error */ RD_KAFKA_RESP_ERR__UNKNOWN_TOPIC = -188, /* Permanent: * Topic does not exist * in cluster. */ RD_KAFKA_RESP_ERR__ALL_BROKERS_DOWN = -187, /* All broker connections * are down. */ RD_KAFKA_RESP_ERR__INVALID_ARG = -186, /* Invalid argument, or * invalid configuration */ RD_KAFKA_RESP_ERR__TIMED_OUT = -185, /* Operation timed out */ RD_KAFKA_RESP_ERR__QUEUE_FULL = -184, /* Queue is full */ RD_KAFKA_RESP_ERR__END = -100, /* end internal error codes */ /* Standard Kafka errors: */ RD_KAFKA_RESP_ERR_UNKNOWN = -1, RD_KAFKA_RESP_ERR_NO_ERROR = 0, RD_KAFKA_RESP_ERR_OFFSET_OUT_OF_RANGE = 1, RD_KAFKA_RESP_ERR_INVALID_MSG = 2, RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_OR_PART = 3, RD_KAFKA_RESP_ERR_INVALID_MSG_SIZE = 4, RD_KAFKA_RESP_ERR_LEADER_NOT_AVAILABLE = 5, RD_KAFKA_RESP_ERR_NOT_LEADER_FOR_PARTITION = 6, RD_KAFKA_RESP_ERR_REQUEST_TIMED_OUT = 7, RD_KAFKA_RESP_ERR_BROKER_NOT_AVAILABLE = 8, RD_KAFKA_RESP_ERR_REPLICA_NOT_AVAILABLE = 9, RD_KAFKA_RESP_ERR_MSG_SIZE_TOO_LARGE = 10, RD_KAFKA_RESP_ERR_STALE_CTRL_EPOCH = 11, RD_KAFKA_RESP_ERR_OFFSET_METADATA_TOO_LARGE = 12, } rd_kafka_resp_err_t; /** * Returns a human readable representation of a kafka error. */ const char *rd_kafka_err2str (rd_kafka_resp_err_t err); /** * Converts `errno` to a `rd_kafka_resp_err_t` error code * upon failure from the following functions: * - rd_kafka_topic_new() * - rd_kafka_consume_start() * - rd_kafka_consume_stop() * - rd_kafka_consume() * - rd_kafka_consume_batch() * - rd_kafka_consume_callback() * - rd_kafka_produce() */ rd_kafka_resp_err_t rd_kafka_errno2err (int errnox); /******************************************************************* * * * Main configuration property interface * * * *******************************************************************/ /** * Configuration result type */ typedef enum { RD_KAFKA_CONF_UNKNOWN = -2, /* Unknown configuration name. */ RD_KAFKA_CONF_INVALID = -1, /* Invalid configuration value. */ RD_KAFKA_CONF_OK = 0, /* Configuration okay */ } rd_kafka_conf_res_t; /** * Create configuration object. * When providing your own configuration to the rd_kafka_*_new_*() calls * the rd_kafka_conf_t objects needs to be created with this function * which will set up the defaults. * I.e.: * * rd_kafka_conf_t *myconf; * rd_kafka_conf_res_t res; * * myconf = rd_kafka_conf_new(); * res = rd_kafka_conf_set(myconf, "socket.timeout.ms", "600", * errstr, sizeof(errstr)); * if (res != RD_KAFKA_CONF_OK) * die("%s\n", errstr); * * rk = rd_kafka_new(..., myconf); * * Please see CONFIGURATION.md for the default settings or use * `rd_kafka_conf_properties_show()` to provide the information at runtime. * * The properties are identical to the Apache Kafka configuration properties * whenever possible. */ rd_kafka_conf_t *rd_kafka_conf_new (void); /** * Destroys a conf object. */ void rd_kafka_conf_destroy (rd_kafka_conf_t *conf); /** * Creates a copy/duplicate of configuration object 'conf'. */ rd_kafka_conf_t *rd_kafka_conf_dup (const rd_kafka_conf_t *conf); /** * Sets a configuration property. * 'conf' must have been previously created with rd_kafka_conf_new(). * * Returns rd_kafka_conf_res_t to indicate success or failure. * In case of failure 'errstr' is updated to contain a human readable * error string. */ rd_kafka_conf_res_t rd_kafka_conf_set (rd_kafka_conf_t *conf, const char *name, const char *value, char *errstr, size_t errstr_size); /** * Producer: * Set delivery report callback in provided conf object. */ void rd_kafka_conf_set_dr_cb (rd_kafka_conf_t *conf, void (*dr_cb) (rd_kafka_t *rk, void *payload, size_t len, rd_kafka_resp_err_t err, void *opaque, void *msg_opaque)); /** * Set error callback in provided conf object. * The error callback is used by librdkafka to signal critical errors * back to the application. */ void rd_kafka_conf_set_error_cb (rd_kafka_conf_t *conf, void (*error_cb) (rd_kafka_t *rk, int err, const char *reason, void *opaque)); /** * Set statistics callback in provided conf object. * The statistics callback is called from `rd_kafka_poll()` every * `statistics.interval.ms` (needs to be configured separately). * Function arguments: * 'rk' - Kafka handle * 'json' - String containing the statistics data in JSON format * 'json_len' - Length of 'json' string. * 'opaque' - Application-provided opaque. * * If the application wishes to hold on to the 'json' pointer and free * it at a later time it must return 1 from the `stats_cb`. * If the application returns 0 from the `stats_cb` then librdkafka * will immediately free the 'json' pointer. */ void rd_kafka_conf_set_stats_cb (rd_kafka_conf_t *conf, int (*stats_cb) (rd_kafka_t *rk, char *json, size_t json_len, void *opaque)); /** * Sets the application's opaque pointer that will be passed to `dr_cb` * and `error_cb_` callbacks as the 'opaque' argument. */ void rd_kafka_conf_set_opaque (rd_kafka_conf_t *conf, void *opaque); /** * Dump the configuration properties and values of `conf` to an array * with "key", "value" pairs. The number of entries in the array is * returned in `*cntp`. * * The dump must be freed with `rd_kafka_conf_dump_free()`. */ const char **rd_kafka_conf_dump (rd_kafka_conf_t *conf, size_t *cntp); /** * Dump the topic configuration properties and values of `conf` to an array * with "key", "value" pairs. The number of entries in the array is * returned in `*cntp`. * * The dump must be freed with `rd_kafka_conf_dump_free()`. */ const char **rd_kafka_topic_conf_dump (rd_kafka_topic_conf_t *conf, size_t *cntp); /** * Frees a configuration dump returned from `rd_kafka_conf_dump()` or * `rd_kafka_topic_conf_dump(). */ void rd_kafka_conf_dump_free (const char **arr, size_t cnt); /** * Prints a table to 'fp' of all supported configuration properties, * their default values as well as a description. */ void rd_kafka_conf_properties_show (FILE *fp); /******************************************************************* * * * Topic configuration property interface * * * *******************************************************************/ /** * Create topic configuration object * * Same semantics as for rd_kafka_conf_new(). */ rd_kafka_topic_conf_t *rd_kafka_topic_conf_new (void); /** * Creates a copy/duplicate of topic configuration object 'conf'. */ rd_kafka_topic_conf_t *rd_kafka_topic_conf_dup (const rd_kafka_topic_conf_t *conf); /** * Destroys a topic conf object. */ void rd_kafka_topic_conf_destroy (rd_kafka_topic_conf_t *topic_conf); /** * Sets a single rd_kafka_topic_conf_t value by property name. * 'topic_conf' should have been previously set up * with `rd_kafka_topic_conf_new()`. * * Returns rd_kafka_conf_res_t to indicate success or failure. */ rd_kafka_conf_res_t rd_kafka_topic_conf_set (rd_kafka_topic_conf_t *conf, const char *name, const char *value, char *errstr, size_t errstr_size); /** * Sets the application's opaque pointer that will be passed to all topic * callbacks as the 'rkt_opaque' argument. */ void rd_kafka_topic_conf_set_opaque (rd_kafka_topic_conf_t *conf, void *opaque); /** * Producer: * Set partitioner callback in provided topic conf object. * * The partitioner may be called in any thread at any time, * it may be called multiple times for the same message/key. * * Partitioner function constraints: * - MUST NOT call any rd_kafka_*() functions except: * rd_kafka_topic_partition_available() * - MUST NOT block or execute for prolonged periods of time. * - MUST return a value between 0 and partition_cnt-1, or the * special RD_KAFKA_PARTITION_UA value if partitioning * could not be performed. */ void rd_kafka_topic_conf_set_partitioner_cb (rd_kafka_topic_conf_t *topic_conf, int32_t (*partitioner) ( const rd_kafka_topic_t *rkt, const void *keydata, size_t keylen, int32_t partition_cnt, void *rkt_opaque, void *msg_opaque)); /** * Check if partition is available (has a leader broker). * * Returns 1 if the partition is available, else 0. * * NOTE: This function must only be called from inside a partitioner function. */ int rd_kafka_topic_partition_available (const rd_kafka_topic_t *rkt, int32_t partition); /******************************************************************* * * * Partitioners provided by rdkafka * * * *******************************************************************/ /** * Random partitioner. * This is the default partitioner. * * Returns a random partition between 0 and 'partition_cnt'-1. */ int32_t rd_kafka_msg_partitioner_random (const rd_kafka_topic_t *rkt, const void *key, size_t keylen, int32_t partition_cnt, void *opaque, void *msg_opaque); /******************************************************************* * * * Kafka object handle * * * *******************************************************************/ /** * Creates a new Kafka handle and starts its operation according to the * specified 'type'. * * 'conf' is an optional struct created with `rd_kafka_conf_new()` that will * be used instead of the default configuration. * The 'conf' object is freed by this function and shall not be destroyed * by the application. * See `rd_kafka_conf_set()` et.al for more information. * * 'errstr' must be a pointer to memory of at least size 'errstr_size' where * `rd_kafka_new()` may write a human readable error message in case the * creation of a new handle fails. In which case the function returns NULL. * * Returns the Kafka handle on success or NULL on error. * * To destroy the Kafka handle, use rd_kafka_destroy(). */ rd_kafka_t *rd_kafka_new (rd_kafka_type_t type, rd_kafka_conf_t *conf, char *errstr, size_t errstr_size); /** * Destroy Kafka handle. * */ void rd_kafka_destroy (rd_kafka_t *rk); /** * Returns Kafka handle name. */ const char *rd_kafka_name (const rd_kafka_t *rk); /** * Creates a new topic handle for topic named 'topic'. * * 'conf' is an optional configuration for the topic created with * `rd_kafka_topic_conf_new()` that will be used instead of the default * topic configuration. * The 'conf' object is freed by this function and shall not be destroyed * by the application. * See `rd_kafka_topic_conf_set()` et.al for more information. * * Returns the new topic handle or NULL on error (see `errno`). */ rd_kafka_topic_t *rd_kafka_topic_new (rd_kafka_t *rk, const char *topic, rd_kafka_topic_conf_t *conf); /** * Destroy topic handle previously created with `rd_kafka_topic_new()`. */ void rd_kafka_topic_destroy (rd_kafka_topic_t *rkt); /** * Returns the topic name. */ const char *rd_kafka_topic_name (const rd_kafka_topic_t *rkt); /** * Unassigned partition. * * The unassigned partition is used by the producer API for messages * that should be partitioned using the configured or default partitioner. */ #define RD_KAFKA_PARTITION_UA ((int32_t)-1) /******************************************************************* * * * Kafka messages * * * *******************************************************************/ /** * A Kafka message as returned by the `rd_kafka_consume*()` family * of functions. * * This object has two purposes: * - provide the application with a consumed message. ('err' == 0) * - report per-topic+partition consumer errors ('err' != 0) * * The application must check 'err' to decide what action to take. * * When the application is finished with a message it must call * `rd_kafka_message_destroy()`. */ typedef struct rd_kafka_message_s { rd_kafka_resp_err_t err; /* Non-zero for error signaling. */ rd_kafka_topic_t *rkt; /* Topic */ int32_t partition; /* Partition */ void *payload; /* err==0: Message payload * err!=0: Error string */ size_t len; /* err==0: Message payload length * err!=0: Error string length */ void *key; /* err==0: Optional message key */ size_t key_len; /* err==0: Optional message key length */ int64_t offset; /* Message offset (or offset for error * if err!=0 if applicable). */ void *_private; /* rdkafka private pointer: DO NOT MODIFY */ } rd_kafka_message_t; /** * Frees resources for 'rkmessage' and hands ownership back to rdkafka. */ void rd_kafka_message_destroy (rd_kafka_message_t *rkmessage); /** * Returns the error string for an errored rd_kafka_message_t or NULL if * there was no error. */ static inline const char * __attribute__((unused)) rd_kafka_message_errstr (const rd_kafka_message_t *rkmessage) { if (!rkmessage->err) return NULL; if (rkmessage->payload) return (const char *)rkmessage->payload; return rd_kafka_err2str(rkmessage->err); } /******************************************************************* * * * Consumer API * * * *******************************************************************/ #define RD_KAFKA_OFFSET_BEGINNING -2 /* Start consuming from beginning of * kafka partition queue: oldest msg */ #define RD_KAFKA_OFFSET_END -1 /* Start consuming from end of kafka * partition queue: next msg */ #define RD_KAFKA_OFFSET_STORED -1000 /* Start consuming from offset retrieved * from offset store */ /** * Start consuming messages for topic 'rkt' and 'partition' * at offset 'offset' which may either be a proper offset (0..N) * or one of the the special offsets: * `RD_KAFKA_OFFSET_BEGINNING` or `RD_KAFKA_OFFSET_END`. * * rdkafka will attempt to keep 'queued.min.messages' (config property) * messages in the local queue by repeatedly fetching batches of messages * from the broker until the threshold is reached. * * The application shall use one of the `rd_kafka_consume*()` functions * to consume messages from the local queue, each kafka message being * represented as a `rd_kafka_message_t *` object. * * `rd_kafka_consume_start()` must not be called multiple times for the same * topic and partition without stopping consumption first with * `rd_kafka_consume_stop()`. * * Returns 0 on success or -1 on error (see `errno`). */ int rd_kafka_consume_start (rd_kafka_topic_t *rkt, int32_t partition, int64_t offset); /** * Stop consuming messages for topic 'rkt' and 'partition', purging * all messages currently in the local queue. * * The application needs to be stop all consumers before calling * `rd_kafka_destroy()` on the main object handle. * * Returns 0 on success or -1 on error (see `errno`). */ int rd_kafka_consume_stop (rd_kafka_topic_t *rkt, int32_t partition); /** * Consume a single message from topic 'rkt' and 'partition'. * * 'timeout_ms' is maximum amount of time to wait for a message to be received. * Consumer must have been previously started with `rd_kafka_consume_start()`. * * Returns a message object on success and NULL on error. * The message object must be destroyed with `rd_kafka_message_destroy()` * when the application is done with it. * * Errors (when returning NULL): * ETIMEDOUT - 'timeout_ms' was reached with no new messages fetched. * ENOENT - 'rkt'+'partition' is unknown. * (no prior `rd_kafka_consume_start()` call) * * The returned message's '..->err' must be checked for errors. */ rd_kafka_message_t *rd_kafka_consume (rd_kafka_topic_t *rkt, int32_t partition, int timeout_ms); /** * Consume up to 'rkmessages_size' from topic 'rkt' and 'partition', * putting a pointer to each message in the application provided * array 'rkmessages' (of size 'rkmessages_size' entries). * * `rd_kafka_consume_batch()` provides higher throughput performance * than `rd_kafka_consume()`. * * 'timeout_ms' is the maximum amount of time to wait for all of * 'rkmessages_size' messages to be put into 'rkmessages'. * This differs somewhat from `rd_kafka_consume()`. * * Returns the number of rkmessages added in 'rkmessages', * or -1 on error (same error codes as for `rd_kafka_consume()`. */ ssize_t rd_kafka_consume_batch (rd_kafka_topic_t *rkt, int32_t partition, int timeout_ms, rd_kafka_message_t **rkmessages, size_t rkmessages_size); /** * Consumes messages from topic 'rkt' and 'partition', calling * the provided callback for each consumed messsage. * * `rd_kafka_consume_callback()` provides higher throughput performance * than both `rd_kafka_consume()` and `rd_kafka_consume_batch()`. * * 'timeout_ms' is the maximum amount of time to wait for one or more messages * to arrive. * * The provided 'consume_cb' function is called for each message, * the application must not call `rd_kafka_message_destroy()` on the provided * 'rkmessage'. * * The 'opaque' argument is passed to the 'consume_cb' as 'opaque'. * * Returns the number of messages processed or -1 on error. */ int rd_kafka_consume_callback (rd_kafka_topic_t *rkt, int32_t partition, int timeout_ms, void (*consume_cb) (rd_kafka_message_t *rkmessage, void *opaque), void *opaque); /** * Topic+partition offset store. * * If auto.commit.enable is true the offset is stored automatically prior to * returning of the message(s) in each of the rd_kafka_consume*() functions * above. */ /** * Store offset 'offset' for topic 'rkt' partition 'partition'. * The offset will be commited (written) to the offset store according * to `auto.commit.interval.ms`. * * NOTE: `auto.commit.enable` must be set to "false" when using this API. * * Returns RD_KAFKA_RESP_ERR_NO_ERROR on success or an error code on error. */ rd_kafka_resp_err_t rd_kafka_offset_store (rd_kafka_topic_t *rkt, int32_t partition, int64_t offset); /******************************************************************* * * * Producer API * * * *******************************************************************/ /** * Produce and send a single message to broker. * * 'rkt' is the target topic which must have been previously created with * `rd_kafka_topic_new()`. * * `rd_kafka_produce()` is an asynch non-blocking API. * * 'partition' is the target partition, either: * - RD_KAFKA_PARTITION_UA (unassigned) for * automatic partitioning using the topic's partitioner function, or * - a fixed partition (0..N) * * 'msgflags' is zero or more of the following flags OR:ed together: * RD_KAFKA_MSG_F_FREE - rdkafka will free(3) 'payload' when it is done * with it. * RD_KAFKA_MSG_F_COPY - the 'payload' data will be copied and the 'payload' * pointer will not be used by rdkafka after the * call returns. * * .._F_FREE and .._F_COPY are mutually exclusive. * * 'payload' is the message payload of size 'len' bytes. * * 'key' is an optional message key of size 'keylen' bytes, if non-NULL it * will be passed to the topic partitioner as well as be sent with the * message to the broker and passed on to the consumer. * * 'msg_opaque' is an optional application-provided per-message opaque * pointer that will provided in the delivery report callback (`dr_cb`) for * referencing this message. * * Returns 0 on success or -1 on error in which case errno is set accordingly: * ENOBUFS - maximum number of outstanding messages has been reached: * "queue.buffering.max.message" * (RD_KAFKA_RESP_ERR__QUEUE_FULL) * EMSGSIZE - message is larger than configured max size: * "messages.max.bytes". * (RD_KAFKA_RESP_ERR_MSG_SIZE_TOO_LARGE) * ESRCH - requested 'partition' is unknown in the Kafka cluster. * (RD_KAFKA_RESP_ERR__UNKNOWN_PARTITION) * ENOENT - topic is unknown in the Kafka cluster. * (RD_KAFKA_RESP_ERR__UNKNOWN_TOPIC) * * NOTE: Use `rd_kafka_errno2err()` to convert `errno` to rdkafka error code. */ #define RD_KAFKA_MSG_F_FREE 0x1 /* Delegate freeing of payload to rdkafka. */ #define RD_KAFKA_MSG_F_COPY 0x2 /* rdkafka will make a copy of the payload. */ int rd_kafka_produce (rd_kafka_topic_t *rkt, int32_t partitition, int msgflags, void *payload, size_t len, const void *key, size_t keylen, void *msg_opaque); /******************************************************************* * * * Misc API * * * *******************************************************************/ /** * Polls the provided kafka handle for events. * * Events will cause application provided callbacks to be called. * * The 'timeout_ms' argument specifies the minimum amount of time * (in milliseconds) that the call will block waiting for events. * For non-blocking calls, provide 0 as 'timeout_ms'. * To wait indefinately for an event, provide -1. * * Events: * - delivery report callbacks (if dr_cb is configured) [producer] * - error callbacks (if error_cb is configured) [producer & consumer] * * Returns the number of events served. */ int rd_kafka_poll (rd_kafka_t *rk, int timeout_ms); /** * Adds a one or more brokers to the kafka handle's list of initial brokers. * Additional brokers will be discovered automatically as soon as rdkafka * connects to a broker by querying the broker metadata. * * If a broker name resolves to multiple addresses (and possibly * address families) all will be used for connection attempts in * round-robin fashion. * * 'brokerlist' is a ,-separated list of brokers in the format: * [:],[:]... * * Returns the number of brokers successfully added. * * NOTE: Brokers may also be defined with the 'metadata.broker.list' * configuration property. */ int rd_kafka_brokers_add (rd_kafka_t *rk, const char *brokerlist); /** * Set logger function. * The default is to print to stderr, but a syslog logger is also available, * see rd_kafka_log_(print|syslog) for the builtin alternatives. * Alternatively the application may provide its own logger callback. * Or pass 'func' as NULL to disable logging. * * NOTE: 'rk' may be passed as NULL. */ void rd_kafka_set_logger (rd_kafka_t *rk, void (*func) (const rd_kafka_t *rk, int level, const char *fac, const char *buf)); /** * Specifies the maximum logging level produced by * internal kafka logging and debugging. * If the 'debug' configuration property is set the level is automatically * adjusted to LOG_DEBUG (7). */ void rd_kafka_set_log_level (rd_kafka_t *rk, int level); /** * Builtin (default) log sink: print to stderr */ void rd_kafka_log_print (const rd_kafka_t *rk, int level, const char *fac, const char *buf); /** * Builtin log sink: print to syslog. */ void rd_kafka_log_syslog (const rd_kafka_t *rk, int level, const char *fac, const char *buf); /** * Returns the current out queue length: * messages waiting to be sent to, or acknowledged by, the broker. */ int rd_kafka_outq_len (rd_kafka_t *rk); /** * Dumps rdkafka's internal state for handle 'rk' to stream 'fp' * This is only useful for debugging rdkafka, showing state and statistics * for brokers, topics, partitions, etc. */ void rd_kafka_dump (FILE *fp, rd_kafka_t *rk); /** * Retrieve the current number of threads in use by librdkafka. * Used by regression tests. */ int rd_kafka_thread_cnt (void); /** * Wait for all rd_kafka_t objects to be destroyed. * Returns 0 if all kafka objects are now destroyed, or -1 if the * timeout was reached. * Since `rd_kafka_destroy()` is an asynch operation the * `rd_kafka_wait_destroyed()` function can be used for applications where * a clean shutdown is required. */ int rd_kafka_wait_destroyed (int timeout_ms); librdkafka-0.8.3/rdkafka_broker.c000066400000000000000000002660621227620010100167320ustar00rootroot00000000000000/* * librdkafka - Apache Kafka C library * * Copyright (c) 2012, Magnus Edenhill * All rights reserved. * * Redistribution and use in source and binary forms, with or without * modification, are permitted provided that the following conditions are met: * * 1. Redistributions of source code must retain the above copyright notice, * this list of conditions and the following disclaimer. * 2. Redistributions in binary form must reproduce the above copyright notice, * this list of conditions and the following disclaimer in the documentation * and/or other materials provided with the distribution. * * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE * POSSIBILITY OF SUCH DAMAGE. */ #define __need_IOV_MAX #define _GNU_SOURCE #include #include #include #include #include #include #include "rd.h" #include "rdkafka_int.h" #include "rdkafka_msg.h" #include "rdkafka_topic.h" #include "rdkafka_broker.h" #include "rdkafka_offset.h" #include "rdtime.h" #include "rdthread.h" #include "rdcrc32.h" #include "rdrand.h" #include "rdgz.h" #include "snappy.h" #include "endian_compat.h" const char *rd_kafka_broker_state_names[] = { "INIT", "DOWN", "UP" }; static void rd_kafka_broker_update (rd_kafka_t *rk, const char *name, uint16_t port, uint32_t nodeid); static int rd_kafka_send (rd_kafka_broker_t *rkb); static void msghdr_print (rd_kafka_t *rk, const char *what, const struct msghdr *msg, int hexdump) RD_UNUSED; static void msghdr_print (rd_kafka_t *rk, const char *what, const struct msghdr *msg, int hexdump) { int i; rd_kafka_dbg(rk, MSG, "MSG", "%s: iovlen %zd", what, (size_t)msg->msg_iovlen); for (i = 0 ; i < msg->msg_iovlen ; i++) { rd_kafka_dbg(rk, MSG, what, " iov #%i: %zd", i, msg->msg_iov[i].iov_len); rd_hexdump(stdout, what, msg->msg_iov[i].iov_base, msg->msg_iov[i].iov_len); } } static size_t rd_kafka_msghdr_size (const struct msghdr *msg) { int i; size_t tot = 0; for (i = 0 ; i < msg->msg_iovlen ; i++) tot += msg->msg_iov[i].iov_len; return tot; } /** * Locks: rd_kafka_broker_lock() MUST be held. * Locality: broker thread */ static void rd_kafka_broker_set_state (rd_kafka_broker_t *rkb, int state) { if (rkb->rkb_state == state) return; rd_kafka_dbg(rkb->rkb_rk, BROKER, "STATE", "%s: Broker changed state %s -> %s", rkb->rkb_name, rd_kafka_broker_state_names[rkb->rkb_state], rd_kafka_broker_state_names[state]); if (state == RD_KAFKA_BROKER_STATE_DOWN) { /* Propagate ALL_BROKERS_DOWN event if all brokers are * now down, unless we're terminating. */ if (rd_atomic_add(&rkb->rkb_rk->rk_broker_down_cnt, 1) == rkb->rkb_rk->rk_broker_cnt && !rkb->rkb_rk->rk_terminate) rd_kafka_op_err(rkb->rkb_rk, RD_KAFKA_RESP_ERR__ALL_BROKERS_DOWN, "%i/%i brokers are down", rkb->rkb_rk->rk_broker_down_cnt, rkb->rkb_rk->rk_broker_cnt); } else if (rkb->rkb_state == RD_KAFKA_BROKER_STATE_DOWN) rd_atomic_sub(&rkb->rkb_rk->rk_broker_down_cnt, 1); rkb->rkb_state = state; } void rd_kafka_buf_destroy (rd_kafka_buf_t *rkbuf) { if (rd_atomic_sub(&rkbuf->rkbuf_refcnt, 1) > 0) return; if (rkbuf->rkbuf_buf2) free(rkbuf->rkbuf_buf2); if (rkbuf->rkbuf_flags & RD_KAFKA_OP_F_FREE && rkbuf->rkbuf_buf) free(rkbuf->rkbuf_buf); free(rkbuf); } static void rd_kafka_buf_auxbuf_add (rd_kafka_buf_t *rkbuf, void *auxbuf) { assert(rkbuf->rkbuf_buf2 == NULL); rkbuf->rkbuf_buf2 = auxbuf; } static void rd_kafka_buf_rewind (rd_kafka_buf_t *rkbuf, int iovindex) { rkbuf->rkbuf_msg.msg_iovlen = iovindex; } static struct iovec *rd_kafka_buf_iov_next (rd_kafka_buf_t *rkbuf) { assert(rkbuf->rkbuf_msg.msg_iovlen + 1 <= rkbuf->rkbuf_iovcnt); return &rkbuf->rkbuf_iov[rkbuf->rkbuf_msg.msg_iovlen++]; } /** * Pushes 'buf' & 'len' onto the previously allocated iov stack for 'rkbuf'. */ static void rd_kafka_buf_push (rd_kafka_buf_t *rkbuf, void *buf, size_t len) { struct iovec *iov; iov = rd_kafka_buf_iov_next(rkbuf); iov->iov_base = buf; iov->iov_len = len; } #define RD_KAFKA_HEADERS_IOV_CNT 2 #define RD_KAFKA_PAYLOAD_IOV_MAX (IOV_MAX-RD_KAFKA_HEADERS_IOV_CNT) static rd_kafka_buf_t *rd_kafka_buf_new (int iovcnt, size_t size) { rd_kafka_buf_t *rkbuf; const int iovcnt_fixed = RD_KAFKA_HEADERS_IOV_CNT; size_t iovsize = sizeof(struct iovec) * (iovcnt+iovcnt_fixed); size_t fullsize = iovsize + sizeof(*rkbuf) + size; rkbuf = malloc(fullsize); memset(rkbuf, 0, sizeof(*rkbuf)); rkbuf->rkbuf_iov = (struct iovec *)(rkbuf+1); rkbuf->rkbuf_iovcnt = (iovcnt+iovcnt_fixed); assert(rkbuf->rkbuf_iovcnt <= IOV_MAX); rkbuf->rkbuf_msg.msg_iov = rkbuf->rkbuf_iov; /* save the first two iovecs for the header + clientid */ rkbuf->rkbuf_msg.msg_iovlen = iovcnt_fixed; memset(rkbuf->rkbuf_iov, 0, sizeof(*rkbuf->rkbuf_iov) * iovcnt_fixed); rkbuf->rkbuf_size = size; rkbuf->rkbuf_buf = ((char *)(rkbuf+1))+iovsize; rd_kafka_msgq_init(&rkbuf->rkbuf_msgq); rd_kafka_buf_keep(rkbuf); return rkbuf; } /** * Create new rkbuf shadowing a memory region in rkbuf_buf2. */ static rd_kafka_buf_t *rd_kafka_buf_new_shadow (void *ptr, size_t size) { rd_kafka_buf_t *rkbuf; rkbuf = calloc(1, sizeof(*rkbuf)); rkbuf->rkbuf_buf2 = ptr; rkbuf->rkbuf_len = size; rd_kafka_msgq_init(&rkbuf->rkbuf_msgq); rd_kafka_buf_keep(rkbuf); return rkbuf; } static void rd_kafka_bufq_enq (rd_kafka_bufq_t *rkbufq, rd_kafka_buf_t *rkbuf) { TAILQ_INSERT_TAIL(&rkbufq->rkbq_bufs, rkbuf, rkbuf_link); (void)rd_atomic_add(&rkbufq->rkbq_cnt, 1); } static void rd_kafka_bufq_deq (rd_kafka_bufq_t *rkbufq, rd_kafka_buf_t *rkbuf) { TAILQ_REMOVE(&rkbufq->rkbq_bufs, rkbuf, rkbuf_link); assert(rkbufq->rkbq_cnt > 0); (void)rd_atomic_sub(&rkbufq->rkbq_cnt, 1); } static void rd_kafka_bufq_init (rd_kafka_bufq_t *rkbufq) { TAILQ_INIT(&rkbufq->rkbq_bufs); rkbufq->rkbq_cnt = 0; } /** * Concat all buffers from 'src' to tail of 'dst' */ static void rd_kafka_bufq_concat (rd_kafka_bufq_t *dst, rd_kafka_bufq_t *src) { TAILQ_CONCAT(&dst->rkbq_bufs, &src->rkbq_bufs, rkbuf_link); (void)rd_atomic_add(&dst->rkbq_cnt, src->rkbq_cnt); rd_kafka_bufq_init(src); } /** * Purge the wait-response queue. * NOTE: 'rkbufq' must be a temporary queue and not one of rkb_waitresps * or rkb_outbufs since buffers may be re-enqueued on those queues. */ static void rd_kafka_bufq_purge (rd_kafka_broker_t *rkb, rd_kafka_bufq_t *rkbufq, rd_kafka_resp_err_t err) { rd_kafka_buf_t *rkbuf, *tmp; assert(pthread_self() == rkb->rkb_thread); rd_rkb_dbg(rkb, QUEUE, "BUFQ", "Purging bufq with %i buffers", rkbufq->rkbq_cnt); TAILQ_FOREACH_SAFE(rkbuf, &rkbufq->rkbq_bufs, rkbuf_link, tmp) rkbuf->rkbuf_cb(rkb, err, NULL, rkbuf, rkbuf->rkbuf_opaque); } /** * Scan the wait-response queue for message timeouts. */ static void rd_kafka_broker_waitresp_timeout_scan (rd_kafka_broker_t *rkb, rd_ts_t now) { rd_kafka_buf_t *rkbuf, *tmp; int cnt = 0; assert(pthread_self() == rkb->rkb_thread); TAILQ_FOREACH_SAFE(rkbuf, &rkb->rkb_waitresps.rkbq_bufs, rkbuf_link, tmp) { if (likely(rkbuf->rkbuf_ts_timeout > now)) continue; rd_kafka_bufq_deq(&rkb->rkb_waitresps, rkbuf); rkbuf->rkbuf_cb(rkb, RD_KAFKA_RESP_ERR__MSG_TIMED_OUT, NULL, rkbuf, rkbuf->rkbuf_opaque); cnt++; } if (cnt > 0) rd_rkb_dbg(rkb, MSG, "REQTMOUT", "Timed out %i requests", cnt); } /** * Failure propagation to application. * Will tear down connection to broker and trigger a reconnect. * * If 'fmt' is NULL nothing will be logged or propagated to the application. * * Locality: Broker thread */ static void rd_kafka_broker_fail (rd_kafka_broker_t *rkb, rd_kafka_resp_err_t err, const char *fmt, ...) { va_list ap; int errno_save = errno; rd_kafka_toppar_t *rktp; rd_kafka_bufq_t tmpq; assert(pthread_self() == rkb->rkb_thread); rd_kafka_broker_lock(rkb); rd_kafka_dbg(rkb->rkb_rk, BROKER, "BROKERFAIL", "%s: failed: err: %s: (errno: %s)", rkb->rkb_name, rd_kafka_err2str(err), strerror(errno_save)); rkb->rkb_err.err = errno_save; if (rkb->rkb_s != -1) { close(rkb->rkb_s); rkb->rkb_s = -1; rkb->rkb_pfd.fd = rkb->rkb_s; } if (rkb->rkb_recv_buf) { rd_kafka_buf_destroy(rkb->rkb_recv_buf); rkb->rkb_recv_buf = NULL; } /* The caller may omit the format if it thinks this is a recurring * failure, in which case the following things are omitted: * - log message * - application OP_ERR * - metadata request */ if (fmt) { int of; /* Insert broker name in log message if it fits. */ of = snprintf(rkb->rkb_err.msg, sizeof(rkb->rkb_err.msg), "%s: ", rkb->rkb_name); if (of >= sizeof(rkb->rkb_err.msg)) of = 0; va_start(ap, fmt); vsnprintf(rkb->rkb_err.msg+of, sizeof(rkb->rkb_err.msg)-of, fmt, ap); va_end(ap); rd_kafka_log(rkb->rkb_rk, LOG_ERR, "FAIL", "%s", rkb->rkb_err.msg); /* Send ERR op back to application for processing. */ rd_kafka_op_err(rkb->rkb_rk, err, "%s", rkb->rkb_err.msg); } /* Set broker state */ rd_kafka_broker_set_state(rkb, RD_KAFKA_BROKER_STATE_DOWN); /* * Purge all buffers * (put on a temporary queue since bufs may be requeued) */ rd_kafka_bufq_init(&tmpq); rd_kafka_bufq_concat(&tmpq, &rkb->rkb_waitresps); rd_kafka_bufq_concat(&tmpq, &rkb->rkb_outbufs); /* Unlock broker since a requeue will try to lock it. */ rd_kafka_broker_unlock(rkb); /* Purge the buffers */ rd_kafka_bufq_purge(rkb, &tmpq, err); /* Undelegate all toppars from this broker. */ rd_kafka_broker_toppars_wrlock(rkb); while ((rktp = TAILQ_FIRST(&rkb->rkb_toppars))) { rd_kafka_topic_t *rkt = rktp->rktp_rkt; rd_kafka_topic_keep(rkt); /* Hold on to rkt */ rd_kafka_toppar_keep(rktp); rd_kafka_broker_toppars_unlock(rkb); rd_rkb_dbg(rkb, TOPIC, "BRKTP", "Undelegating %.*s [%"PRId32"]", RD_KAFKAP_STR_PR(rktp->rktp_rkt->rkt_topic), rktp->rktp_partition); rd_kafka_topic_wrlock(rktp->rktp_rkt); /* Undelegate */ rd_kafka_toppar_broker_delegate(rktp, NULL); rd_kafka_topic_unlock(rktp->rktp_rkt); rd_kafka_toppar_destroy(rktp); rd_kafka_topic_destroy0(rkt); /* Let go of rkt */ rd_kafka_broker_toppars_wrlock(rkb); } rd_kafka_broker_toppars_unlock(rkb); /* Query for the topic leaders (async) */ if (fmt && err != RD_KAFKA_RESP_ERR__DESTROY) rd_kafka_topic_leader_query(rkb->rkb_rk, NULL); } static ssize_t rd_kafka_broker_send (rd_kafka_broker_t *rkb, const struct msghdr *msg) { ssize_t r; assert(rkb->rkb_state >= RD_KAFKA_BROKER_STATE_UP); assert(rkb->rkb_s != -1); r = sendmsg(rkb->rkb_s, msg, MSG_DONTWAIT #ifdef MSG_NOSIGNAL |MSG_NOSIGNAL #endif ); if (r == -1) { if (errno == EAGAIN) return 0; rd_kafka_dbg(rkb->rkb_rk, BROKER, "BRKSEND", "sendmsg FAILED for iovlen %zd (%i)", (size_t)msg->msg_iovlen, IOV_MAX); rd_kafka_broker_fail(rkb, RD_KAFKA_RESP_ERR__TRANSPORT, "Send failed: %s", strerror(errno)); rkb->rkb_c.tx_err++; return -1; } (void)rd_atomic_add(&rkb->rkb_c.tx_bytes, r); (void)rd_atomic_add(&rkb->rkb_c.tx, 1); return r; } static int rd_kafka_broker_resolve (rd_kafka_broker_t *rkb) { const char *errstr; if (rkb->rkb_rsal && rkb->rkb_t_rsal_last + rkb->rkb_rk->rk_conf.broker_addr_ttl < time(NULL)) { /* Address list has expired. */ rd_sockaddr_list_destroy(rkb->rkb_rsal); rkb->rkb_rsal = NULL; } if (!rkb->rkb_rsal) { /* Resolve */ rkb->rkb_rsal = rd_getaddrinfo(rkb->rkb_nodename, RD_KAFKA_PORT_STR, AI_ADDRCONFIG, AF_UNSPEC, SOCK_STREAM, IPPROTO_TCP, &errstr); if (!rkb->rkb_rsal) { char tmp[512]; snprintf(tmp, sizeof(tmp), "Failed to resolve '%s': %s", rkb->rkb_nodename, errstr); /* Send ERR op back to application for processing. */ rd_kafka_op_err(rkb->rkb_rk,RD_KAFKA_RESP_ERR__RESOLVE, "%s", tmp); rd_rkb_log(rkb, LOG_ERR, "GETADDR", "%s", tmp); return -1; } } return 0; } static void rd_kafka_broker_buf_enq0 (rd_kafka_broker_t *rkb, rd_kafka_buf_t *rkbuf, int at_head) { assert(pthread_self() == rkb->rkb_thread); if (unlikely(at_head)) { /* Insert message at head of queue */ rd_kafka_buf_t *prev; /* Put us behind any flash messages. */ TAILQ_FOREACH(prev, &rkb->rkb_outbufs.rkbq_bufs, rkbuf_link) if (!(prev->rkbuf_flags & RD_KAFKA_OP_F_FLASH)) break; if (prev) TAILQ_INSERT_AFTER(&rkb->rkb_outbufs.rkbq_bufs, prev, rkbuf, rkbuf_link); else TAILQ_INSERT_HEAD(&rkb->rkb_outbufs.rkbq_bufs, rkbuf, rkbuf_link); } else { /* Insert message at tail of queue */ TAILQ_INSERT_TAIL(&rkb->rkb_outbufs.rkbq_bufs, rkbuf, rkbuf_link); } (void)rd_atomic_add(&rkb->rkb_outbufs.rkbq_cnt, 1); } static void rd_kafka_broker_buf_enq1 (rd_kafka_broker_t *rkb, int16_t ApiKey, rd_kafka_buf_t *rkbuf, void (*reply_cb) ( rd_kafka_broker_t *, rd_kafka_resp_err_t err, rd_kafka_buf_t *reply, rd_kafka_buf_t *request, void *opaque), void *opaque) { assert(pthread_self() == rkb->rkb_thread); rkbuf->rkbuf_corrid = ++rkb->rkb_corrid; /* Header */ rkbuf->rkbuf_reqhdr.ApiKey = htons(ApiKey); rkbuf->rkbuf_reqhdr.ApiVersion = 0; rkbuf->rkbuf_reqhdr.CorrId = htonl(rkbuf->rkbuf_corrid); rkbuf->rkbuf_iov[0].iov_base = &rkbuf->rkbuf_reqhdr; rkbuf->rkbuf_iov[0].iov_len = sizeof(rkbuf->rkbuf_reqhdr); /* Header ClientId */ rkbuf->rkbuf_iov[1].iov_base = rkb->rkb_rk->rk_clientid; rkbuf->rkbuf_iov[1].iov_len = RD_KAFKAP_STR_SIZE(rkb->rkb_rk->rk_clientid); rkbuf->rkbuf_cb = reply_cb; rkbuf->rkbuf_opaque = opaque; rkbuf->rkbuf_ts_enq = rd_clock(); /* Calculate total message buffer length. */ rkbuf->rkbuf_of = 0; rkbuf->rkbuf_len = rd_kafka_msghdr_size(&rkbuf->rkbuf_msg); rkbuf->rkbuf_reqhdr.Size = ntohl(rkbuf->rkbuf_len-4); rd_kafka_broker_buf_enq0(rkb, rkbuf, (rkbuf->rkbuf_flags & RD_KAFKA_OP_F_FLASH)? 1/*head*/: 0/*tail*/); } static void rd_kafka_broker_buf_enq (rd_kafka_broker_t *rkb, int16_t ApiKey, char *buf, int32_t size, int flags, void (*reply_cb) ( rd_kafka_broker_t *, int err, rd_kafka_buf_t *reply, rd_kafka_buf_t *request, void *opaque), void *opaque) { rd_kafka_buf_t *rkbuf; rkbuf = rd_kafka_buf_new(1, flags & RD_KAFKA_OP_F_FREE ? 0 : size); rkbuf->rkbuf_ts_timeout = rd_clock() + rkb->rkb_rk->rk_conf.socket_timeout_ms * 1000; rkbuf->rkbuf_flags |= flags; if (size > 0) { if (!(flags & RD_KAFKA_OP_F_FREE)) { /* Duplicate data */ memcpy(rkbuf->rkbuf_buf, buf, size); buf = rkbuf->rkbuf_buf; } else { rkbuf->rkbuf_buf = buf; rkbuf->rkbuf_size = size; } rd_kafka_buf_push(rkbuf, buf, size); } rd_kafka_broker_buf_enq1(rkb, ApiKey, rkbuf, reply_cb, opaque); } /** * Memory reading helper macros to be used when parsing network responses. * * Assumptions: * - data base pointer is in 'char *buf' * - data total size is in 'size_t size' * - current read offset is in 'size_t of' which must be initialized to 0. * - the broker the message was received from must be 'rkb' * - an 'err:' label must be available for error bailouts. */ #define _FAIL(fmt...) do { \ rd_rkb_dbg(rkb, BROKER, "PROTOERR", \ "Protocol parse failure at %s:%i", \ __FUNCTION__, __LINE__); \ rd_rkb_dbg(rkb, BROKER, "PROTOERR", fmt); \ goto err; \ } while (0) #define _REMAIN() (size - of) #define _CHECK_LEN(len) do { \ int _LEN = (int)(len); \ if (unlikely(_LEN > _REMAIN())) { \ _FAIL("expected %i bytes > %i remaining bytes", \ _LEN, (int)_REMAIN()); \ goto err; \ } \ } while (0) #define _SKIP(len) do { \ _CHECK_LEN(len); \ of += (len); \ } while (0) #define _READ(dstptr,len) do { \ _CHECK_LEN(len); \ memcpy((dstptr), buf+(of), (len)); \ of += (len); \ } while (0) #define _READ_I64(dstptr) do { \ _READ(dstptr, 8); \ *(int64_t *)(dstptr) = be64toh(*(int64_t *)(dstptr)); \ } while (0) #define _READ_I32(dstptr) do { \ _READ(dstptr, 4); \ *(int32_t *)(dstptr) = ntohl(*(int32_t *)(dstptr)); \ } while (0) #define _READ_I16(dstptr) do { \ _READ(dstptr, 2); \ *(int16_t *)(dstptr) = ntohs(*(int16_t *)(dstptr)); \ } while (0) /* Read Kafka String representation (2+N) */ #define _READ_STR(kstr) do { \ int _klen; \ _CHECK_LEN(2); \ kstr = (rd_kafkap_str_t *)((char *)buf+of); \ _klen = RD_KAFKAP_STR_SIZE(kstr); \ of += _klen; \ } while (0) /* Read Kafka Bytes representation (4+N) */ #define _READ_BYTES(kbytes) do { \ int32_t _klen; \ _CHECK_LEN(4); \ kbytes = (rd_kafkap_bytes_t *)((char *)buf+of); \ _klen = RD_KAFKAP_BYTES_SIZE(kbytes); \ of += (_klen); \ } while (0) /* Reference memory, dont copy it */ #define _READ_REF(dstptr,len) do { \ _CHECK_LEN(len); \ (dstptr) = (void *)((char *)buf+of); \ of += (len); \ } while(0) /** * Handle a Metadata response message. * If 'rkt' is non-NULL the metadata originated from a topic-specific request. * * Locality: broker thread */ static void rd_kafka_metadata_handle (rd_kafka_broker_t *rkb, rd_kafka_topic_t *req_rkt, const char *buf, size_t size) { struct { int32_t NodeId; rd_kafkap_str_t *Host; int32_t Port; } *Brokers = NULL; int32_t Broker_cnt; struct rd_kafka_TopicMetadata *TopicMetadata = NULL; int32_t TopicMetadata_cnt; int i, j, k; int of = 0; int req_rkt_seen = 0; /* Read Brokers */ _READ_I32(&Broker_cnt); if (Broker_cnt > RD_KAFKAP_BROKERS_MAX) _FAIL("Broker_cnt %"PRId32" > BROKERS_MAX %i", Broker_cnt, RD_KAFKAP_BROKERS_MAX); Brokers = malloc(sizeof(*Brokers) * Broker_cnt); for (i = 0 ; i < Broker_cnt ; i++) { _READ_I32(&Brokers[i].NodeId); _READ_STR(Brokers[i].Host); _READ_I32(&Brokers[i].Port); } /* Read TopicMetadata */ _READ_I32(&TopicMetadata_cnt); rd_rkb_dbg(rkb, METADATA, "METADATA", "%"PRId32" brokers, " "%"PRId32" topics", Broker_cnt, TopicMetadata_cnt); if (TopicMetadata_cnt > RD_KAFKAP_TOPICS_MAX) _FAIL("TopicMetadata_cnt %"PRId32" > TOPICS_MAX %i", TopicMetadata_cnt, RD_KAFKAP_TOPICS_MAX); TopicMetadata = malloc(sizeof(*TopicMetadata) * TopicMetadata_cnt); for (i = 0 ; i < TopicMetadata_cnt ; i++) { _READ_I16(&TopicMetadata[i].ErrorCode); _READ_STR(TopicMetadata[i].Name); /* PartitionMetadata */ _READ_I32(&TopicMetadata[i].PartitionMetadata_cnt); if (TopicMetadata[i].PartitionMetadata_cnt > RD_KAFKAP_PARTITIONS_MAX) _FAIL("TopicMetadata[%i].PartitionMetadata_cnt " "%"PRId32" > PARTITIONS_MAX %i", i, TopicMetadata[i].PartitionMetadata_cnt, RD_KAFKAP_PARTITIONS_MAX); TopicMetadata[i].PartitionMetadata = alloca(sizeof(*TopicMetadata[i].PartitionMetadata) * TopicMetadata[i].PartitionMetadata_cnt); for (j = 0 ; j < TopicMetadata[i].PartitionMetadata_cnt ; j++) { _READ_I16(&TopicMetadata[i].PartitionMetadata[j]. ErrorCode); _READ_I32(&TopicMetadata[i].PartitionMetadata[j]. PartitionId); _READ_I32(&TopicMetadata[i].PartitionMetadata[j]. Leader); /* Replicas */ _READ_I32(&TopicMetadata[i].PartitionMetadata[j]. Replicas_cnt); if (TopicMetadata[i].PartitionMetadata[j].Replicas_cnt > RD_KAFKAP_BROKERS_MAX) _FAIL("TopicMetadata[%i]." "PartitionMetadata[%i].Replicas_cnt " "%"PRId32" > BROKERS_MAX %i", i, j, TopicMetadata[i].PartitionMetadata[j]. Replicas_cnt, RD_KAFKAP_BROKERS_MAX); TopicMetadata[i].PartitionMetadata[j].Replicas = alloca(sizeof(*TopicMetadata[i]. PartitionMetadata[j].Replicas) * TopicMetadata[i].PartitionMetadata[j]. Replicas_cnt); for (k = 0 ; k < TopicMetadata[i].PartitionMetadata[j]. Replicas_cnt ; k++) _READ_I32(&TopicMetadata[i]. PartitionMetadata[j].Replicas[k]); /* Isr */ _READ_I32(&TopicMetadata[i].PartitionMetadata[j]. Isr_cnt); if (TopicMetadata[i].PartitionMetadata[j].Isr_cnt > RD_KAFKAP_BROKERS_MAX) _FAIL("TopicMetadata[%i]." "PartitionMetadata[%i].Isr_cnt %"PRId32 " > BROKERS_MAX %i", i, j, TopicMetadata[i]. PartitionMetadata[j].Isr_cnt, RD_KAFKAP_BROKERS_MAX); TopicMetadata[i].PartitionMetadata[j].Isr = alloca(sizeof(*TopicMetadata[i]. PartitionMetadata[j].Isr) * TopicMetadata[i]. PartitionMetadata[j].Isr_cnt); for (k = 0 ; k < TopicMetadata[i].PartitionMetadata[j]. Isr_cnt ; k++) _READ_I32(&TopicMetadata[i]. PartitionMetadata[j].Isr[k]); } } /* Update our list of brokers. */ for (i = 0 ; i < Broker_cnt ; i++) { rd_rkb_dbg(rkb, METADATA, "METADATA", " Broker #%i/%i: %.*s:%"PRId32" NodeId %"PRId32, i, Broker_cnt, RD_KAFKAP_STR_PR(Brokers[i].Host), Brokers[i].Port, Brokers[i].NodeId); rd_kafka_broker_update(rkb->rkb_rk, rd_kafkap_strdupa(Brokers[i].Host), Brokers[i].Port, Brokers[i].NodeId); } /* Update partition count and leader for each topic we know about */ for (i = 0 ; i < TopicMetadata_cnt ; i++) { if (req_rkt && !rd_kafkap_str_cmp(TopicMetadata[i].Name, req_rkt->rkt_topic)) req_rkt_seen++; rd_kafka_topic_metadata_update(rkb, &TopicMetadata[i]); } /* Requested topics not seen in metadata? Propogate to topic code. */ if (req_rkt) { rd_rkb_dbg(rkb, TOPIC, "METADATA", "Requested topic %s %sseen in metadata", req_rkt->rkt_topic->str, req_rkt_seen ? "" : "not "); if (!req_rkt_seen) rd_kafka_topic_metadata_none(req_rkt); } err: if (Brokers) free(Brokers); if (TopicMetadata) free(TopicMetadata); } static void rd_kafka_broker_metadata_reply (rd_kafka_broker_t *rkb, int err, rd_kafka_buf_t *reply, rd_kafka_buf_t *request, void *opaque) { rd_kafka_topic_t *rkt = opaque; rd_rkb_dbg(rkb, METADATA, "METADATA", "===== Received metadata from %s =====", rkb->rkb_name); /* Avoid metadata updates when we're terminating. */ if (rkb->rkb_rk->rk_terminate) goto done; if (unlikely(err)) { /* FIXME: handle error */ rd_rkb_log(rkb, LOG_WARNING, "METADATA", "Metadata request failed: %s", rd_kafka_err2str(err)); } else { rd_kafka_metadata_handle(rkb, rkt, reply->rkbuf_buf2, reply->rkbuf_len); } done: if (rkt) { rd_kafka_topic_wrlock(rkt); rkt->rkt_flags &= ~RD_KAFKA_TOPIC_F_LEADER_QUERY; rd_kafka_topic_unlock(rkt); rd_kafka_topic_destroy0(rkt); } rd_kafka_buf_destroy(request); if (reply) rd_kafka_buf_destroy(reply); } /** * all_topics := if 1: retreive all topics&partitions from the broker * if 0: just retrieve the topics we know about. * rkt := all_topics=0 && only_rkt is set: only ask for specified topic. */ static void rd_kafka_broker_metadata_req (rd_kafka_broker_t *rkb, int all_topics, rd_kafka_topic_t *only_rkt, const char *reason) { char *buf; size_t of = 0; int32_t arrsize = 0; size_t tnamelen = 0; rd_kafka_topic_t *rkt; rd_rkb_dbg(rkb, METADATA, "METADATA", "Request metadata for %s: %s", only_rkt ? only_rkt->rkt_topic->str : (all_topics ? "all topics" : "locally known topics"), reason ? : ""); /* If called from other thread than the broker's own then post an * op for the broker's thread instead since all transmissions must * be performed by the broker thread. */ if (pthread_self() != rkb->rkb_thread) { rd_kafka_op_t *rko = rd_kafka_op_new(RD_KAFKA_OP_METADATA_REQ); if (only_rkt) { rko->rko_rkt = only_rkt; rd_kafka_topic_keep(only_rkt); } rd_kafka_q_enq(&rkb->rkb_ops, rko); rd_rkb_dbg(rkb, METADATA, "METADATA", "Request metadata: scheduled: not in broker thread"); return; } /* FIXME: Use iovs and ..next here */ rd_rkb_dbg(rkb, METADATA, "METADATA", "Requesting metadata for %stopics", all_topics ? "all ": "known "); if (!only_rkt) { /* Push the next intervalled metadata refresh forward since * we are performing one now (which might be intervalled). */ if (rkb->rkb_rk->rk_conf.metadata_refresh_interval_ms >= 0) { if (rkb->rkb_metadata_fast_poll_cnt > 0) { /* Fast poll after topic loosings its leader */ rkb->rkb_metadata_fast_poll_cnt--; rkb->rkb_ts_metadata_poll = rd_clock() + (rkb->rkb_rk->rk_conf. metadata_refresh_fast_interval_ms * 1000); } else { /* According to configured poll interval */ rkb->rkb_ts_metadata_poll = rd_clock() + (rkb->rkb_rk->rk_conf. metadata_refresh_interval_ms * 1000); } } } if (only_rkt || !all_topics) { rd_kafka_lock(rkb->rkb_rk); /* Calculate size to hold all known topics */ TAILQ_FOREACH(rkt, &rkb->rkb_rk->rk_topics, rkt_link) { if (only_rkt && only_rkt != rkt) continue; arrsize++; tnamelen += RD_KAFKAP_STR_SIZE(rkt->rkt_topic); } } buf = malloc(sizeof(arrsize) + tnamelen); arrsize = htonl(arrsize); memcpy(buf+of, &arrsize, 4); of += 4; if (only_rkt || !all_topics) { /* Just our locally known topics */ TAILQ_FOREACH(rkt, &rkb->rkb_rk->rk_topics, rkt_link) { int tlen; if (only_rkt && only_rkt != rkt) continue; tlen = RD_KAFKAP_STR_SIZE(rkt->rkt_topic); memcpy(buf+of, rkt->rkt_topic, tlen); of += tlen; } rd_kafka_unlock(rkb->rkb_rk); } if (only_rkt) rd_kafka_topic_keep(only_rkt); rd_kafka_broker_buf_enq(rkb, RD_KAFKAP_Metadata, buf, of, RD_KAFKA_OP_F_FREE|RD_KAFKA_OP_F_FLASH, rd_kafka_broker_metadata_reply, only_rkt); } /** * Locks: rd_kafka_lock(rk) MUST be held. * Locality: any thread */ static rd_kafka_broker_t *rd_kafka_broker_any (rd_kafka_t *rk, int state) { rd_kafka_broker_t *rkb; TAILQ_FOREACH(rkb, &rk->rk_brokers, rkb_link) { rd_kafka_broker_lock(rkb); if (rkb->rkb_state == state) { rd_kafka_broker_keep(rkb); rd_kafka_broker_unlock(rkb); return rkb; } rd_kafka_broker_unlock(rkb); } return NULL; } /** * Trigger broker metadata query for topic leader. * 'rkt' may be NULL to query for all topics. */ void rd_kafka_topic_leader_query0 (rd_kafka_t *rk, rd_kafka_topic_t *rkt, int do_rk_lock) { rd_kafka_broker_t *rkb; if (do_rk_lock) rd_kafka_lock(rk); if (!(rkb = rd_kafka_broker_any(rk, RD_KAFKA_BROKER_STATE_UP))) { if (do_rk_lock) rd_kafka_unlock(rk); return; /* No brokers are up */ } if (do_rk_lock) rd_kafka_unlock(rk); if (rkt) { rd_kafka_topic_wrlock(rkt); /* Avoid multiple leader queries if there is already * an outstanding one waiting for reply. */ if (rkt->rkt_flags & RD_KAFKA_TOPIC_F_LEADER_QUERY) { rd_kafka_topic_unlock(rkt); rd_kafka_broker_destroy(rkb); return; } rkt->rkt_flags |= RD_KAFKA_TOPIC_F_LEADER_QUERY; rd_kafka_topic_unlock(rkt); } rd_kafka_broker_metadata_req(rkb, 0, rkt, "leader query"); /* Release refcnt from rd_kafka_broker_any() */ rd_kafka_broker_destroy(rkb); } /** * Find a waitresp (rkbuf awaiting response) by the correlation id. */ static rd_kafka_buf_t *rd_kafka_waitresp_find (rd_kafka_broker_t *rkb, int32_t corrid) { rd_kafka_buf_t *rkbuf; rd_ts_t now = rd_clock(); assert(pthread_self() == rkb->rkb_thread); TAILQ_FOREACH(rkbuf, &rkb->rkb_waitresps.rkbq_bufs, rkbuf_link) if (rkbuf->rkbuf_corrid == corrid) { rd_kafka_avg_add(&rkb->rkb_rtt_curr, now - rkbuf->rkbuf_ts_sent); rd_kafka_bufq_deq(&rkb->rkb_waitresps, rkbuf); return rkbuf; } return NULL; } /** * Map a response message to a request. */ static int rd_kafka_req_response (rd_kafka_broker_t *rkb, rd_kafka_buf_t *rkbuf) { rd_kafka_buf_t *req; assert(pthread_self() == rkb->rkb_thread); /* Find corresponding request message by correlation id */ if (unlikely(!(req = rd_kafka_waitresp_find(rkb, rkbuf->rkbuf_reshdr.CorrId)))) { /* unknown response. probably due to request timeout */ rkb->rkb_c.rx_corrid_err++; rd_rkb_dbg(rkb, BROKER, "RESPONSE", "Response for unknown CorrId %"PRId32" (timed out?)", rkbuf->rkbuf_reshdr.CorrId); rd_kafka_buf_destroy(rkbuf); return -1; } /* Call callback. Ownership of 'rkbuf' is delegated to callback. */ req->rkbuf_cb(rkb, 0, rkbuf, req, req->rkbuf_opaque); return 0; } /** * Rebuilds 'src' into 'dst' starting at byte offset 'of'. */ static void rd_kafka_msghdr_rebuild (struct msghdr *dst, size_t dst_len, struct msghdr *src, off_t of) { int i; size_t len = 0; void *iov = dst->msg_iov; *dst = *src; dst->msg_iov = iov; dst->msg_iovlen = 0; for (i = 0 ; i < src->msg_iovlen ; i++) { off_t vof = of - len; if (0) printf(" #%i/%zd and %zd: of %jd, len %zd, " "vof %jd: iov %zd\n", i, (size_t)src->msg_iovlen, (size_t)dst->msg_iovlen, (intmax_t)of, len, (intmax_t)vof, src->msg_iov[i].iov_len); if (vof < 0) vof = 0; if (vof < src->msg_iov[i].iov_len) { assert(dst->msg_iovlen < dst_len); dst->msg_iov[dst->msg_iovlen].iov_base = (char *)src->msg_iov[i].iov_base + vof; dst->msg_iov[dst->msg_iovlen].iov_len = src->msg_iov[i].iov_len - vof; dst->msg_iovlen++; } len += src->msg_iov[i].iov_len; } } static int rd_kafka_recv (rd_kafka_broker_t *rkb) { rd_kafka_buf_t *rkbuf; ssize_t r; struct msghdr msg; char errstr[512]; rd_kafka_resp_err_t err_code = 0; if (0) rd_rkb_dbg(rkb, BROKER, "RECV", "%s: Receive on socket %i (buf %p)", rkb->rkb_name, rkb->rkb_s, rkb->rkb_recv_buf); /** * The receive buffers are split up in two parts: * - the first part is mainly for reading the first 4 bytes * where the remaining length is coded. * But for short packets we want to avoid a second recv() call * so the first buffer should be large enough for common short * packets. * This is iov[0] and iov[1]. * * - the second part is mainly for data response, this buffer * must be contigious and will be provided to the application * as is (Fetch response). * This is iov[2]. * * It is impossible to estimate the correct size of the first * buffer, so we make it big enough to probably fit all kinds of * non-data responses so we dont have to allocate a second buffer * for such responses. And we make it small enough that a copy * to the second buffer isn't too costly in case we receive a * real data packet. * * Minimum packet sizes per response type: * Metadata: 4+4+2+host+4+4+2+2+topic+2+4+4+4+4+4+4.. =~ 48 * Produce: 4+2+topic+4+4+2+8.. =~ 24 * Fetch: 4+2+topic+4+4+2+8+8+4.. =~ 36 * Offset: 4+2+topic+4+4+2+4+8.. =~ 28 * ... * * Plus 4 + 4 for Size and CorrId. * * First buffer size should thus be: 96 bytes */ /* FIXME: skip the above, just go for the header. */ if (!(rkbuf = rkb->rkb_recv_buf)) { /* No receive in progress: new message. */ rkbuf = rd_kafka_buf_new(0, 0); rkbuf->rkbuf_iov[0].iov_base = &rkbuf->rkbuf_reshdr; rkbuf->rkbuf_iov[0].iov_len = sizeof(rkbuf->rkbuf_reshdr); rkbuf->rkbuf_msg.msg_iov = rkbuf->rkbuf_iov; rkbuf->rkbuf_msg.msg_iovlen = 1; msg = rkbuf->rkbuf_msg; rkb->rkb_recv_buf = rkbuf; } else { /* Receive in progress: adjust the msg to allow more data. */ msg.msg_iov = alloca(sizeof(struct iovec) * rkbuf->rkbuf_iovcnt); rd_kafka_msghdr_rebuild(&msg, rkbuf->rkbuf_msg.msg_iovlen, &rkbuf->rkbuf_msg, rkbuf->rkbuf_of); } assert(rd_kafka_msghdr_size(&msg) > 0); if ((r = recvmsg(rkb->rkb_s, &msg, MSG_DONTWAIT)) == -1) { if (errno == EAGAIN) return 0; snprintf(errstr, sizeof(errstr), "Receive error: %s", strerror(errno)); err_code = RD_KAFKA_RESP_ERR__TRANSPORT; rkb->rkb_c.rx_err++; goto err; } if (r == 0) { /* Receive 0 after POLLIN event means connection closed. */ snprintf(errstr, sizeof(errstr), "Disconnected"); err_code = RD_KAFKA_RESP_ERR__TRANSPORT; goto err; } rkbuf->rkbuf_of += r; if (rkbuf->rkbuf_len == 0) { /* Packet length not known yet. */ if (unlikely(rkbuf->rkbuf_of < sizeof(rkbuf->rkbuf_reshdr))) { /* Need response header for packet length and corrid. * Wait for more data. */ return 0; } rkbuf->rkbuf_len = ntohl(rkbuf->rkbuf_reshdr.Size); rkbuf->rkbuf_reshdr.CorrId = ntohl(rkbuf->rkbuf_reshdr.CorrId); /* Make sure message size is within tolerable limits. */ if (rkbuf->rkbuf_len < 4/*CorrId*/ || rkbuf->rkbuf_len > rkb->rkb_rk->rk_conf.recv_max_msg_size) { snprintf(errstr, sizeof(errstr), "Invalid message size %zd (0..%i): " "increase receive.message.max.bytes", rkbuf->rkbuf_len-4, rkb->rkb_rk->rk_conf.max_msg_size); rkb->rkb_c.rx_err++; err_code = RD_KAFKA_RESP_ERR__BAD_MSG; goto err; } rkbuf->rkbuf_len -= 4; /*CorrId*/ if (rkbuf->rkbuf_len > 0) { /* Allocate another buffer that fits all data (short of * the common response header). We want all * data to be in contigious memory. */ rkbuf->rkbuf_buf2 = malloc(rkbuf->rkbuf_len); assert(rkbuf->rkbuf_msg.msg_iovlen == 1); rkbuf->rkbuf_iov[1].iov_base = rkbuf->rkbuf_buf2; rkbuf->rkbuf_iov[1].iov_len = rkbuf->rkbuf_len; rkbuf->rkbuf_msg.msg_iovlen = 2; } } if (rkbuf->rkbuf_of == rkbuf->rkbuf_len + sizeof(rkbuf->rkbuf_reshdr)) { /* Message is complete, pass it on to the original requester. */ rkb->rkb_recv_buf = NULL; (void)rd_atomic_add(&rkb->rkb_c.rx, 1); (void)rd_atomic_add(&rkb->rkb_c.rx_bytes, rkbuf->rkbuf_of); rd_kafka_req_response(rkb, rkbuf); } return 1; err: rd_kafka_broker_fail(rkb, err_code, "Receive failed: %s", errstr); return -1; } static int rd_kafka_broker_connect (rd_kafka_broker_t *rkb) { rd_sockaddr_inx_t *sinx; int one __attribute__((unused)) = 1; rd_rkb_dbg(rkb, BROKER, "CONNECT", "broker in state %s connecting", rd_kafka_broker_state_names[rkb->rkb_state]); if (rd_kafka_broker_resolve(rkb) == -1) return -1; sinx = rd_sockaddr_list_next(rkb->rkb_rsal); assert(rkb->rkb_s == -1); if ((rkb->rkb_s = socket(sinx->sinx_family, SOCK_STREAM, IPPROTO_TCP)) == -1) { rd_kafka_broker_fail(rkb, RD_KAFKA_RESP_ERR__FAIL, "Failed to create %s socket: %s", rd_family2str(sinx->sinx_family), strerror(errno)); return -1; } #ifdef SO_NOSIGPIPE /* Disable SIGPIPE signalling for this socket on OSX */ if (setsockopt(rkb->rkb_s, SOL_SOCKET, SO_NOSIGPIPE, &one, sizeof(one)) == -1) rd_rkb_dbg(rkb, BROKER, "SOCKET", "Failed to set SO_NOSIGPIPE: %s", strerror(errno)); #endif if (connect(rkb->rkb_s, (struct sockaddr *)sinx, RD_SOCKADDR_INX_LEN(sinx)) == -1) { rd_rkb_dbg(rkb, BROKER, "CONNECT", "couldn't connect to %s: %s", rd_sockaddr2str(sinx, RD_SOCKADDR2STR_F_PORT | RD_SOCKADDR2STR_F_FAMILY), strerror(errno)); /* Avoid duplicate log messages */ if (rkb->rkb_err.err == errno) rd_kafka_broker_fail(rkb, RD_KAFKA_RESP_ERR__FAIL, NULL); else rd_kafka_broker_fail(rkb, RD_KAFKA_RESP_ERR__TRANSPORT, "Failed to connect to " "broker at %s: %s", rd_sockaddr2str(sinx, RD_SOCKADDR2STR_F_NICE), strerror(errno)); return -1; } rd_rkb_dbg(rkb, BROKER, "CONNECTED", "connected to %s", rd_sockaddr2str(sinx, RD_SOCKADDR2STR_F_NICE)); /* Set socket send & receive buffer sizes if configuerd */ if (rkb->rkb_rk->rk_conf.socket_sndbuf_size != 0) { if (setsockopt(rkb->rkb_s, SOL_SOCKET, SO_SNDBUF, &rkb->rkb_rk->rk_conf.socket_sndbuf_size, sizeof(rkb->rkb_rk->rk_conf. socket_sndbuf_size)) == -1) rd_rkb_log(rkb, LOG_WARNING, "SNDBUF", "Failed to set socket send " "buffer size to %i: %s", rkb->rkb_rk->rk_conf.socket_sndbuf_size, strerror(errno)); } if (rkb->rkb_rk->rk_conf.socket_rcvbuf_size != 0) { if (setsockopt(rkb->rkb_s, SOL_SOCKET, SO_RCVBUF, &rkb->rkb_rk->rk_conf.socket_rcvbuf_size, sizeof(rkb->rkb_rk->rk_conf. socket_rcvbuf_size)) == -1) rd_rkb_log(rkb, LOG_WARNING, "RCVBUF", "Failed to set socket receive " "buffer size to %i: %s", rkb->rkb_rk->rk_conf.socket_rcvbuf_size, strerror(errno)); } rd_kafka_broker_lock(rkb); rd_kafka_broker_set_state(rkb, RD_KAFKA_BROKER_STATE_UP); rkb->rkb_err.err = 0; rd_kafka_broker_unlock(rkb); rkb->rkb_pfd.fd = rkb->rkb_s; rkb->rkb_pfd.events = POLLIN; /* Request metadata (async) */ rd_kafka_broker_metadata_req(rkb, 1 /* all topics */, NULL, "connected"); return 0; } /** * Send queued messages to broker * * Locality: io thread */ static int rd_kafka_send (rd_kafka_broker_t *rkb) { rd_kafka_buf_t *rkbuf; unsigned int cnt = 0; assert(pthread_self() == rkb->rkb_thread); while (rkb->rkb_state == RD_KAFKA_BROKER_STATE_UP && (rkbuf = TAILQ_FIRST(&rkb->rkb_outbufs.rkbq_bufs))) { ssize_t r; struct msghdr *msg = &rkbuf->rkbuf_msg; struct msghdr msg2; struct iovec iov[IOV_MAX]; if (rkbuf->rkbuf_of != 0) { /* If message has been partially sent we need * to construct a new msg+iovec skipping the * sent bytes. */ msg2.msg_iov = iov; rd_kafka_msghdr_rebuild(&msg2, IOV_MAX, &rkbuf->rkbuf_msg, rkbuf->rkbuf_of); msg = &msg2; } if (0) rd_rkb_dbg(rkb, BROKER, "SEND", "Send buf corrid %"PRId32" at " "offset %zd/%zd", rkbuf->rkbuf_corrid, rkbuf->rkbuf_of, rkbuf->rkbuf_len); if ((r = rd_kafka_broker_send(rkb, msg)) == -1) { /* FIXME: */ return -1; } rkbuf->rkbuf_of += r; /* Partial send? Continue next time. */ if (rkbuf->rkbuf_of < rkbuf->rkbuf_len) { return 0; } /* Entire buffer sent, unlink from outbuf */ rd_kafka_bufq_deq(&rkb->rkb_outbufs, rkbuf); /* Store time for RTT calculation */ rkbuf->rkbuf_ts_sent = rd_clock(); /* Put buffer on response wait list unless we are not * expecting a response (required_acks=0). */ if (!(rkbuf->rkbuf_flags & RD_KAFKA_OP_F_NO_RESPONSE)) rd_kafka_bufq_enq(&rkb->rkb_waitresps, rkbuf); else /* Call buffer callback for delivery report. */ rkbuf->rkbuf_cb(rkb, 0, NULL, rkbuf, rkbuf->rkbuf_opaque); cnt++; } rd_rkb_dbg(rkb, BROKER, "SEND", "Sent %i bufs", cnt); return cnt; } /** * Add 'rkbuf' to broker 'rkb's retry queue. */ static void rd_kafka_broker_buf_retry (rd_kafka_broker_t *rkb, rd_kafka_buf_t *rkbuf) { assert(pthread_self() == rkb->rkb_thread); rkb->rkb_c.tx_retries++; rkbuf->rkbuf_ts_retry = rd_clock() + (rkb->rkb_rk->rk_conf.retry_backoff_ms * 1000); /* Reset send offset */ rkbuf->rkbuf_of = 0; rd_kafka_bufq_enq(&rkb->rkb_retrybufs, rkbuf); } /** * Move buffers that have expired their retry backoff time from the * retry queue to the outbuf. */ static void rd_kafka_broker_retry_bufs_move (rd_kafka_broker_t *rkb) { rd_ts_t now = rd_clock(); rd_kafka_buf_t *rkbuf; while ((rkbuf = TAILQ_FIRST(&rkb->rkb_retrybufs.rkbq_bufs))) { if (rkbuf->rkbuf_ts_retry > now) break; rd_kafka_bufq_deq(&rkb->rkb_retrybufs, rkbuf); rd_kafka_broker_buf_enq0(rkb, rkbuf, 0/*tail*/); } } /** * Propagate delivery report for entire message queue. */ void rd_kafka_dr_msgq (rd_kafka_t *rk, rd_kafka_msgq_t *rkmq, rd_kafka_resp_err_t err) { if (rk->rk_conf.dr_cb) { /* Pass all messages to application thread in one op. */ rd_kafka_op_t *rko; rko = rd_kafka_op_new(RD_KAFKA_OP_DR); rko->rko_err = err; /* Move all messages to op's msgq */ rd_kafka_msgq_move(&rko->rko_msgq, rkmq); rd_kafka_op_reply2(rk, rko); } else { /* No delivery report callback, destroy the messages * right away. */ rd_kafka_msgq_purge(rk, rkmq); } } /** * Parses a Produce reply. * Returns 0 on success or an error code on failure. */ static rd_kafka_resp_err_t rd_kafka_produce_reply_handle (rd_kafka_broker_t *rkb, rd_kafka_buf_t *rkbuf) { char *buf = rkbuf->rkbuf_buf2; size_t size = rkbuf->rkbuf_len; size_t of = 0; int32_t TopicArrayCnt; rd_kafkap_str_t *topic; int32_t PartitionArrayCnt; struct { int32_t Partition; int16_t ErrorCode; int64_t Offset; } RD_PACKED *hdr; _READ_I32(&TopicArrayCnt); if (TopicArrayCnt != 1) goto err; /* Since we only produce to one single topic+partition in each * request we assume that the reply only contains one topic+partition * and that it is the same that we requested. * If not the broker is buggy. */ _READ_STR(topic); _READ_I32(&PartitionArrayCnt); if (PartitionArrayCnt != 1) goto err; _READ_REF(hdr, sizeof(*hdr)); return ntohs(hdr->ErrorCode); err: return RD_KAFKA_RESP_ERR__BAD_MSG; } /** * Locality: io thread */ static void rd_kafka_produce_msgset_reply (rd_kafka_broker_t *rkb, rd_kafka_resp_err_t err, rd_kafka_buf_t *reply, rd_kafka_buf_t *request, void *opaque) { rd_kafka_toppar_t *rktp = opaque; rd_rkb_dbg(rkb, MSG, "MSGSET", "MessageSet with %i message(s) %sdelivered", request->rkbuf_msgq.rkmq_msg_cnt, err ? "not ": ""); /* Parse Produce reply (unless the request errored) */ if (!err && reply) err = rd_kafka_produce_reply_handle(rkb, reply); if (err) { rd_rkb_dbg(rkb, MSG, "MSGSET", "MessageSet with %i message(s) " "encountered error: %s", request->rkbuf_msgq.rkmq_msg_cnt, rd_kafka_err2str(err)); switch (err) { case RD_KAFKA_RESP_ERR__DESTROY: case RD_KAFKA_RESP_ERR_INVALID_MSG: case RD_KAFKA_RESP_ERR_INVALID_MSG_SIZE: case RD_KAFKA_RESP_ERR_MSG_SIZE_TOO_LARGE: /* Fatal errors: no message transmission retries */ break; case RD_KAFKA_RESP_ERR_REQUEST_TIMED_OUT: case RD_KAFKA_RESP_ERR__MSG_TIMED_OUT: /* Try again */ if (++request->rkbuf_retries < rkb->rkb_rk->rk_conf.max_retries) { if (reply) rd_kafka_buf_destroy(reply); rd_kafka_broker_buf_retry(rkb, request); return; } break; case RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_OR_PART: case RD_KAFKA_RESP_ERR_LEADER_NOT_AVAILABLE: case RD_KAFKA_RESP_ERR_NOT_LEADER_FOR_PARTITION: case RD_KAFKA_RESP_ERR_BROKER_NOT_AVAILABLE: case RD_KAFKA_RESP_ERR_REPLICA_NOT_AVAILABLE: case RD_KAFKA_RESP_ERR__TRANSPORT: default: /* Request metadata information update */ rkb->rkb_metadata_fast_poll_cnt = rkb->rkb_rk->rk_conf.metadata_refresh_fast_cnt; rd_kafka_topic_leader_query(rkb->rkb_rk, rktp->rktp_rkt); /* FIXME: Should message retries be incremented? */ /* Move messages (in the rkbuf) back to the partition's * queue head. They will be resent when a new leader * is delegated. */ rd_kafka_toppar_insert_msgq(rktp, &request->rkbuf_msgq); goto done; } /* FALLTHRU */ } /* Enqueue messages for delivery report */ rd_kafka_dr_msgq(rkb->rkb_rk, &request->rkbuf_msgq, err); done: rd_kafka_toppar_destroy(rktp); /* from produce_toppar() */ rd_kafka_buf_destroy(request); if (reply) rd_kafka_buf_destroy(reply); } /** * Produce messages from 'rktp's queue. */ static int rd_kafka_broker_produce_toppar (rd_kafka_broker_t *rkb, rd_kafka_toppar_t *rktp) { int cnt; rd_kafka_msg_t *rkm; int msgcnt; rd_kafka_buf_t *rkbuf; rd_kafka_topic_t *rkt = rktp->rktp_rkt; struct { struct { int16_t RequiredAcks; int32_t Timeout; int32_t TopicArrayCnt; } __attribute__((packed)) part1; /* TopicName is inserted here */ struct { int32_t PartitionArrayCnt; int32_t Partition; int32_t MessageSetSize; } __attribute__((packed)) part2; /* MessageSet headers follows */ } *prodhdr; /* Both MessageSet and Message headers combined: */ struct { struct { int64_t Offset; int32_t MessageSize; int32_t Crc; int8_t MagicByte; int8_t Attributes; } __attribute__((packed)) part3; struct { int32_t Value_len; } __attribute__((packed)) part4; } *msghdr; int iovcnt; int iov_firstmsg; /* iovs: * 1 * RequiredAcks + Timeout (part1) * 1 * Topic * 1 * Partition + MessageSetSize (part2) * msgcnt * messagehdr (part3) * msgcnt * key * msgcnt * Value_len (part4) * msgcnt * messagepayload * = 3 + (4 * msgcnt) * * We are bound both by configuration and IOV_MAX */ if (rktp->rktp_xmit_msgq.rkmq_msg_cnt > 0) assert(TAILQ_FIRST(&rktp->rktp_xmit_msgq.rkmq_msgs)); msgcnt = RD_MIN(rktp->rktp_xmit_msgq.rkmq_msg_cnt, rkb->rkb_rk->rk_conf.batch_num_messages); assert(msgcnt > 0); iovcnt = 3 + (4 * msgcnt); if (iovcnt > RD_KAFKA_PAYLOAD_IOV_MAX) { iovcnt = RD_KAFKA_PAYLOAD_IOV_MAX; msgcnt = ((iovcnt / 4) - 3); } if (0) rd_rkb_dbg(rkb, MSG, "PRODUCE", "Serve %i/%i messages (%i iovecs) " "for %.*s [%"PRId32"] (%"PRIu64" bytes)", msgcnt, rktp->rktp_msgq.rkmq_msg_cnt, iovcnt, RD_KAFKAP_STR_PR(rkt->rkt_topic), rktp->rktp_partition, rktp->rktp_msgq.rkmq_msg_bytes); /* Allocate iovecs to hold all headers and messages, * and allocate auxilliery space for the headers. */ rkbuf = rd_kafka_buf_new(iovcnt, sizeof(*prodhdr) + (sizeof(*msghdr) * msgcnt)); prodhdr = (void *)rkbuf->rkbuf_buf; msghdr = (void *)(prodhdr+1); /* Insert first part of Produce header */ prodhdr->part1.RequiredAcks = htons(rkt->rkt_conf.required_acks); prodhdr->part1.Timeout = htonl(rkt->rkt_conf.request_timeout_ms); prodhdr->part1.TopicArrayCnt = htonl(1); rd_kafka_buf_push(rkbuf, &prodhdr->part1, sizeof(prodhdr->part1)); /* Insert topic */ rd_kafka_buf_push(rkbuf, rkt->rkt_topic, RD_KAFKAP_STR_SIZE(rkt->rkt_topic)); /* Insert second part of Produce header */ prodhdr->part2.PartitionArrayCnt = htonl(1); prodhdr->part2.Partition = htonl(rktp->rktp_partition); /* Will be finalized later*/ prodhdr->part2.MessageSetSize = 0; rd_kafka_buf_push(rkbuf, &prodhdr->part2, sizeof(prodhdr->part2)); iov_firstmsg = rkbuf->rkbuf_msg.msg_iovlen; while (msgcnt > 0 && (rkm = TAILQ_FIRST(&rktp->rktp_xmit_msgq.rkmq_msgs))) { if (prodhdr->part2.MessageSetSize + rkm->rkm_len > rkb->rkb_rk->rk_conf.max_msg_size) { rd_rkb_dbg(rkb, MSG, "PRODUCE", "No more space in current message " "(%i messages)", rkbuf->rkbuf_msgq.rkmq_msg_cnt); /* Not enough remaining size. */ break; } rd_kafka_msgq_deq(&rktp->rktp_xmit_msgq, rkm, 1); rd_kafka_msgq_enq(&rkbuf->rkbuf_msgq, rkm); msgcnt--; /* Message header */ msghdr->part3.Offset = 0; msghdr->part3.MessageSize = (sizeof(msghdr->part3) - sizeof(msghdr->part3.Offset) - sizeof(msghdr->part3.MessageSize)) + RD_KAFKAP_BYTES_SIZE(rkm->rkm_key) + sizeof(msghdr->part4) + rkm->rkm_len; prodhdr->part2.MessageSetSize += sizeof(msghdr->part3.Offset) + sizeof(msghdr->part3.MessageSize) + msghdr->part3.MessageSize; msghdr->part3.MessageSize = htonl(msghdr->part3.MessageSize); msghdr->part3.Crc = rd_crc32_init(); msghdr->part3.MagicByte = 0; /* FIXME: what? */ msghdr->part3.Attributes = 0; /* No compression */ msghdr->part3.Crc = rd_crc32_update(msghdr->part3.Crc, (void *) &msghdr->part3.MagicByte, sizeof(msghdr->part3. MagicByte) + sizeof(msghdr->part3. Attributes)); /* Message header */ rd_kafka_buf_push(rkbuf, &msghdr->part3, sizeof(msghdr->part3)); /* Key */ msghdr->part3.Crc = rd_crc32_update(msghdr->part3.Crc, (void *)rkm->rkm_key, RD_KAFKAP_BYTES_SIZE(rkm-> rkm_key)); rd_kafka_buf_push(rkbuf, rkm->rkm_key, RD_KAFKAP_BYTES_SIZE(rkm->rkm_key)); /* Value(payload) length */ msghdr->part4.Value_len = htonl(rkm->rkm_len); msghdr->part3.Crc = rd_crc32_update(msghdr->part3.Crc, (void *) &msghdr->part4.Value_len, sizeof(msghdr->part4. Value_len)); rd_kafka_buf_push(rkbuf, &msghdr->part4, sizeof(msghdr->part4)); /* Payload */ msghdr->part3.Crc = rd_crc32_update(msghdr->part3.Crc, rkm->rkm_payload, rkm->rkm_len); rd_kafka_buf_push(rkbuf, rkm->rkm_payload, rkm->rkm_len); /* Finalize Crc */ msghdr->part3.Crc = htonl(rd_crc32_finalize(msghdr->part3.Crc)); msghdr++; } /* No messages added, bail out early. */ if (unlikely(rkbuf->rkbuf_msgq.rkmq_msg_cnt == 0)) { rd_kafka_buf_destroy(rkbuf); return -1; } /* Compress the messages */ if (rkb->rkb_rk->rk_conf.compression_codec) { int siovlen = 1; size_t coutlen; int r; struct { int64_t Offset; int32_t MessageSize; int32_t Crc; int8_t MagicByte; int8_t Attributes; int32_t Key_len; /* -1 */ int32_t Value_len; } RD_PACKED *msghdr2 = NULL; int32_t ctotlen; struct snappy_env senv; struct iovec siov; z_stream strm; int i; switch (rkb->rkb_rk->rk_conf.compression_codec) { case RD_KAFKA_COMPRESSION_NONE: abort(); /* unreachable */ break; case RD_KAFKA_COMPRESSION_GZIP: /* Initialize gzip compression */ memset(&strm, 0, sizeof(strm)); r = deflateInit2(&strm, Z_DEFAULT_COMPRESSION, Z_DEFLATED, 15+16, 8, Z_DEFAULT_STRATEGY); if (r != Z_OK) { rd_rkb_log(rkb, LOG_ERR, "GZIP", "Failed to initialize gzip for " "compressing %"PRId32" bytes in " "topic %.*s [%"PRId32"]: %s (%i): " "sending uncompressed", prodhdr->part2.MessageSetSize, RD_KAFKAP_STR_PR(rktp->rktp_rkt-> rkt_topic), rktp->rktp_partition, strm.msg ? : "", r); goto do_send; } /* Calculate maximum compressed size and * allocate an output buffer accordingly, being * prefixed with the Message header. */ siov.iov_len = deflateBound(&strm, prodhdr->part2. MessageSetSize); msghdr2 = malloc(sizeof(*msghdr2) + siov.iov_len); siov.iov_base = (void *)(msghdr2+1); strm.next_out = siov.iov_base; strm.avail_out = siov.iov_len; /* Iterate through each message and compress it. */ for (i = iov_firstmsg ; i < rkbuf->rkbuf_msg.msg_iovlen ; i++) { if (rkbuf->rkbuf_msg.msg_iov[i].iov_len == 0) continue; strm.next_in = rkbuf->rkbuf_msg. msg_iov[i].iov_base; strm.avail_in = rkbuf->rkbuf_msg. msg_iov[i].iov_len; /* Compress message */ if ((r = deflate(&strm, Z_NO_FLUSH) != Z_OK)) { rd_rkb_log(rkb, LOG_ERR, "GZIP", "Failed to gzip-compress " "%zd bytes for " "topic %.*s [%"PRId32"]: " "%s (%i): " "sending uncompressed", rkbuf->rkbuf_msg.msg_iov[i]. iov_len, RD_KAFKAP_STR_PR(rktp-> rktp_rkt-> rkt_topic), rktp->rktp_partition, strm.msg ? : "", r); deflateEnd(&strm); free(msghdr2); goto do_send; } assert(strm.avail_in == 0); } /* Finish the compression */ if ((r = deflate(&strm, Z_FINISH)) != Z_STREAM_END) { rd_rkb_log(rkb, LOG_ERR, "GZIP", "Failed to finish gzip compression " " of %"PRId32" bytes for " "topic %.*s [%"PRId32"]: " "%s (%i): " "sending uncompressed", prodhdr->part2.MessageSetSize, RD_KAFKAP_STR_PR(rktp->rktp_rkt-> rkt_topic), rktp->rktp_partition, strm.msg ? : "", r); deflateEnd(&strm); free(msghdr2); goto do_send; } coutlen = strm.total_out; /* Deinitialize compression */ deflateEnd(&strm); break; case RD_KAFKA_COMPRESSION_SNAPPY: /* Initialize snappy compression environment */ snappy_init_env_sg(&senv, 1/*iov enable*/); /* Calculate maximum compressed size and * allocate an output buffer accordingly, being * prefixed with the Message header. */ siov.iov_len = snappy_max_compressed_length(prodhdr->part2. MessageSetSize); msghdr2 = malloc(sizeof(*msghdr2) + siov.iov_len); siov.iov_base = (void *)(msghdr2+1); /* Compress each message */ if ((r = snappy_compress_iov(&senv, &rkbuf-> rkbuf_iov[iov_firstmsg], rkbuf->rkbuf_msg. msg_iovlen - iov_firstmsg, prodhdr->part2. MessageSetSize, &siov, &siovlen, &coutlen)) != 0) { rd_rkb_log(rkb, LOG_ERR, "SNAPPY", "Failed to snappy-compress " "%"PRId32" bytes for " "topic %.*s [%"PRId32"]: %s: " "sending uncompressed", prodhdr->part2.MessageSetSize, RD_KAFKAP_STR_PR(rktp->rktp_rkt-> rkt_topic), rktp->rktp_partition, strerror(-r)); free(msghdr2); goto do_send; } /* Free snappy environment */ snappy_free_env(&senv); } /* Create a new Message who's Value is the compressed data */ ctotlen = sizeof(*msghdr2) + coutlen; msghdr2->Offset = 0; msghdr2->MessageSize = htonl(4+1+1+4+4 + coutlen); msghdr2->MagicByte = 0; msghdr2->Attributes = rkb->rkb_rk->rk_conf. compression_codec & 0x7; msghdr2->Key_len = htonl(-1); msghdr2->Value_len = htonl(coutlen); msghdr2->Crc = rd_crc32_init(); msghdr2->Crc = rd_crc32_update(msghdr2->Crc, (void *)&msghdr2->MagicByte, 1+1+4+4); msghdr2->Crc = rd_crc32_update(msghdr2->Crc, siov.iov_base, coutlen); msghdr2->Crc = htonl(rd_crc32_finalize(msghdr2->Crc)); /* Update enveloping MessageSet's length. */ prodhdr->part2.MessageSetSize = ctotlen; /* Rewind rkbuf to the pre-message checkpoint. * This replaces all the original Messages with just the * Message containing the compressed payload. */ rd_kafka_buf_rewind(rkbuf, iov_firstmsg); /* Add allocated buffer as auxbuf to rkbuf so that * it will get freed with the rkbuf */ rd_kafka_buf_auxbuf_add(rkbuf, msghdr2); /* Push the new Message onto the buffer stack. */ rd_kafka_buf_push(rkbuf, msghdr2, ctotlen); } do_send: (void)rd_atomic_add(&rktp->rktp_c.tx_msgs, rkbuf->rkbuf_msgq.rkmq_msg_cnt); (void)rd_atomic_add(&rktp->rktp_c.tx_bytes, prodhdr->part2.MessageSetSize); prodhdr->part2.MessageSetSize = htonl(prodhdr->part2.MessageSetSize); rd_rkb_dbg(rkb, MSG, "PRODUCE", "produce messageset with %i messages " "(%"PRId32" bytes)", rkbuf->rkbuf_msgq.rkmq_msg_cnt, ntohl(prodhdr->part2.MessageSetSize)); cnt = rkbuf->rkbuf_msgq.rkmq_msg_cnt; if (!rkt->rkt_conf.required_acks) rkbuf->rkbuf_flags |= RD_KAFKA_OP_F_NO_RESPONSE; /* Use timeout from first message. */ rkbuf->rkbuf_ts_timeout = TAILQ_FIRST(&rkbuf->rkbuf_msgq.rkmq_msgs)->rkm_ts_timeout; rd_kafka_toppar_keep(rktp); /* refcount for msgset_reply() */ rd_kafka_broker_buf_enq1(rkb, RD_KAFKAP_Produce, rkbuf, rd_kafka_produce_msgset_reply, rktp); return cnt; } /** * Serve a broker op (an op posted by another thread to be handled by * this broker's thread). */ static void rd_kafka_broker_op_serve (rd_kafka_broker_t *rkb, rd_kafka_op_t *rko) { assert(pthread_self() == rkb->rkb_thread); switch (rko->rko_type) { case RD_KAFKA_OP_METADATA_REQ: if (rko->rko_rkt) { rd_kafka_broker_metadata_req(rkb, 0, rko->rko_rkt, NULL); /* Loose refcnt from op enq */ rd_kafka_topic_destroy0(rko->rko_rkt); } else rd_kafka_broker_metadata_req(rkb, 1 /*all topics*/, NULL, NULL); break; default: assert(!*"unhandled op type"); } rd_kafka_op_destroy(rko); } static void rd_kafka_broker_io_serve (rd_kafka_broker_t *rkb) { rd_kafka_op_t *rko; rd_ts_t now = rd_clock(); /* Serve broker ops */ if (unlikely(rd_kafka_q_len(&rkb->rkb_ops) > 0)) while ((rko = rd_kafka_q_pop(&rkb->rkb_ops, RD_POLL_NOWAIT))) rd_kafka_broker_op_serve(rkb, rko); /* Periodic metadata poll */ if (unlikely(now >= rkb->rkb_ts_metadata_poll)) rd_kafka_broker_metadata_req(rkb, 1 /* all topics */, NULL, "periodic refresh"); if (rkb->rkb_outbufs.rkbq_cnt > 0) rkb->rkb_pfd.events |= POLLOUT; else rkb->rkb_pfd.events &= ~POLLOUT; if (poll(&rkb->rkb_pfd, 1, rkb->rkb_rk->rk_conf.buffering_max_ms) <= 0) return; if (rkb->rkb_pfd.revents & POLLIN) while (rd_kafka_recv(rkb) > 0) ; if (rkb->rkb_pfd.revents & POLLHUP) return rd_kafka_broker_fail(rkb, RD_KAFKA_RESP_ERR__TRANSPORT, "Connection closed"); if (rkb->rkb_pfd.revents & POLLOUT) while (rd_kafka_send(rkb) > 0) ; } /** * Idle function for unassigned brokers */ static void rd_kafka_broker_ua_idle (rd_kafka_broker_t *rkb) { while (!rkb->rkb_rk->rk_terminate && rkb->rkb_state == RD_KAFKA_BROKER_STATE_UP) rd_kafka_broker_io_serve(rkb); } /** * Producer serving */ static void rd_kafka_broker_producer_serve (rd_kafka_broker_t *rkb) { rd_ts_t last_timeout_scan = rd_clock(); rd_kafka_msgq_t timedout = RD_KAFKA_MSGQ_INITIALIZER(timedout); assert(pthread_self() == rkb->rkb_thread); rd_kafka_broker_lock(rkb); while (!rkb->rkb_rk->rk_terminate && rkb->rkb_state == RD_KAFKA_BROKER_STATE_UP) { rd_kafka_toppar_t *rktp; int cnt; rd_ts_t now; int do_timeout_scan = 0; now = rd_clock(); if (unlikely(last_timeout_scan + 1000000 < now)) { do_timeout_scan = 1; last_timeout_scan = now; } rd_kafka_broker_toppars_rdlock(rkb); do { cnt = 0; TAILQ_FOREACH(rktp, &rkb->rkb_toppars, rktp_rkblink) { rd_rkb_dbg(rkb, TOPIC, "TOPPAR", "%.*s [%"PRId32"] %i+%i msgs", RD_KAFKAP_STR_PR(rktp->rktp_rkt-> rkt_topic), rktp->rktp_partition, rktp->rktp_msgq.rkmq_msg_cnt, rktp->rktp_xmit_msgq.rkmq_msg_cnt); rd_kafka_toppar_lock(rktp); if (rktp->rktp_msgq.rkmq_msg_cnt > 0) rd_kafka_msgq_concat(&rktp-> rktp_xmit_msgq, &rktp->rktp_msgq); rd_kafka_toppar_unlock(rktp); /* Timeout scan */ if (unlikely(do_timeout_scan)) rd_kafka_msgq_age_scan(&rktp-> rktp_xmit_msgq, &timedout, now); if (rktp->rktp_xmit_msgq.rkmq_msg_cnt == 0) continue; /* Attempt to fill the batch size, but limit * our waiting to queue.buffering.max.ms * and batch.num.messages. */ if (rktp->rktp_ts_last_xmit + (rkb->rkb_rk->rk_conf. buffering_max_ms * 1000) > now && rktp->rktp_xmit_msgq.rkmq_msg_cnt < rkb->rkb_rk->rk_conf. batch_num_messages) { /* Wait for more messages */ continue; } rktp->rktp_ts_last_xmit = now; /* Send Produce requests for this toppar */ while (rktp->rktp_xmit_msgq.rkmq_msg_cnt > 0) { int r = rd_kafka_broker_produce_toppar( rkb, rktp); if (likely(r > 0)) cnt += r; else break; } } } while (cnt); /* Trigger delivery report for timed out messages */ if (unlikely(timedout.rkmq_msg_cnt > 0)) rd_kafka_dr_msgq(rkb->rkb_rk, &timedout, RD_KAFKA_RESP_ERR__MSG_TIMED_OUT); rd_kafka_broker_toppars_unlock(rkb); /* Check and move retry buffers */ if (unlikely(rkb->rkb_retrybufs.rkbq_cnt) > 0) rd_kafka_broker_retry_bufs_move(rkb); rd_kafka_broker_unlock(rkb); /* Serve IO */ rd_kafka_broker_io_serve(rkb); /* Scan wait-response queue * Note: 'now' may be a bit outdated by now. */ if (do_timeout_scan) rd_kafka_broker_waitresp_timeout_scan(rkb, now); rd_kafka_broker_lock(rkb); } rd_kafka_broker_unlock(rkb); } /** * Decompress Snappy message with Snappy-java framing. * Returns a malloced buffer with the uncompressed data, or NULL on failure. */ static char *rd_kafka_snappy_java_decompress (rd_kafka_broker_t *rkb, int64_t Offset, const char *inbuf, size_t inlen, size_t *outlenp) { int pass; char *outbuf = NULL; /** * Traverse all chunks in two passes: * pass 1: calculate total uncompressed length * pass 2: uncompress * * Each chunk is prefixed with 4: length */ for (pass = 1 ; pass <= 2 ; pass++) { ssize_t of = 0; /* inbuf offset */ ssize_t uof = 0; /* outbuf offset */ while (of + 4 <= inlen) { /* compressed length */ uint32_t clen = ntohl(*(uint32_t *)(inbuf+of)); /* uncompressed length */ size_t ulen; int r; of += 4; if (unlikely(clen > inlen - of)) { rd_rkb_dbg(rkb, MSG, "SNAPPY", "Invalid snappy-java chunk length for " "message at offset %"PRId64" " "(%"PRIu32">%zd: ignoring message", Offset, clen, inlen - of); return NULL; } /* Acquire uncompressed length */ if (unlikely(!snappy_uncompressed_length(inbuf+of, clen, &ulen))) { rd_rkb_dbg(rkb, MSG, "SNAPPY", "Failed to get length of " "(snappy-java framed) Snappy " "compressed payload for message at " "offset %"PRId64" (%"PRId32" bytes): " "ignoring message", Offset, clen); return NULL; } if (pass == 1) { /* pass 1: calculate total length */ of += clen; uof += ulen; continue; } /* pass 2: Uncompress to outbuf */ if (unlikely((r = snappy_uncompress(inbuf+of, clen, outbuf+uof)))) { rd_rkb_dbg(rkb, MSG, "SNAPPY", "Failed to decompress Snappy-java framed " "payload for message at offset %"PRId64 " (%"PRId32" bytes): %s: ignoring message", Offset, clen, strerror(-r/*negative errno*/)); free(outbuf); return NULL; } of += clen; uof += ulen; } if (unlikely(of != inlen)) { rd_rkb_dbg(rkb, MSG, "SNAPPY", "%zd trailing bytes in Snappy-java framed compressed " "data at offset %"PRId64": ignoring message", inlen - of, Offset); return NULL; } if (pass == 1) { if (uof <= 0) { rd_rkb_dbg(rkb, MSG, "SNAPPY", "Empty Snappy-java framed data " "at offset %"PRId64" (%zd bytes): " "ignoring message", Offset, uof); return NULL; } /* Allocate memory for uncompressed data */ outbuf = malloc(uof); if (unlikely(!outbuf)) { rd_rkb_dbg(rkb, MSG, "SNAPPY", "Failed to allocate memory for uncompressed " "Snappy data at offset %"PRId64 " (%zd bytes): %s", Offset, uof, strerror(errno)); return NULL; } } else { /* pass 2 */ *outlenp = uof; } } return outbuf; } /** * Parses a MessageSet and enqueues internal ops on the local * application queue for each Message. */ static rd_kafka_resp_err_t rd_kafka_messageset_handle (rd_kafka_broker_t *rkb, rd_kafka_toppar_t *rktp, rd_kafka_q_t *rkq, rd_kafka_buf_t *rkbuf, void *buf, size_t size) { size_t of = 0; rd_kafka_buf_t *rkbufz; if (_REMAIN() == 0) _FAIL("empty messageset"); while (_REMAIN() > 0) { struct { int64_t Offset; int32_t MessageSize; int32_t Crc; int8_t MagicByte; int8_t Attributes; } RD_PACKED *hdr; rd_kafkap_bytes_t *Key; rd_kafkap_bytes_t *Value; int32_t Value_len; rd_kafka_op_t *rko; size_t outlen; void *outbuf = NULL; _READ_REF(hdr, sizeof(*hdr)); hdr->Offset = be64toh(hdr->Offset); hdr->MessageSize = ntohl(hdr->MessageSize); _CHECK_LEN(hdr->MessageSize - 6/*Crc+Magic+Attr*/); /* Ignore CRC (for now) */ /* Extract key */ _READ_BYTES(Key); /* Extract Value */ _READ_BYTES(Value); Value_len = RD_KAFKAP_BYTES_LEN(Value); /* Check for message compression. * The Key is ignored for compressed messages. */ switch (hdr->Attributes & 0x3) { case RD_KAFKA_COMPRESSION_NONE: /* Pure uncompressed message, this is the innermost * handler after all compression and cascaded * messagesets have been peeled off. */ /* Create op and push on temporary queue. */ rko = rd_kafka_op_new(RD_KAFKA_OP_FETCH); if (!RD_KAFKAP_BYTES_IS_NULL(Key)) { rko->rko_rkmessage.key = Key->data; rko->rko_rkmessage.key_len = RD_KAFKAP_BYTES_LEN(Key); } rko->rko_rkmessage.payload = Value->data; rko->rko_rkmessage.len = Value_len; rko->rko_rkmessage.offset = hdr->Offset; rko->rko_rkmessage.rkt = rktp->rktp_rkt; rko->rko_rkmessage.partition = rktp->rktp_partition; rktp->rktp_next_offset = hdr->Offset + 1; /* Since all the ops share the same payload buffer * (rkbuf->rkbuf_buf2) a refcnt is used on the rkbuf * that makes sure all consume_cb() will have been * called for each of these ops before the rkbuf * and its rkbuf_buf2 are freed. */ rko->rko_rkbuf = rkbuf; rd_kafka_buf_keep(rkbuf); if (0) rd_rkb_dbg(rkb, MSG, "MSG", "Pushed message at offset %"PRId64 " onto queue", hdr->Offset); rd_kafka_q_enq(rkq, rko); break; case RD_KAFKA_COMPRESSION_GZIP: { uint64_t outlenx = 0; /* Decompress Message payload */ outbuf = rd_gz_decompress(Value->data, Value_len, &outlenx); if (unlikely(!outbuf)) { rd_rkb_dbg(rkb, MSG, "GZIP", "Failed to decompress Gzip " "message at offset %"PRId64 " of %"PRId32" bytes: " "ignoring message", hdr->Offset, Value_len); continue; } outlen = outlenx; } break; case RD_KAFKA_COMPRESSION_SNAPPY: { char *inbuf = Value->data; int r; static const char snappy_java_magic[] = { 0x82, 'S','N','A','P','P','Y', 0 }; static const int snappy_java_hdrlen = 8+4+4; /* snappy-java adds its own header (SnappyCodec) * which is not compatible with the official Snappy * implementation. * 8: magic, 4: version, 4: compatible * followed by any number of chunks: * 4: length * ...: snappy-compressed data. */ if (likely(Value_len > snappy_java_hdrlen + 4 && !memcmp(inbuf, snappy_java_magic, 8))) { /* snappy-java framing */ inbuf = inbuf + snappy_java_hdrlen; Value_len -= snappy_java_hdrlen; outbuf = rd_kafka_snappy_java_decompress(rkb, hdr->Offset, inbuf, Value_len, &outlen); if (unlikely(!outbuf)) continue; } else { /* no framing */ /* Acquire uncompressed length */ if (unlikely(!snappy_uncompressed_length(inbuf, Value_len, &outlen))) { rd_rkb_dbg(rkb, MSG, "SNAPPY", "Failed to get length of Snappy " "compressed payload " "for message at offset %"PRId64 " (%"PRId32" bytes): " "ignoring message", hdr->Offset, Value_len); continue; } /* Allocate output buffer for uncompressed data */ outbuf = malloc(outlen); /* Uncompress to outbuf */ if (unlikely((r = snappy_uncompress(inbuf, Value_len, outbuf)))) { rd_rkb_dbg(rkb, MSG, "SNAPPY", "Failed to decompress Snappy " "payload for message at offset " "%"PRId64 " (%"PRId32" bytes): %s: " "ignoring message", hdr->Offset, Value_len, strerror(-r/*negative errno*/)); free(outbuf); continue; } } } break; } if (outbuf) { /* With a new allocated buffer (outbuf) we need * a separate rkbuf for it to allow multiple fetch ops * to share the same payload buffer. */ rkbufz = rd_kafka_buf_new_shadow(outbuf, outlen); /* Now parse the contained Messages */ rd_kafka_messageset_handle(rkb, rktp, rkq, rkbufz, outbuf, outlen); /* Loose our refcnt of the rkbuf. * Individual rko's will have their own. */ rd_kafka_buf_destroy(rkbufz); } } return 0; err: /* "A Guide To The Kafka Protocol" states: * "As an optimization the server is allowed to return a partial * message at the end of the message set. * Clients should handle this case." * We're handling it by not passing the error upstream. */ return RD_KAFKA_RESP_ERR_NO_ERROR; } /** * Backoff the next Fetch request (due to error). */ static void rd_kafka_broker_fetch_backoff (rd_kafka_broker_t *rkb) { rkb->rkb_ts_fetch_backoff = rd_clock() + (rkb->rkb_rk->rk_conf.fetch_error_backoff_ms*1000); } /** * Parses and handles a Fetch reply. * Returns 0 on success or an error code on failure. */ static rd_kafka_resp_err_t rd_kafka_fetch_reply_handle (rd_kafka_broker_t *rkb, rd_kafka_buf_t *rkbuf) { char *buf = rkbuf->rkbuf_buf2; size_t size = rkbuf->rkbuf_len; size_t of = 0; int32_t TopicArrayCnt; int i; _READ_I32(&TopicArrayCnt); /* Verify that TopicArrayCnt seems to be in line with remaining size */ _CHECK_LEN(TopicArrayCnt * (3/*topic min size*/ + 4/*PartitionArrayCnt*/ + 4+2+8+4/*inner header*/)); for (i = 0 ; i < TopicArrayCnt ; i++) { rd_kafkap_str_t *topic; rd_kafka_toppar_t *rktp; int32_t PartitionArrayCnt; struct { int32_t Partition; int16_t ErrorCode; int64_t HighwaterMarkOffset; int32_t MessageSetSize; } RD_PACKED *hdr; rd_kafka_resp_err_t err2; int j; _READ_STR(topic); _READ_I32(&PartitionArrayCnt); _CHECK_LEN(PartitionArrayCnt * (4+2+8+4/*inner header*/)); for (j = 0 ; j < PartitionArrayCnt ; j++) { rd_kafka_q_t tmp_opq; /* Temporary queue for ops */ _READ_REF(hdr, sizeof(*hdr)); hdr->Partition = ntohl(hdr->Partition); hdr->ErrorCode = ntohs(hdr->ErrorCode); hdr->HighwaterMarkOffset = be64toh(hdr-> HighwaterMarkOffset); hdr->MessageSetSize = ntohl(hdr->MessageSetSize); /* Look up topic+partition */ rktp = rd_kafka_toppar_get2(rkb->rkb_rk, topic, hdr->Partition, 0); if (unlikely(!rktp)) { rd_rkb_dbg(rkb, TOPIC, "UNKTOPIC", "Received Fetch response " "(error %hu) for unknown topic " "%.*s [%"PRId32"]: ignoring", hdr->ErrorCode, RD_KAFKAP_STR_PR(topic), hdr->Partition); _SKIP(hdr->MessageSetSize); continue; } rd_rkb_dbg(rkb, MSG, "FETCH", "Topic %.*s [%"PRId32"] MessageSet " "size %"PRId32", error \"%s\", " "MaxOffset %"PRId64, RD_KAFKAP_STR_PR(topic), hdr->Partition, hdr->MessageSetSize, rd_kafka_err2str(hdr->ErrorCode), hdr->HighwaterMarkOffset); /* If this is the last message of the queue, * signal EOF back to the application. */ if (hdr->HighwaterMarkOffset == rktp->rktp_next_offset && rktp->rktp_eof_offset != rktp->rktp_next_offset) { hdr->ErrorCode = RD_KAFKA_RESP_ERR__PARTITION_EOF; rktp->rktp_eof_offset = rktp->rktp_next_offset; } /* Handle partition-level errors. */ if (unlikely(hdr->ErrorCode != RD_KAFKA_RESP_ERR_NO_ERROR)) { rd_kafka_op_t *rko; /* Some errors should be passed to the * application while some handled by rdkafka */ switch (hdr->ErrorCode) { /* Errors handled by rdkafka */ case RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_OR_PART: case RD_KAFKA_RESP_ERR_LEADER_NOT_AVAILABLE: case RD_KAFKA_RESP_ERR_NOT_LEADER_FOR_PARTITION: case RD_KAFKA_RESP_ERR_BROKER_NOT_AVAILABLE: /* Request metadata information update*/ rd_kafka_topic_leader_query(rkb->rkb_rk, rktp-> rktp_rkt); break; /* Application errors */ case RD_KAFKA_RESP_ERR_OFFSET_OUT_OF_RANGE: rd_kafka_offset_reset(rktp, rktp-> rktp_next_offset, hdr->ErrorCode, "Fetch response"); break; case RD_KAFKA_RESP_ERR__PARTITION_EOF: case RD_KAFKA_RESP_ERR_MSG_SIZE_TOO_LARGE: default: /* and all other errors */ rko = rd_kafka_op_new(RD_KAFKA_OP_ERR); rko->rko_err = hdr->ErrorCode; rko->rko_rkmessage.offset = rktp->rktp_next_offset; rko->rko_rkmessage.rkt = rktp->rktp_rkt; rko->rko_rkmessage.partition = rktp->rktp_partition; rd_kafka_q_enq(&rktp->rktp_fetchq, rko); break; } rd_kafka_broker_fetch_backoff(rkb); rd_kafka_toppar_destroy(rktp); /* from get2() */ continue; } if (hdr->MessageSetSize <= 0) { rd_kafka_toppar_destroy(rktp); /* from get2() */ continue; } /* All parsed messages are put on this temporary op * queue first and then moved in one go to the * real op queue. */ rd_kafka_q_init(&tmp_opq); /* Parse and handle the message set */ err2 = rd_kafka_messageset_handle(rkb, rktp, &tmp_opq, rkbuf, buf+of, hdr->MessageSetSize); if (err2) { rd_kafka_toppar_destroy(rktp); /* from get2() */ _FAIL("messageset handle failed"); } /* Concat all messages onto the real op queue */ rd_rkb_dbg(rkb, MSG, "CONSUME", "Enqueue %i messages on %s [%"PRId32"] " "fetch queue (%i)", tmp_opq.rkq_qlen, rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition, rd_kafka_q_len(&rktp->rktp_fetchq)); if (tmp_opq.rkq_qlen > 0) rd_kafka_q_concat(&rktp->rktp_fetchq, &tmp_opq); rd_kafka_toppar_destroy(rktp); /* from get2() */ rd_kafka_q_destroy(&tmp_opq); _SKIP(hdr->MessageSetSize); } } if (_REMAIN() != 0) _FAIL("Remaining data after message set parse: %zd bytes", _REMAIN()); return 0; err: return RD_KAFKA_RESP_ERR__BAD_MSG; } static void rd_kafka_broker_fetch_reply (rd_kafka_broker_t *rkb, rd_kafka_resp_err_t err, rd_kafka_buf_t *reply, rd_kafka_buf_t *request, void *opaque) { assert(rkb->rkb_fetching > 0); rkb->rkb_fetching = 0; /* Parse and handle the messages (unless the request errored) */ if (!err && reply) err = rd_kafka_fetch_reply_handle(rkb, reply); rd_rkb_dbg(rkb, MSG, "FETCH", "Fetch reply: %s", rd_kafka_err2str(err)); if (unlikely(err)) { switch (err) { case RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_OR_PART: case RD_KAFKA_RESP_ERR_LEADER_NOT_AVAILABLE: case RD_KAFKA_RESP_ERR_NOT_LEADER_FOR_PARTITION: case RD_KAFKA_RESP_ERR_BROKER_NOT_AVAILABLE: case RD_KAFKA_RESP_ERR_REPLICA_NOT_AVAILABLE: /* Request metadata information update */ rd_kafka_topic_leader_query(rkb->rkb_rk, NULL); /* FALLTHRU */ case RD_KAFKA_RESP_ERR__TRANSPORT: case RD_KAFKA_RESP_ERR_REQUEST_TIMED_OUT: case RD_KAFKA_RESP_ERR__MSG_TIMED_OUT: /* Try again */ /* FIXME: Not sure we should retry here, the fetch * is already intervalled. */ if (++request->rkbuf_retries < /* FIXME: producer? */ rkb->rkb_rk->rk_conf.max_retries) { if (reply) rd_kafka_buf_destroy(reply); rd_kafka_broker_buf_retry(rkb, request); return; } break; default: break; } rd_kafka_broker_fetch_backoff(rkb); /* FALLTHRU */ } rd_kafka_buf_destroy(request); if (reply) rd_kafka_buf_destroy(reply); } /** * Parse and handle an OffsetResponse message. * Returns an error code. */ static rd_kafka_resp_err_t rd_kafka_toppar_offset_reply_handle (rd_kafka_broker_t *rkb, rd_kafka_buf_t *rkbuf, rd_kafka_toppar_t *rktp) { char *buf = rkbuf->rkbuf_buf2; size_t size = rkbuf->rkbuf_len; size_t of = 0; int32_t TopicArrayCnt; int i; _READ_I32(&TopicArrayCnt); for (i = 0 ; i < TopicArrayCnt ; i++) { rd_kafkap_str_t *topic; int32_t PartitionOffsetsArrayCnt; int j; _READ_STR(topic); _READ_I32(&PartitionOffsetsArrayCnt); for (j = 0 ; j < PartitionOffsetsArrayCnt ; j++) { int32_t Partition; int16_t ErrorCode; int32_t OffsetArrayCnt; int64_t Offset; _READ_I32(&Partition); _READ_I16(&ErrorCode); _READ_I32(&OffsetArrayCnt); /* Skip toppars we didnt ask for. */ if (unlikely(rktp->rktp_partition != Partition || rd_kafkap_str_cmp(rktp-> rktp_rkt-> rkt_topic, topic))) continue; if (unlikely(ErrorCode != RD_KAFKA_RESP_ERR_NO_ERROR)) return ErrorCode; /* No offsets returned, convert to error code. */ if (OffsetArrayCnt == 0) return RD_KAFKA_RESP_ERR_OFFSET_OUT_OF_RANGE; /* We only asked for one offset, so just read the * first one returned. */ _READ_I64(&Offset); rd_rkb_dbg(rkb, TOPIC, "OFFSET", "OffsetReply for topic %s [%"PRId32"]: " "offset %"PRId64": activating fetch", rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition, Offset); rktp->rktp_next_offset = Offset; rktp->rktp_fetch_state = RD_KAFKA_TOPPAR_FETCH_ACTIVE; /* We just asked for one toppar and one offset, so * we're probably done now. */ return RD_KAFKA_RESP_ERR_NO_ERROR; } } return RD_KAFKA_RESP_ERR_NO_ERROR; err: return RD_KAFKA_RESP_ERR__BAD_MSG; } /** * Parses and handles an OffsetRequest reply. */ static void rd_kafka_toppar_offset_reply (rd_kafka_broker_t *rkb, rd_kafka_resp_err_t err, rd_kafka_buf_t *reply, rd_kafka_buf_t *request, void *opaque) { rd_kafka_toppar_t *rktp = opaque; rd_kafka_op_t *rko; if (likely(!err && reply)) err = rd_kafka_toppar_offset_reply_handle(rkb, reply, rktp); rd_rkb_dbg(rkb, TOPIC, "OFFSET", "OffsetReply for topic %s [%"PRId32"]: %s", rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition, rd_kafka_err2str(err)); if (unlikely(err)) { switch (err) { case RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_OR_PART: case RD_KAFKA_RESP_ERR_LEADER_NOT_AVAILABLE: case RD_KAFKA_RESP_ERR_NOT_LEADER_FOR_PARTITION: case RD_KAFKA_RESP_ERR_BROKER_NOT_AVAILABLE: case RD_KAFKA_RESP_ERR_REPLICA_NOT_AVAILABLE: /* Request metadata information update */ rd_kafka_topic_leader_query(rkb->rkb_rk, rktp->rktp_rkt); /* FALLTHRU */ case RD_KAFKA_RESP_ERR__TRANSPORT: case RD_KAFKA_RESP_ERR_REQUEST_TIMED_OUT: case RD_KAFKA_RESP_ERR__MSG_TIMED_OUT: /* Try again */ if (++request->rkbuf_retries < /* FIXME: producer? */ rkb->rkb_rk->rk_conf.max_retries) { if (reply) rd_kafka_buf_destroy(reply); rd_kafka_broker_buf_retry(rkb, request); return; } break; default: break; } /* Backoff until next retry */ rktp->rktp_ts_offset_req_next = rd_clock() + 500000; /* 500ms */ rktp->rktp_fetch_state = RD_KAFKA_TOPPAR_FETCH_OFFSET_QUERY; /* Signal error back to application */ rko = rd_kafka_op_new(RD_KAFKA_OP_ERR); rko->rko_err = err; rko->rko_rkmessage.offset = rktp->rktp_query_offset; rko->rko_rkmessage.rkt = rktp->rktp_rkt; rko->rko_rkmessage.partition = rktp->rktp_partition; rd_kafka_q_enq(&rktp->rktp_fetchq, rko); /* FALLTHRU */ } rd_kafka_toppar_destroy(rktp); /* refcnt from request */ rd_kafka_buf_destroy(request); if (reply) rd_kafka_buf_destroy(reply); } /** * Send OffsetRequest for toppar. */ static void rd_kafka_toppar_offset_request (rd_kafka_broker_t *rkb, rd_kafka_toppar_t *rktp) { struct { int32_t ReplicaId; int32_t TopicArrayCnt; } RD_PACKED *part1; /* Topic goes here */ struct { int32_t PartitionArrayCnt; int32_t Partition; int64_t Time; int32_t MaxNumberOfOffsets; } RD_PACKED *part2; rd_kafka_buf_t *rkbuf; rkbuf = rd_kafka_buf_new(3/*part1,topic,part2*/, sizeof(*part1) + sizeof(*part2)); part1 = (void *)rkbuf->rkbuf_buf; part1->ReplicaId = htonl(-1); part1->TopicArrayCnt = htonl(1); rd_kafka_buf_push(rkbuf, part1, sizeof(*part1)); rd_kafka_buf_push(rkbuf, rktp->rktp_rkt->rkt_topic, RD_KAFKAP_STR_SIZE(rktp->rktp_rkt->rkt_topic)); rd_kafka_toppar_lock(rktp); part2 = (void *)(part1+1); part2->PartitionArrayCnt = htonl(1); part2->Partition = htonl(rktp->rktp_partition); part2->Time = htobe64(rktp->rktp_query_offset); part2->MaxNumberOfOffsets = htonl(1); rd_kafka_buf_push(rkbuf, part2, sizeof(*part2)); rktp->rktp_fetch_state = RD_KAFKA_TOPPAR_FETCH_OFFSET_WAIT; rd_kafka_toppar_unlock(rktp); rd_kafka_toppar_keep(rktp); /* refcnt for request */ rd_rkb_dbg(rkb, TOPIC, "OFFSET", "OffsetRequest (%"PRId64") for topic %s [%"PRId32"]", rktp->rktp_query_offset, rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition); rkbuf->rkbuf_ts_timeout = rd_clock() + rkb->rkb_rk->rk_conf.socket_timeout_ms * 1000; rd_kafka_broker_buf_enq1(rkb, RD_KAFKAP_Offset, rkbuf, rd_kafka_toppar_offset_reply, rktp); } /** * Build and send a Fetch request message for all underflowed toppars * for a specific broker. */ static int rd_kafka_broker_fetch_toppars (rd_kafka_broker_t *rkb) { struct rd_kafkap_FetchRequest *fr; struct { /* Per toppar header */ int32_t PartitionArrayCnt; int32_t Partition; int64_t FetchOffset; int32_t MaxBytes; } RD_PACKED *tp; rd_kafka_toppar_t *rktp; char *next; int cnt = 0; rd_kafka_buf_t *rkbuf; rd_ts_t now = rd_clock(); /* Create buffer and iovecs: * 1 x part1 header * N x topic name * N x tp header * where N = number of toppars */ rd_kafka_broker_toppars_rdlock(rkb); rkbuf = rd_kafka_buf_new(1 + (rkb->rkb_toppar_cnt * (1 + 1)), sizeof(*fr) + (sizeof(*tp) * rkb->rkb_toppar_cnt)); /* Set up header from pre-built header */ fr = (void *)rkbuf->rkbuf_buf; *fr = rkb->rkb_rk->rk_conf.FetchRequest; rd_kafka_buf_push(rkbuf, fr, sizeof(*fr)); next = (void *)(fr+1); TAILQ_FOREACH(rktp, &rkb->rkb_toppars, rktp_rkblink) { /* Check Toppar Fetch state */ if (unlikely(rktp->rktp_fetch_state != RD_KAFKA_TOPPAR_FETCH_ACTIVE)) { switch (rktp->rktp_fetch_state) { case RD_KAFKA_TOPPAR_FETCH_OFFSET_QUERY: if (rktp->rktp_ts_offset_req_next <= now) rd_kafka_toppar_offset_request(rkb, rktp); break; default: break; } /* Skip this toppar until its state changes */ rd_rkb_dbg(rkb, TOPIC, "FETCH", "Skipping topic %s [%"PRId32"] in state %s", rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition, rd_kafka_fetch_states[rktp-> rktp_fetch_state]); continue; } /* Skip toppars who's local message queue is already above * the lower threshold. */ if (rd_kafka_q_len(&rktp->rktp_fetchq) >= rkb->rkb_rk->rk_conf.queued_min_msgs) continue; /* Push topic name onto buffer stack. */ rd_kafka_buf_push(rkbuf, rktp->rktp_rkt->rkt_topic, RD_KAFKAP_STR_SIZE(rktp->rktp_rkt-> rkt_topic)); /* Set up toppar header and push it */ tp = (void *)next; tp->PartitionArrayCnt = htonl(1); tp->Partition = htonl(rktp->rktp_partition); tp->FetchOffset = htobe64(rktp->rktp_next_offset); tp->MaxBytes = htonl(rkb->rkb_rk->rk_conf.fetch_msg_max_bytes); rd_kafka_buf_push(rkbuf, tp, sizeof(*tp)); next = (void *)(tp+1); rd_rkb_dbg(rkb, TOPIC, "FETCH", "Fetch topic %.*s [%"PRId32"] at offset %"PRId64, RD_KAFKAP_STR_PR(rktp->rktp_rkt->rkt_topic), rktp->rktp_partition, rktp->rktp_next_offset); cnt++; } rd_kafka_broker_toppars_unlock(rkb); rd_rkb_dbg(rkb, MSG, "CONSUME", "consume from %i toppar(s)", cnt); if (!cnt) { rd_kafka_buf_destroy(rkbuf); return cnt; } fr->TopicArrayCnt = htonl(cnt); /* Use configured timeout */ rkbuf->rkbuf_ts_timeout = now + ((rkb->rkb_rk->rk_conf.socket_timeout_ms + rkb->rkb_rk->rk_conf.fetch_wait_max_ms) * 1000); rkb->rkb_fetching = 1; rd_kafka_broker_buf_enq1(rkb, RD_KAFKAP_Fetch, rkbuf, rd_kafka_broker_fetch_reply, NULL); return cnt; } /** * Consumer serving */ static void rd_kafka_broker_consumer_serve (rd_kafka_broker_t *rkb) { assert(pthread_self() == rkb->rkb_thread); rd_kafka_broker_lock(rkb); while (!rkb->rkb_rk->rk_terminate && rkb->rkb_state == RD_KAFKA_BROKER_STATE_UP) { int cnt = 0; rd_ts_t now; int do_timeout_scan = 1; /* FIXME */ now = rd_clock(); /* Send Fetch request message for all underflowed toppars */ if (!rkb->rkb_fetching && rkb->rkb_ts_fetch_backoff < now) cnt = rd_kafka_broker_fetch_toppars(rkb); rd_rkb_dbg(rkb, BROKER, "FETCH", "Fetch for %i toppars, fetching=%i", cnt, rkb->rkb_fetching); /* Check and move retry buffers */ if (unlikely(rkb->rkb_retrybufs.rkbq_cnt) > 0) rd_kafka_broker_retry_bufs_move(rkb); rd_kafka_broker_unlock(rkb); /* Serve IO */ rd_kafka_broker_io_serve(rkb); /* Scan wait-response queue * Note: 'now' may be a bit outdated by now. */ if (do_timeout_scan) rd_kafka_broker_waitresp_timeout_scan(rkb, now); rd_kafka_broker_lock(rkb); } rd_kafka_broker_unlock(rkb); } static void *rd_kafka_broker_thread_main (void *arg) { rd_kafka_broker_t *rkb = arg; rd_kafka_t *rk = rkb->rkb_rk; (void)rd_atomic_add(&rd_kafka_thread_cnt_curr, 1); rd_thread_sigmask(SIG_BLOCK, SIGHUP, SIGINT, SIGTERM, SIGUSR1, SIGUSR2, RD_SIG_END); rd_rkb_dbg(rkb, BROKER, "BRKMAIN", "Enter main broker thread"); while (!rkb->rkb_rk->rk_terminate) { switch (rkb->rkb_state) { case RD_KAFKA_BROKER_STATE_INIT: /* The INIT state exists so that an initial connection * failure triggers a state transition which might * trigger a ALL_BROKERS_DOWN error. */ case RD_KAFKA_BROKER_STATE_DOWN: /* ..connect() will block until done (or failure) */ if (rd_kafka_broker_connect(rkb) == -1) { /* Try the next one until we've tried them all, * in which case we sleep a short while to * avoid the busy looping. */ if (!rkb->rkb_rsal || rkb->rkb_rsal->rsal_cnt == 0 || rkb->rkb_rsal->rsal_curr + 1 == rkb->rkb_rsal->rsal_cnt) sleep(1); } break; case RD_KAFKA_BROKER_STATE_UP: if (rkb->rkb_nodeid == RD_KAFKA_NODEID_UA) rd_kafka_broker_ua_idle(rkb); else if (rk->rk_type == RD_KAFKA_PRODUCER) rd_kafka_broker_producer_serve(rkb); else if (rk->rk_type == RD_KAFKA_CONSUMER) rd_kafka_broker_consumer_serve(rkb); break; } } rd_kafka_lock(rkb->rkb_rk); TAILQ_REMOVE(&rkb->rkb_rk->rk_brokers, rkb, rkb_link); rd_atomic_sub(&rkb->rkb_rk->rk_broker_cnt, 1); rd_kafka_unlock(rkb->rkb_rk); rd_kafka_broker_fail(rkb, RD_KAFKA_RESP_ERR__DESTROY, NULL); rd_kafka_broker_destroy(rkb); (void)rd_atomic_sub(&rd_kafka_thread_cnt_curr, 1); return NULL; } void rd_kafka_broker_destroy (rd_kafka_broker_t *rkb) { rd_kafka_op_t *rko; if (rd_atomic_sub(&rkb->rkb_refcnt, 1) > 0) return; assert(TAILQ_EMPTY(&rkb->rkb_outbufs.rkbq_bufs)); assert(TAILQ_EMPTY(&rkb->rkb_toppars)); if (rkb->rkb_recv_buf) rd_kafka_buf_destroy(rkb->rkb_recv_buf); if (rkb->rkb_rsal) rd_sockaddr_list_destroy(rkb->rkb_rsal); /* Clean up any references in the op queue before purging it */ TAILQ_FOREACH(rko, &rkb->rkb_ops.rkq_q, rko_link) { switch (rko->rko_type) { case RD_KAFKA_OP_METADATA_REQ: if (rko->rko_rkt) rd_kafka_topic_destroy0(rko->rko_rkt); break; default: break; } } rd_kafka_q_purge(&rkb->rkb_ops); rd_kafka_q_destroy(&rkb->rkb_ops); rd_kafka_destroy0(rkb->rkb_rk); pthread_rwlock_destroy(&rkb->rkb_toppar_lock); pthread_mutex_destroy(&rkb->rkb_lock); free(rkb); } /** * * Locks: rd_kafka_lock(rk) must be held */ static rd_kafka_broker_t *rd_kafka_broker_add (rd_kafka_t *rk, rd_kafka_confsource_t source, const char *name, uint16_t port, int32_t nodeid) { rd_kafka_broker_t *rkb; int err; pthread_attr_t attr; rd_kafka_keep(rk); rkb = calloc(1, sizeof(*rkb)); snprintf(rkb->rkb_nodename, sizeof(rkb->rkb_nodename), "%s:%hu", name, port); if (nodeid == RD_KAFKA_NODEID_UA) snprintf(rkb->rkb_name, sizeof(rkb->rkb_name), "%s/bootstrap", rkb->rkb_nodename); else snprintf(rkb->rkb_name, sizeof(rkb->rkb_name), "%s/%"PRId32, rkb->rkb_nodename, nodeid); rkb->rkb_source = source; rkb->rkb_rk = rk; rkb->rkb_nodeid = nodeid; rkb->rkb_s = -1; pthread_rwlock_init(&rkb->rkb_toppar_lock, NULL); TAILQ_INIT(&rkb->rkb_toppars); rd_kafka_bufq_init(&rkb->rkb_outbufs); rd_kafka_bufq_init(&rkb->rkb_waitresps); rd_kafka_bufq_init(&rkb->rkb_retrybufs); rd_kafka_q_init(&rkb->rkb_ops); rd_kafka_broker_keep(rkb); /* Set next intervalled metadata refresh, offset by a random * value to avoid all brokers to be queried simultaneously. */ if (rkb->rkb_rk->rk_conf.metadata_refresh_interval_ms >= 0) rkb->rkb_ts_metadata_poll = rd_clock() + (rkb->rkb_rk->rk_conf. metadata_refresh_interval_ms * 1000) + (rd_jitter(500,1500) * 1000); else /* disabled */ rkb->rkb_ts_metadata_poll = UINT64_MAX; pthread_attr_init(&attr); pthread_attr_setdetachstate(&attr, PTHREAD_CREATE_DETACHED); if ((err = pthread_create(&rkb->rkb_thread, &attr, rd_kafka_broker_thread_main, rkb))) { char tmp[512]; snprintf(tmp, sizeof(tmp), "Unable to create broker thread: %s", strerror(err)); rd_kafka_log(rk, LOG_CRIT, "THREAD", "%s", tmp); /* Send ERR op back to application for processing. */ rd_kafka_op_err(rk, RD_KAFKA_RESP_ERR__CRIT_SYS_RESOURCE, "%s", tmp); free(rkb); rd_kafka_destroy(rk); return NULL; } TAILQ_INSERT_TAIL(&rkb->rkb_rk->rk_brokers, rkb, rkb_link); rd_atomic_add(&rkb->rkb_rk->rk_broker_cnt, 1); rd_rkb_dbg(rkb, BROKER, "BROKER", "Added new broker with NodeId %"PRId32, rkb->rkb_nodeid); return rkb; } /** * Locks: rd_kafka_lock() * NOTE: caller must release rkb reference by rd_kafka_broker_destroy() */ rd_kafka_broker_t *rd_kafka_broker_find_by_nodeid (rd_kafka_t *rk, int32_t nodeid) { rd_kafka_broker_t *rkb; TAILQ_FOREACH(rkb, &rk->rk_brokers, rkb_link) { rd_kafka_broker_lock(rkb); if (!rk->rk_terminate && rkb->rkb_nodeid == nodeid) { rd_kafka_broker_keep(rkb); rd_kafka_broker_unlock(rkb); return rkb; } rd_kafka_broker_unlock(rkb); } return NULL; } /** * Locks: rd_kafka_lock(rk) must be held */ static rd_kafka_broker_t *rd_kafka_broker_find (rd_kafka_t *rk, const char *name, uint16_t port) { rd_kafka_broker_t *rkb; char fullname[sizeof(rkb->rkb_name)]; snprintf(fullname, sizeof(fullname), "%s:%hu", name, port); TAILQ_FOREACH(rkb, &rk->rk_brokers, rkb_link) { rd_kafka_broker_lock(rkb); if (!rk->rk_terminate && !strcmp(rkb->rkb_name, fullname)) { rd_kafka_broker_keep(rkb); rd_kafka_broker_unlock(rkb); return rkb; } rd_kafka_broker_unlock(rkb); } return NULL; } int rd_kafka_brokers_add (rd_kafka_t *rk, const char *brokerlist) { char *s = strdupa(brokerlist); char *t, *t2, *n; int cnt = 0; rd_kafka_broker_t *rkb; /* Parse comma-separated list of brokers. */ while (*s) { uint16_t port = 0; if (*s == ',' || *s == ' ') { s++; continue; } if ((n = strchr(s, ','))) *n = '\0'; else n = s + strlen(s)-1; /* Check if port has been specified, but try to identify IPv6 * addresses first: * t = last ':' in string * t2 = first ':' in string * If t and t2 are equal then only one ":" exists in name * and thus an IPv4 address with port specified. * Else if not equal and t is prefixed with "]" then it's an * IPv6 address with port specified. * Else no port specified. */ if ((t = strrchr(s, ':')) && ((t2 = strchr(s, ':')) == t || *(t-1) == ']')) { *t = '\0'; port = atoi(t+1); } if (!port) port = RD_KAFKA_PORT; rd_kafka_lock(rk); if ((rkb = rd_kafka_broker_find(rk, s, port)) && rkb->rkb_source == RD_KAFKA_CONFIGURED) { cnt++; } else if (rd_kafka_broker_add(rk, RD_KAFKA_CONFIGURED, s, port, RD_KAFKA_NODEID_UA) != NULL) cnt++; rd_kafka_unlock(rk); s = n+1; } return cnt; } /** * Adds a new broker or updates an existing one. */ static void rd_kafka_broker_update (rd_kafka_t *rk, const char *name, uint16_t port, uint32_t nodeid) { rd_kafka_broker_t *rkb; rd_kafka_lock(rk); if ((rkb = rd_kafka_broker_find_by_nodeid(rk, nodeid))) rd_kafka_broker_destroy(rkb); else rd_kafka_broker_add(rk, RD_KAFKA_LEARNED, name, port, nodeid); rd_kafka_unlock(rk); /* FIXME: invalidate Leader if required. */ } void rd_kafka_brokers_init (void) { } librdkafka-0.8.3/rdkafka_broker.h000066400000000000000000000040161227620010100167240ustar00rootroot00000000000000/* * librdkafka - Apache Kafka C library * * Copyright (c) 2012,2013 Magnus Edenhill * All rights reserved. * * Redistribution and use in source and binary forms, with or without * modification, are permitted provided that the following conditions are met: * * 1. Redistributions of source code must retain the above copyright notice, * this list of conditions and the following disclaimer. * 2. Redistributions in binary form must reproduce the above copyright notice, * this list of conditions and the following disclaimer in the documentation * and/or other materials provided with the distribution. * * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE * POSSIBILITY OF SUCH DAMAGE. */ #pragma once extern const char *rd_kafka_broker_state_names[]; #define rd_kafka_buf_keep(rkbuf) (void)rd_atomic_add(&(rkbuf)->rkbuf_refcnt, 1) void rd_kafka_buf_destroy (rd_kafka_buf_t *rkbuf); rd_kafka_broker_t *rd_kafka_broker_find_by_nodeid (rd_kafka_t *rk, int32_t nodeid); void rd_kafka_topic_leader_query0 (rd_kafka_t *rk, rd_kafka_topic_t *rkt, int do_rk_lock); #define rd_kafka_topic_leader_query(rk,rkt) \ rd_kafka_topic_leader_query0(rk,rkt,1) void rd_kafka_broker_destroy (rd_kafka_broker_t *rkb); void rd_kafka_dr_msgq (rd_kafka_t *rk, rd_kafka_msgq_t *rkmq, rd_kafka_resp_err_t err); librdkafka-0.8.3/rdkafka_defaultconf.c000077500000000000000000000601621227620010100177340ustar00rootroot00000000000000/* * librdkafka - Apache Kafka C library * * Copyright (c) 2012,2013 Magnus Edenhill * All rights reserved. * * Redistribution and use in source and binary forms, with or without * modification, are permitted provided that the following conditions are met: * * 1. Redistributions of source code must retain the above copyright notice, * this list of conditions and the following disclaimer. * 2. Redistributions in binary form must reproduce the above copyright notice, * this list of conditions and the following disclaimer in the documentation * and/or other materials provided with the distribution. * * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE * POSSIBILITY OF SUCH DAMAGE. */ #include #include #include #include #include "rdkafka.h" #include "rdkafka_int.h" #include "rd.h" struct rd_kafka_property { rd_kafka_conf_scope_t scope; const char *name; enum { _RK_C_STR, _RK_C_INT, _RK_C_S2I, /* String to Integer mapping */ _RK_C_S2F, /* CSV String to Integer flag mapping (OR:ed) */ _RK_C_BOOL, _RK_C_PTR, /* Only settable through special set functions */ } type; int offset; const char *desc; int vmin; int vmax; int vdef; /* Default value (int) */ const char *sdef; /* Defalut value (string) */ struct { int val; const char *str; } s2i[10]; /* _RK_C_S2I and _RK_C_S2F */ }; #define _RK(field) offsetof(rd_kafka_conf_t, field) #define _RKT(field) offsetof(rd_kafka_topic_conf_t, field) /** * librdkafka configuration property definitions. */ static const struct rd_kafka_property rd_kafka_properties[] = { /* Global properties */ { _RK_GLOBAL, "client.id", _RK_C_STR, _RK(clientid), "Client identifier.", .sdef = "rdkafka" }, { _RK_GLOBAL, "metadata.broker.list", _RK_C_STR, _RK(brokerlist), "Initial list of brokers. " "The application may also use `rd_kafka_brokers_add()` to add " "brokers during runtime." }, { _RK_GLOBAL, "message.max.bytes", _RK_C_INT, _RK(max_msg_size), "Maximum transmit message size.", 1000, 1000000000, 4000000 }, { _RK_GLOBAL, "receive.message.max.bytes", _RK_C_INT, _RK(recv_max_msg_size), "Maximum receive message size. " "This is a safety precaution to avoid memory exhaustion in case of " "protocol hickups. " "The value should be at least fetch.message.max.bytes * number of " "partitions consumed from.", 1000, 1000000000, 100000000 }, { _RK_GLOBAL, "metadata.request.timeout.ms", _RK_C_INT, _RK(metadata_request_timeout_ms), "Non-topic request timeout in milliseconds. " "This is for metadata requests, etc.", /* FIXME: OffsetReq? */ 10, 900*1000, 60*1000}, { _RK_GLOBAL, "topic.metadata.refresh.interval.ms", _RK_C_INT, _RK(metadata_refresh_interval_ms), "Topic metadata refresh interval in milliseconds. " "The metadata is automatically refreshed on error and connect. " "Use -1 to disable the intervalled refresh.", -1, 3600*1000, 10*1000 }, { _RK_GLOBAL, "topic.metadata.refresh.fast.cnt", _RK_C_INT, _RK(metadata_refresh_fast_cnt), "When a topic looses its leader this number of metadata requests " "are sent with `topic.metadata.refresh.fast.interval.ms` interval " "disregarding the `topic.metadata.refresh.interval.ms` value. " "This is used to recover quickly from transitioning leader brokers.", 0, 1000, 10 }, { _RK_GLOBAL, "topic.metadata.refresh.fast.interval.ms", _RK_C_INT, _RK(metadata_refresh_fast_interval_ms), "See `topic.metadata.refresh.fast.cnt` description", 1, 60*1000, 250 }, { _RK_GLOBAL, "debug", _RK_C_S2F, _RK(debug), "A comma-separated list of debug contexts to enable: " RD_KAFKA_DEBUG_CONTEXTS, .s2i = { { RD_KAFKA_DBG_GENERIC, "generic" }, { RD_KAFKA_DBG_BROKER, "broker" }, { RD_KAFKA_DBG_TOPIC, "topic" }, { RD_KAFKA_DBG_METADATA, "metadata" }, { RD_KAFKA_DBG_PRODUCER, "producer" }, { RD_KAFKA_DBG_QUEUE, "queue" }, { RD_KAFKA_DBG_MSG, "msg" }, { RD_KAFKA_DBG_ALL, "all" }, } }, { _RK_GLOBAL, "socket.timeout.ms", _RK_C_INT, _RK(socket_timeout_ms), "Timeout for network requests.", 10, 300*1000, 60*1000 }, { _RK_GLOBAL, "socket.send.buffer.bytes", _RK_C_INT, _RK(socket_sndbuf_size), "Broker socket send buffer size. System default is used if 0.", 0, 100000000, 0 }, { _RK_GLOBAL, "socket.receive.buffer.bytes", _RK_C_INT, _RK(socket_rcvbuf_size), "Broker socket receive buffer size. System default is used if 0.", 0, 100000000, 0 }, { _RK_GLOBAL, "broker.address.ttl", _RK_C_INT, _RK(broker_addr_ttl), "How long to cache the broker address resolving results.", 0, 86400*1000, 300*1000 }, { _RK_GLOBAL, "statistics.interval.ms", _RK_C_INT, _RK(stats_interval_ms), "librdkafka statistics emit interval. The application also needs to " "register a stats callback using `rd_kafka_conf_set_stats_cb()`. " "The granularity is 1000ms. A value of 0 disables statistics.", 0, 86400*1000, 0 }, { _RK_GLOBAL, "error_cb", _RK_C_PTR, _RK(error_cb), "Error callback (set with rd_kafka_conf_set_error_cb())" }, { _RK_GLOBAL, "stats_cb", _RK_C_PTR, _RK(stats_cb), "Statistics callback (set with rd_kafka_conf_set_stats_cb())" }, { _RK_GLOBAL, "opaque", _RK_C_PTR, _RK(opaque), "Application opaque (set with rd_kafka_conf_set_opaque())" }, /* Global consumer properties */ { _RK_GLOBAL|_RK_CONSUMER, "queued.min.messages", _RK_C_INT, _RK(queued_min_msgs), "Minimum number of messages that should to be available " "for consumption by application.", 1, 10000000, 100000 }, { _RK_GLOBAL|_RK_CONSUMER, "fetch.wait.max.ms", _RK_C_INT, _RK(fetch_wait_max_ms), "Maximum time the broker may wait to fill the response " "with fetch.min.bytes.", 0, 300*1000, 100 }, { _RK_GLOBAL|_RK_CONSUMER, "fetch.message.max.bytes", _RK_C_INT, _RK(fetch_msg_max_bytes), "Maximum number of bytes per topic+partition to request when " "fetching messages from the broker.", 1, 1000000000, 1024*1024 }, { _RK_GLOBAL|_RK_CONSUMER, "fetch.min.bytes", _RK_C_INT, _RK(fetch_min_bytes), "Minimum number of bytes the broker responds with. " "If fetch.wait.max.ms expires the accumulated data will " "be sent to the client regardless of this setting.", 1, 100000000, 1 }, { _RK_GLOBAL|_RK_CONSUMER, "fetch.error.backoff.ms", _RK_C_INT, _RK(fetch_error_backoff_ms), "How long to postpone the next fetch request for a " "topic+partition in case of a fetch error.", 1, 300*1000, 500 }, /* Global producer properties */ { _RK_GLOBAL|_RK_PRODUCER, "queue.buffering.max.messages", _RK_C_INT, _RK(queue_buffering_max_msgs), "Maximum number of messages allowed on the producer queue.", 1, 10000000, 100000 }, { _RK_GLOBAL|_RK_PRODUCER, "queue.buffering.max.ms", _RK_C_INT, _RK(buffering_max_ms), "Maximum time, in milliseconds, for buffering data " "on the producer queue.", 1, 900*1000, 1000 }, { _RK_GLOBAL|_RK_PRODUCER, "message.send.max.retries", _RK_C_INT, _RK(max_retries), "How many times to retry sending a failing MessageSet. " "**Note:** retrying may cause reordering.", 0, 100, 2 }, { _RK_GLOBAL|_RK_PRODUCER, "retry.backoff.ms", _RK_C_INT, _RK(retry_backoff_ms), "The backoff time in milliseconds before retrying a message send.", 1, 300*1000, 100 }, { _RK_GLOBAL|_RK_PRODUCER, "compression.codec", _RK_C_S2I, _RK(compression_codec), "Compression codec to use for compressing message sets: " "none, gzip or snappy", .vdef = RD_KAFKA_COMPRESSION_NONE, .s2i = { { RD_KAFKA_COMPRESSION_NONE, "none" }, { RD_KAFKA_COMPRESSION_GZIP, "gzip" }, { RD_KAFKA_COMPRESSION_SNAPPY, "snappy" }, } }, { _RK_GLOBAL|_RK_PRODUCER, "batch.num.messages", _RK_C_INT, _RK(batch_num_messages), "Maximum number of messages batched in one MessageSet.", 1, 1000000, 1000 }, { _RK_GLOBAL|_RK_PRODUCER, "dr_cb", _RK_C_PTR, _RK(dr_cb), "Delivery report callback (set with rd_kafka_conf_set_dr_cb())" }, /* Topic properties */ { _RK_TOPIC|_RK_PRODUCER, "request.required.acks", _RK_C_INT, _RKT(required_acks), "This field indicates how many acknowledgements the leader broker " "must receive from ISR brokers before responding to the request: " "*0*=broker does not send any response, " "*1*=broker will wait until the data is written to local " "log before sending a response, " "*-1*=broker will block until message is committed by all " "in sync replicas (ISRs) before sending response. " "*>1*=for any number > 1 the broker will block waiting for this " "number of acknowledgements to be received (but the broker " "will never wait for more acknowledgements than there are ISRs).", -1, 1000, 1 }, { _RK_TOPIC|_RK_PRODUCER, "request.timeout.ms", _RK_C_INT, _RKT(request_timeout_ms), "The ack timeout of the producer request in milliseconds. " "This value is only enforced by the broker and relies " "on `request.required.acks` being > 0.", 1, 900*1000, 5*1000 }, { _RK_TOPIC|_RK_PRODUCER, "message.timeout.ms", _RK_C_INT, _RKT(message_timeout_ms), "Local message timeout. " "This value is only enforced locally and limits the time a " "produced message waits for successful delivery.", 1, 900*1000, 300*1000 }, { _RK_TOPIC|_RK_PRODUCER, "partitioner", _RK_C_PTR, _RKT(partitioner), "Partitioner callback " "(set with rd_kafka_topic_conf_set_partitioner_cb())" }, { _RK_TOPIC, "opaque", _RK_C_PTR, _RKT(opaque), "Application opaque (set with rd_kafka_topic_conf_set_opaque())" }, { _RK_TOPIC|_RK_CONSUMER, "auto.commit.enable", _RK_C_BOOL, _RKT(auto_commit), "If true, periodically commit offset of the last message handed " "to the application. This commited offset will be used when the " "process restarts to pick up where it left off. " "If false, the application will have to call " "`rd_kafka_offset_store()` to store an offset (optional). " "**NOTE:** There is currently no zookeeper integration, offsets " "will be written to local file according to offset.store.path.", 0, 1, 1 }, { _RK_TOPIC|_RK_CONSUMER, "auto.commit.interval.ms", _RK_C_INT, _RKT(auto_commit_interval_ms), "The frequency in milliseconds that the consumer offsets " "are commited (written) to offset storage.", 10, 86400*1000, 60*1000 }, { _RK_TOPIC|_RK_CONSUMER, "auto.offset.reset", _RK_C_S2I, _RKT(auto_offset_reset), "Action to take when there is no initial offset in offset store " "or the desired offset is out of range: " "'smallest' - automatically reset the offset to the smallest offset, " "'largest' - automatically reset the offset to the largest offset, " "'error' - trigger an error which is retrieved by consuming messages " "and checking 'message->err'.", .vdef = RD_KAFKA_OFFSET_END, .s2i = { { RD_KAFKA_OFFSET_BEGINNING, "smallest" }, { RD_KAFKA_OFFSET_END, "largest" }, { RD_KAFKA_OFFSET_ERROR, "error" }, } }, { _RK_TOPIC|_RK_CONSUMER, "offset.store.path", _RK_C_STR, _RKT(offset_store_path), "Path to local file for storing offsets. If the path is a directory " "a filename will be automatically generated in that directory based " "on the topic and partition.", .sdef = "." }, { _RK_TOPIC|_RK_CONSUMER, "offset.store.sync.interval.ms", _RK_C_INT, _RKT(offset_store_sync_interval_ms), "fsync() interval for the offset file, in milliseconds. " "Use -1 to disable syncing, and 0 for immediate sync after " "each write.", -1, 86400*1000, -1 }, { /* End */ } }; static rd_kafka_conf_res_t rd_kafka_anyconf_set_prop0 (int scope, void *conf, const struct rd_kafka_property *prop, const char *istr, int ival) { #define _RK_PTR(TYPE,BASE,OFFSET) (TYPE)(((char *)(BASE))+(OFFSET)) switch (prop->type) { case _RK_C_STR: { char **str = _RK_PTR(char **, conf, prop->offset); if (*str) free(*str); if (istr) *str = strdup(istr); else *str = NULL; return RD_KAFKA_CONF_OK; } case _RK_C_PTR: *_RK_PTR(const void **, conf, prop->offset) = istr; return RD_KAFKA_CONF_OK; case _RK_C_BOOL: case _RK_C_INT: case _RK_C_S2I: case _RK_C_S2F: { int *val = _RK_PTR(int *, conf, prop->offset); if (prop->type == _RK_C_S2F) { /* Flags: OR it in */ *val |= ival; } else { /* Single assignment */ *val = ival; } return RD_KAFKA_CONF_OK; } default: assert(!*"unknown conf type"); } /* unreachable */ return RD_KAFKA_CONF_INVALID; } static rd_kafka_conf_res_t rd_kafka_anyconf_set_prop (int scope, void *conf, const struct rd_kafka_property *prop, const char *value, char *errstr, size_t errstr_size) { int ival; switch (prop->type) { case _RK_C_STR: rd_kafka_anyconf_set_prop0(scope, conf, prop, value, 0); return RD_KAFKA_CONF_OK; case _RK_C_PTR: snprintf(errstr, errstr_size, "Property \"%s\" must be set through dedicated " ".._set_..() function", prop->name); return RD_KAFKA_CONF_INVALID; case _RK_C_BOOL: if (!value) { snprintf(errstr, errstr_size, "Bool configuration property \"%s\" cannot " "be set to empty value", prop->name); return RD_KAFKA_CONF_INVALID; } if (!strcasecmp(value, "true") || !strcasecmp(value, "t") || !strcmp(value, "1")) ival = 1; else if (!strcasecmp(value, "false") || !strcasecmp(value, "f") || !strcmp(value, "0")) ival = 0; else { snprintf(errstr, errstr_size, "Expected bool value for \"%s\": " "true or false", prop->name); return RD_KAFKA_CONF_INVALID; } rd_kafka_anyconf_set_prop0(scope, conf, prop, NULL, ival); return RD_KAFKA_CONF_OK; case _RK_C_INT: if (!value) { snprintf(errstr, errstr_size, "Integer configuration " "property \"%s\" cannot be set " "to empty value", prop->name); return RD_KAFKA_CONF_INVALID; } ival = atoi(value); if (ival < prop->vmin || ival > prop->vmax) { snprintf(errstr, errstr_size, "Configuration property \"%s\" value " "%i is outside allowed range %i..%i\n", prop->name, ival, prop->vmin, prop->vmax); return RD_KAFKA_CONF_INVALID; } rd_kafka_anyconf_set_prop0(scope, conf, prop, NULL, ival); return RD_KAFKA_CONF_OK; case _RK_C_S2I: case _RK_C_S2F: { int j; const char *next; if (!value) { snprintf(errstr, errstr_size, "Configuration " "property \"%s\" cannot be set " "to empty value", prop->name); return RD_KAFKA_CONF_INVALID; } next = value; while (next && *next) { const char *s, *t; s = next; if (prop->type == _RK_C_S2F && (t = strchr(s, ','))) { /* CSV flag field */ next = t+1; } else { /* Single string */ t = s+strlen(s); next = NULL; } /* Left trim */ while (s < t && isspace((int)*s)) s++; /* Right trim */ while (t > s && isspace((int)*t)) t--; /* Empty string? */ if (s == t) continue; /* Match string to s2i table entry */ for (j = 0 ; j < RD_ARRAYSIZE(prop->s2i); j++) { if (!prop->s2i[j].str || strlen(prop->s2i[j].str) != (int)(t-s) || strncmp(prop->s2i[j].str, s, (int)(t-s))) continue; rd_kafka_anyconf_set_prop0(scope, conf, prop, NULL, prop->s2i[j].val); if (prop->type == _RK_C_S2F) { /* Flags: OR it in: do next */ break; } else { /* Single assignment */ return RD_KAFKA_CONF_OK; } } /* S2F: Good match: continue with next */ if (j < RD_ARRAYSIZE(prop->s2i)) continue; /* No match */ snprintf(errstr, errstr_size, "Invalid value for " "configuration property \"%s\"", prop->name); return RD_KAFKA_CONF_INVALID; } return RD_KAFKA_CONF_OK; } default: assert(!*"unknown conf type"); } /* not reachable */ return RD_KAFKA_CONF_INVALID; } static void rd_kafka_defaultconf_set (int scope, void *conf) { const struct rd_kafka_property *prop; for (prop = rd_kafka_properties ; prop->name ; prop++) { if (!(prop->scope & scope)) continue; if (prop->sdef || prop->vdef) rd_kafka_anyconf_set_prop0(scope, conf, prop, prop->sdef, prop->vdef); } } rd_kafka_conf_t *rd_kafka_conf_new (void) { rd_kafka_conf_t *conf = calloc(1, sizeof(*conf)); rd_kafka_defaultconf_set(_RK_GLOBAL, conf); return conf; } rd_kafka_topic_conf_t *rd_kafka_topic_conf_new (void) { rd_kafka_topic_conf_t *tconf = calloc(1, sizeof(*tconf)); rd_kafka_defaultconf_set(_RK_TOPIC, tconf); return tconf; } static int rd_kafka_anyconf_set (int scope, void *conf, const char *name, const char *value, char *errstr, size_t errstr_size) { char estmp[1]; const struct rd_kafka_property *prop; if (!errstr) { errstr = estmp; errstr_size = 0; } if (value && !*value) value = NULL; for (prop = rd_kafka_properties ; prop->name ; prop++) { if (!(prop->scope & scope)) continue; if (strcmp(prop->name, name)) continue; return rd_kafka_anyconf_set_prop(scope, conf, prop, value, errstr, errstr_size); } snprintf(errstr, errstr_size, "No such configuration property: \"%s\"", name); return RD_KAFKA_CONF_UNKNOWN; } rd_kafka_conf_res_t rd_kafka_conf_set (rd_kafka_conf_t *conf, const char *name, const char *value, char *errstr, size_t errstr_size) { return rd_kafka_anyconf_set(_RK_GLOBAL, conf, name, value, errstr, errstr_size); } rd_kafka_conf_res_t rd_kafka_topic_conf_set (rd_kafka_topic_conf_t *conf, const char *name, const char *value, char *errstr, size_t errstr_size) { if (!strncmp(name, "topic.", strlen("topic."))) name += strlen("topic."); return rd_kafka_anyconf_set(_RK_TOPIC, conf, name, value, errstr, errstr_size); } static void rd_kafka_anyconf_clear (void *conf, const struct rd_kafka_property *prop) { switch (prop->type) { case _RK_C_STR: { char **str = _RK_PTR(char **, conf, prop->offset); if (*str) { free(*str); *str = NULL; } } break; default: break; } } void rd_kafka_anyconf_destroy (int scope, void *conf) { const struct rd_kafka_property *prop; for (prop = rd_kafka_properties; prop->name ; prop++) { if (!(prop->scope & scope)) continue; rd_kafka_anyconf_clear(conf, prop); } } void rd_kafka_conf_destroy (rd_kafka_conf_t *conf) { rd_kafka_anyconf_destroy(_RK_GLOBAL, conf); free(conf); } void rd_kafka_topic_conf_destroy (rd_kafka_topic_conf_t *topic_conf) { rd_kafka_anyconf_destroy(_RK_TOPIC, topic_conf); free(topic_conf); } static void rd_kafka_anyconf_copy (int scope, void *dst, const void *src) { const struct rd_kafka_property *prop; for (prop = rd_kafka_properties ; prop->name ; prop++) { const char *val = NULL; int ival = 0; if (!(prop->scope & scope)) continue; switch (prop->type) { case _RK_C_STR: case _RK_C_PTR: val = *_RK_PTR(const char **, src, prop->offset); break; case _RK_C_BOOL: case _RK_C_INT: case _RK_C_S2I: case _RK_C_S2F: ival = *_RK_PTR(const int *, src, prop->offset); break; default: continue; } rd_kafka_anyconf_set_prop0(scope, dst, prop, val, ival); } } rd_kafka_conf_t *rd_kafka_conf_dup (const rd_kafka_conf_t *conf) { rd_kafka_conf_t *new = rd_kafka_conf_new(); rd_kafka_anyconf_copy(_RK_GLOBAL, new, conf); return new; } rd_kafka_topic_conf_t *rd_kafka_topic_conf_dup (const rd_kafka_topic_conf_t *conf) { rd_kafka_topic_conf_t *new = rd_kafka_topic_conf_new(); rd_kafka_anyconf_copy(_RK_TOPIC, new, conf); return new; } void rd_kafka_conf_set_dr_cb (rd_kafka_conf_t *conf, void (*dr_cb) (rd_kafka_t *rk, void *payload, size_t len, rd_kafka_resp_err_t err, void *opaque, void *msg_opaque)) { conf->dr_cb = dr_cb; } void rd_kafka_conf_set_error_cb (rd_kafka_conf_t *conf, void (*error_cb) (rd_kafka_t *rk, int err, const char *reason, void *opaque)) { conf->error_cb = error_cb; } void rd_kafka_conf_set_stats_cb (rd_kafka_conf_t *conf, int (*stats_cb) (rd_kafka_t *rk, char *json, size_t json_len, void *opaque)) { conf->stats_cb = stats_cb; } void rd_kafka_conf_set_opaque (rd_kafka_conf_t *conf, void *opaque) { conf->opaque = opaque; } void rd_kafka_topic_conf_set_partitioner_cb (rd_kafka_topic_conf_t *topic_conf, int32_t (*partitioner) ( const rd_kafka_topic_t *rkt, const void *keydata, size_t keylen, int32_t partition_cnt, void *rkt_opaque, void *msg_opaque)) { topic_conf->partitioner = partitioner; } void rd_kafka_topic_conf_set_opaque (rd_kafka_topic_conf_t *topic_conf, void *opaque) { topic_conf->opaque = opaque; } static const char **rd_kafka_anyconf_dump (int scope, void *conf, size_t *cntp) { const struct rd_kafka_property *prop; char **arr; int cnt = 0; arr = calloc(sizeof(char *), RD_ARRAYSIZE(rd_kafka_properties)*2); for (prop = rd_kafka_properties; prop->name ; prop++) { char tmp[22]; const char *val = NULL; int j; if (!(prop->scope & scope)) continue; switch (prop->type) { case _RK_C_STR: val = *_RK_PTR(const char **, conf, prop->offset); break; case _RK_C_PTR: val = *_RK_PTR(const void **, conf, prop->offset); if (val) { snprintf(tmp, sizeof(tmp), "%p", (void *)val); val = tmp; } break; case _RK_C_BOOL: val = (*_RK_PTR(int *, conf, prop->offset) ? "true":"false"); break; case _RK_C_INT: snprintf(tmp, sizeof(tmp), "%i", *_RK_PTR(int *, conf, prop->offset)); val = tmp; break; case _RK_C_S2I: for (j = 0 ; j < RD_ARRAYSIZE(prop->s2i); j++) if (prop->s2i[j].val == *_RK_PTR(int *, conf, prop->offset)) val = prop->s2i[j].str; break; case _RK_C_S2F: /* FIXME: ignore for now, just used with "debug" */ default: break; } if (val) { arr[cnt++] = strdup(prop->name); arr[cnt++] = strdup(val); } } *cntp = cnt; return (const char **)arr; } const char **rd_kafka_conf_dump (rd_kafka_conf_t *conf, size_t *cntp) { return rd_kafka_anyconf_dump(_RK_GLOBAL, conf, cntp); } const char **rd_kafka_topic_conf_dump (rd_kafka_topic_conf_t *conf, size_t *cntp) { return rd_kafka_anyconf_dump(_RK_TOPIC, conf, cntp); } void rd_kafka_conf_dump_free (const char **arr, size_t cnt) { char **_arr = (char **)arr; int i; for (i = 0 ; i < cnt ; i++) if (_arr[i]) free(_arr[i]); free(_arr); } void rd_kafka_conf_properties_show (FILE *fp) { const struct rd_kafka_property *prop; int last = 0; int j; const char *dash80 = "----------------------------------------" "----------------------------------------"; for (prop = rd_kafka_properties; prop->name ; prop++) { if (!(prop->scope & last)) { fprintf(fp, "%s## %s configuration properties\n\n", last ? "\n\n":"", prop->scope == _RK_GLOBAL ? "Global": "Topic"); fprintf(fp, "%-40s | %13s | %-25s\n" "%.*s-|-%.*s:|-%.*s\n", "Property", "Default", "Description", 40, dash80, 13, dash80, 25, dash80); last = prop->scope & (_RK_GLOBAL|_RK_TOPIC); } fprintf(fp, "%-40s | ", prop->name); switch (prop->type) { case _RK_C_STR: fprintf(fp, "%13s", prop->sdef ? : ""); break; case _RK_C_BOOL: fprintf(fp, "%13s", prop->vdef ? "true" : "false"); break; case _RK_C_INT: fprintf(fp, "%13i", prop->vdef); break; case _RK_C_S2I: for (j = 0 ; j < RD_ARRAYSIZE(prop->s2i); j++) { if (prop->s2i[j].val == prop->vdef) { fprintf(fp, "%13s", prop->s2i[j].str); break; } } if (j == RD_ARRAYSIZE(prop->s2i)) fprintf(fp, "%13s", " "); break; case _RK_C_S2F: default: /* FIXME when needed */ fprintf(fp, "%-13s", " "); break; } fprintf(fp, " | %s\n", prop->desc); } fprintf(fp, "\n"); } librdkafka-0.8.3/rdkafka_int.h000066400000000000000000000504421227620010100162360ustar00rootroot00000000000000/* * librdkafka - Apache Kafka C library * * Copyright (c) 2012-2013, Magnus Edenhill * All rights reserved. * * Redistribution and use in source and binary forms, with or without * modification, are permitted provided that the following conditions are met: * * 1. Redistributions of source code must retain the above copyright notice, * this list of conditions and the following disclaimer. * 2. Redistributions in binary form must reproduce the above copyright notice, * this list of conditions and the following disclaimer in the documentation * and/or other materials provided with the distribution. * * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE * POSSIBILITY OF SUCH DAMAGE. */ #pragma once #include #include #include #include #include #include #include "rdkafka.h" #include "rd.h" #include "rdaddr.h" #include "rdlog.h" #include "rdkafka_timer.h" #include "rdsysqueue.h" #define RD_POLL_INFINITE -1 #define RD_POLL_NOWAIT 0 /* * Portability */ /* MacOSX does not have strndupa() */ #ifndef strndupa #define strndupa(PTR,LEN) ({ int _L = (LEN); char *_x = alloca(_L+1); \ memcpy(_x, (PTR), _L); *(_x+_L) = 0; _x;}) #endif #ifndef strdupa #define strdupa(PTR) ({ const char *_P = (PTR); int _L = strlen(_P); \ char *_x = alloca(_L+1); memcpy(_x, _P, _L); *(_x+_L) = 0; _x;}) #endif /* Some versions of MacOSX dont have IOV_MAX */ #ifndef IOV_MAX #define IOV_MAX 1024 #endif #include "rdkafka_proto.h" /** * Protocol level sanity */ #define RD_KAFKAP_BROKERS_MAX 1000 #define RD_KAFKAP_TOPICS_MAX 10000 #define RD_KAFKAP_PARTITIONS_MAX 1000 #define RD_KAFKA_OFFSET_ERROR -1001 struct rd_kafka_s; struct rd_kafka_toppar_s; struct rd_kafka_topic_s; struct rd_kafka_msg_s; struct rd_kafka_broker_s; struct rd_kafka_conf_s; struct rd_kafka_topic_conf_s; /** * MessageSet compression codecs */ typedef enum { RD_KAFKA_COMPRESSION_NONE, RD_KAFKA_COMPRESSION_GZIP, RD_KAFKA_COMPRESSION_SNAPPY, } rd_kafka_compression_t; /** * Optional configuration struct passed to rd_kafka_new*(). * * The struct is populated ted through string properties * by calling rd_kafka_conf_set(). * */ struct rd_kafka_conf_s { /* * Generic configuration */ int max_msg_size; int recv_max_msg_size; int metadata_request_timeout_ms; int metadata_refresh_interval_ms; int metadata_refresh_fast_cnt; int metadata_refresh_fast_interval_ms; int debug; int broker_addr_ttl; int socket_timeout_ms; int socket_sndbuf_size; int socket_rcvbuf_size; char *clientid; char *brokerlist; int stats_interval_ms; /* * Consumer configuration */ int queued_min_msgs; int fetch_wait_max_ms; int fetch_msg_max_bytes; int fetch_min_bytes; int fetch_error_backoff_ms; /* Pre-built Fetch request header. */ struct rd_kafkap_FetchRequest FetchRequest; /* * Producer configuration */ int queue_buffering_max_msgs; int buffering_max_ms; int max_retries; int retry_backoff_ms; int batch_num_messages; rd_kafka_compression_t compression_codec; /* Message delivery report callback. * Called once for each produced message, either on * successful and acknowledged delivery to the broker in which * case 'err' is 0, or if the message could not be delivered * in which case 'err' is non-zero (use rd_kafka_err2str() * to obtain a human-readable error reason). * * If the message was produced with neither RD_KAFKA_MSG_F_FREE * or RD_KAFKA_MSG_F_COPY set then 'payload' is the original * pointer provided to rd_kafka_produce(). * rdkafka will not perform any further actions on 'payload' * at this point and the application may free the payload data * at this point. * * 'opaque' is 'conf.opaque', while 'msg_opaque' is * the opaque pointer provided in the rd_kafka_produce() call. */ void (*dr_cb) (rd_kafka_t *rk, void *payload, size_t len, rd_kafka_resp_err_t err, void *opaque, void *msg_opaque); /* Error callback */ void (*error_cb) (rd_kafka_t *rk, int err, const char *reason, void *opaque); /* Stats callback */ int (*stats_cb) (rd_kafka_t *rk, char *json, size_t json_len, void *opaque); /* Opaque passed to callbacks. */ void *opaque; }; struct rd_kafka_topic_conf_s { int16_t required_acks; int32_t request_timeout_ms; int message_timeout_ms; int32_t (*partitioner) (const rd_kafka_topic_t *rkt, const void *keydata, size_t keylen, int32_t partition_cnt, void *rkt_opaque, void *msg_opaque); int auto_commit; int auto_commit_interval_ms; int auto_offset_reset; char *offset_store_path; int offset_store_sync_interval_ms; /* Application provided opaque pointer (this is rkt_opaque) */ void *opaque; }; typedef struct rd_kafka_avg_s { rd_ts_t ra_max; rd_ts_t ra_min; rd_ts_t ra_avg; rd_ts_t ra_sum; int ra_cnt; } rd_kafka_avg_t; /** * Add timestamp 'ts' to averager 'ra'. */ static RD_UNUSED void rd_kafka_avg_add (rd_kafka_avg_t *ra, rd_ts_t ts) { if (ts > ra->ra_max) ra->ra_max = ts; if (ra->ra_min == 0 || ts < ra->ra_min) ra->ra_min = ts; ra->ra_sum += ts; ra->ra_cnt++; } /** * Rolls over statistics in 'src' and stores the average in 'dst'. * 'src' is cleared and ready to be reused. */ static RD_UNUSED void rd_kafka_avg_rollover (rd_kafka_avg_t *dst, rd_kafka_avg_t *src) { *dst = *src; if (dst->ra_cnt) dst->ra_avg = dst->ra_sum / dst->ra_cnt; else dst->ra_avg = 0; memset(src, 0, sizeof(*src)); } typedef struct rd_kafka_msg_s { TAILQ_ENTRY(rd_kafka_msg_s) rkm_link; int rkm_flags; size_t rkm_len; void *rkm_payload; void *rkm_opaque; int32_t rkm_partition; /* partition specified */ rd_kafkap_bytes_t *rkm_key; rd_ts_t rkm_ts_timeout; } rd_kafka_msg_t; typedef struct rd_kafka_msgq_s { TAILQ_HEAD(, rd_kafka_msg_s) rkmq_msgs; int rkmq_msg_cnt; uint64_t rkmq_msg_bytes; } rd_kafka_msgq_t; #define RD_KAFKA_MSGQ_INITIALIZER(rkmq) \ { .rkmq_msgs = TAILQ_HEAD_INITIALIZER((rkmq).rkmq_msgs) } #define RD_KAFKA_MSGQ_FOREACH(elm,head) \ TAILQ_FOREACH(elm, &(head)->rkmq_msgs, rkm_link) typedef struct rd_kafka_buf_s { TAILQ_ENTRY(rd_kafka_buf_s) rkbuf_link; int32_t rkbuf_corrid; rd_ts_t rkbuf_ts_retry; /* Absolute send retry time */ int rkbuf_flags; /* RD_KAFKA_OP_F */ struct msghdr rkbuf_msg; struct iovec *rkbuf_iov; int rkbuf_iovcnt; size_t rkbuf_of; /* recv/send: byte offset */ size_t rkbuf_len; /* send: total length */ size_t rkbuf_size; /* allocated size */ char *rkbuf_buf; /* Main buffer */ char *rkbuf_buf2; /* Aux buffer */ struct rd_kafkap_reqhdr rkbuf_reqhdr; struct rd_kafkap_reshdr rkbuf_reshdr; int32_t rkbuf_expected_size; /* expected size of message */ /* Response callback */ void (*rkbuf_cb) (struct rd_kafka_broker_s *, int err, struct rd_kafka_buf_s *reprkbuf, struct rd_kafka_buf_s *reqrkbuf, void *opaque); int rkbuf_refcnt; void *rkbuf_opaque; int rkbuf_retries; rd_ts_t rkbuf_ts_enq; rd_ts_t rkbuf_ts_sent; rd_ts_t rkbuf_ts_timeout; rd_kafka_msgq_t rkbuf_msgq; } rd_kafka_buf_t; typedef struct rd_kafka_bufq_s { TAILQ_HEAD(, rd_kafka_buf_s) rkbq_bufs; int rkbq_cnt; } rd_kafka_bufq_t; typedef enum { RD_KAFKA_OP_FETCH, /* Kafka thread -> Application */ RD_KAFKA_OP_ERR, /* Kafka thread -> Application */ RD_KAFKA_OP_DR, /* Kafka thread -> Application * Produce message delivery report */ RD_KAFKA_OP_STATS, /* Kafka thread -> Application */ RD_KAFKA_OP_METADATA_REQ, /* any -> Broker thread: request metadata */ } rd_kafka_op_type_t; typedef struct rd_kafka_op_s { TAILQ_ENTRY(rd_kafka_op_s) rko_link; rd_kafka_op_type_t rko_type; int rko_flags; #define RD_KAFKA_OP_F_FREE 0x1 /* Free the payload when done with it. */ #define RD_KAFKA_OP_F_FLASH 0x2 /* Internal: insert at head of queue */ #define RD_KAFKA_OP_F_NO_RESPONSE 0x4 /* rkbuf: Not expecting a response */ rd_kafka_msgq_t rko_msgq; /* For PRODUCE */ rd_kafka_msg_t *rko_rkm; /* For ERR */ #define rko_err rko_rkmessage.err #define rko_payload rko_rkmessage.payload #define rko_len rko_rkmessage.len /* For FETCH */ rd_kafka_message_t rko_rkmessage; rd_kafka_buf_t *rko_rkbuf; /* For METADATA */ #define rko_rkt rko_rkmessage.rkt /* For STATS */ #define rko_json rko_rkmessage.payload #define rko_json_len rko_rkmessage.len } rd_kafka_op_t; typedef struct rd_kafka_q_s { pthread_mutex_t rkq_lock; pthread_cond_t rkq_cond; TAILQ_HEAD(, rd_kafka_op_s) rkq_q; int rkq_qlen; } rd_kafka_q_t; typedef enum { RD_KAFKA_CONFIGURED, RD_KAFKA_LEARNED, } rd_kafka_confsource_t; typedef struct rd_kafka_broker_s { TAILQ_ENTRY(rd_kafka_broker_s) rkb_link; int32_t rkb_nodeid; #define RD_KAFKA_NODEID_UA -1 rd_sockaddr_list_t *rkb_rsal; time_t rkb_t_rsal_last; int rkb_s; /* TCP socket */ struct pollfd rkb_pfd; uint32_t rkb_corrid; rd_kafka_q_t rkb_ops; TAILQ_HEAD(, rd_kafka_toppar_s) rkb_toppars; pthread_rwlock_t rkb_toppar_lock; int rkb_toppar_cnt; rd_ts_t rkb_ts_fetch_backoff; int rkb_fetching; #define rd_kafka_broker_toppars_rdlock(rkb) \ pthread_rwlock_rdlock(&(rkb)->rkb_toppar_lock) #define rd_kafka_broker_toppars_wrlock(rkb) \ pthread_rwlock_wrlock(&(rkb)->rkb_toppar_lock) #define rd_kafka_broker_toppars_unlock(rkb) \ pthread_rwlock_unlock(&(rkb)->rkb_toppar_lock) enum { RD_KAFKA_BROKER_STATE_INIT, RD_KAFKA_BROKER_STATE_DOWN, RD_KAFKA_BROKER_STATE_UP, } rkb_state; rd_kafka_confsource_t rkb_source; struct { uint64_t tx_bytes; uint64_t tx; /* Kafka-messages (not payload msgs) */ uint64_t tx_err; uint64_t tx_retries; uint64_t rx_bytes; uint64_t rx; /* Kafka messages (not payload msgs) */ uint64_t rx_err; uint64_t rx_corrid_err; /* CorrId misses */ } rkb_c; rd_ts_t rkb_ts_metadata_poll; /* Next metadata poll time */ int rkb_metadata_fast_poll_cnt; /* Perform fast * metadata polls. */ pthread_mutex_t rkb_lock; pthread_t rkb_thread; int rkb_refcnt; struct rd_kafka_s *rkb_rk; struct { char msg[512]; int err; /* errno */ } rkb_err; rd_kafka_buf_t *rkb_recv_buf; rd_kafka_bufq_t rkb_outbufs; rd_kafka_bufq_t rkb_waitresps; rd_kafka_bufq_t rkb_retrybufs; rd_kafka_avg_t rkb_rtt_curr; /* Current averaging period */ rd_kafka_avg_t rkb_rtt_last; /* Last averaging period */ char rkb_name[128]; /* Display name */ char rkb_nodename[128]; /* host:port */ } rd_kafka_broker_t; #define rd_kafka_broker_keep(rkb) (void)rd_atomic_add(&(rkb)->rkb_refcnt, 1) #define rd_kafka_broker_lock(rkb) pthread_mutex_lock(&(rkb)->rkb_lock) #define rd_kafka_broker_unlock(rkb) pthread_mutex_unlock(&(rkb)->rkb_lock) /* rd_kafka_topic_t */ struct rd_kafka_topic_s { TAILQ_ENTRY(rd_kafka_topic_s) rkt_link; int rkt_refcnt; pthread_rwlock_t rkt_lock; rd_kafkap_str_t *rkt_topic; struct rd_kafka_toppar_s *rkt_ua; /* unassigned partition */ struct rd_kafka_toppar_s **rkt_p; int32_t rkt_partition_cnt; TAILQ_HEAD(, rd_kafka_toppar_s) rkt_desp; /* Desired partitions * that are not yet seen * in the cluster. */ rd_ts_t rkt_ts_metadata; /* Timestamp of last metadata * update for this topic. */ enum { RD_KAFKA_TOPIC_S_INIT, RD_KAFKA_TOPIC_S_EXISTS, RD_KAFKA_TOPIC_S_UNKNOWN, } rkt_state; int rkt_flags; #define RD_KAFKA_TOPIC_F_LEADER_QUERY 0x1 /* There is an outstanding * leader query for this topic */ struct rd_kafka_s *rkt_rk; rd_kafka_topic_conf_t rkt_conf; }; #define rd_kafka_topic_rdlock(rkt) pthread_rwlock_rdlock(&(rkt)->rkt_lock) #define rd_kafka_topic_wrlock(rkt) pthread_rwlock_wrlock(&(rkt)->rkt_lock) #define rd_kafka_topic_unlock(rkt) pthread_rwlock_unlock(&(rkt)->rkt_lock) /** * Topic + Partition combination */ typedef struct rd_kafka_toppar_s { TAILQ_ENTRY(rd_kafka_toppar_s) rktp_rklink; /* rd_kafka_t link */ TAILQ_ENTRY(rd_kafka_toppar_s) rktp_rkblink; /* rd_kafka_broker_t link*/ TAILQ_ENTRY(rd_kafka_toppar_s) rktp_rktlink; /* rd_kafka_topic_t link */ rd_kafka_topic_t *rktp_rkt; int32_t rktp_partition; rd_kafka_broker_t *rktp_leader; /* Leader broker */ int rktp_refcnt; pthread_mutex_t rktp_lock; rd_kafka_msgq_t rktp_msgq; /* application->rdkafka queue. * protected by rktp_lock */ rd_kafka_msgq_t rktp_xmit_msgq; /* internal broker xmit queue */ rd_ts_t rktp_ts_last_xmit; /* Consumer */ rd_kafka_q_t rktp_fetchq; /* Queue of fetched messages * from broker. */ enum { RD_KAFKA_TOPPAR_FETCH_NONE = 0, RD_KAFKA_TOPPAR_FETCH_OFFSET_QUERY, RD_KAFKA_TOPPAR_FETCH_OFFSET_WAIT, RD_KAFKA_TOPPAR_FETCH_ACTIVE, } rktp_fetch_state; rd_ts_t rktp_ts_offset_req_next; int64_t rktp_query_offset; int64_t rktp_next_offset; /* Next offset to fetch */ int64_t rktp_app_offset; /* Last offset delivered to * application */ int64_t rktp_stored_offset; /* Last stored offset, but * maybe not commited yet. */ int64_t rktp_commited_offset; /* Last commited offset */ rd_ts_t rktp_ts_commited_offset; /* Timestamp of last * commit */ int64_t rktp_eof_offset; /* The last offset we reported * EOF for. */ char *rktp_offset_path; /* Path to offset file */ int rktp_offset_fd; /* Offset file fd */ rd_kafka_timer_t rktp_offset_commit_tmr; /* Offste commit timer */ rd_kafka_timer_t rktp_offset_sync_tmr; /* Offset file sync timer */ int rktp_flags; #define RD_KAFKA_TOPPAR_F_DESIRED 0x1 /* This partition is desired * by a consumer. */ #define RD_KAFKA_TOPPAR_F_UNKNOWN 0x2 /* Topic is (not yet) seen on * a broker. */ struct { uint64_t tx_msgs; uint64_t tx_bytes; } rktp_c; } rd_kafka_toppar_t; #define rd_kafka_toppar_keep(rktp) (void)rd_atomic_add(&(rktp)->rktp_refcnt, 1) #define rd_kafka_toppar_destroy(rktp) do { \ if (rd_atomic_sub(&(rktp)->rktp_refcnt, 1) == 0) \ rd_kafka_toppar_destroy0(rktp); \ } while (0) #define rd_kafka_toppar_lock(rktp) pthread_mutex_lock(&(rktp)->rktp_lock) #define rd_kafka_toppar_unlock(rktp) pthread_mutex_unlock(&(rktp)->rktp_lock) static const char *rd_kafka_toppar_name (const rd_kafka_toppar_t *rktp) RD_UNUSED; static const char *rd_kafka_toppar_name (const rd_kafka_toppar_t *rktp) { static __thread char ret[256]; snprintf(ret, sizeof(ret), "%.*s [%"PRId32"]", RD_KAFKAP_STR_PR(rktp->rktp_rkt->rkt_topic), rktp->rktp_partition); return ret; } /** * Kafka handle. (rd_kafka_t) */ struct rd_kafka_s { rd_kafka_q_t rk_rep; /* kafka -> application reply queue */ TAILQ_HEAD(, rd_kafka_broker_s) rk_brokers; int rk_broker_cnt; /* atomic */ int rk_broker_down_cnt; /* atomic */ TAILQ_HEAD(, rd_kafka_topic_s) rk_topics; int rk_topic_cnt; char rk_name[128]; rd_kafkap_str_t *rk_clientid; rd_kafka_conf_t rk_conf; int rk_flags; int rk_terminate; pthread_mutex_t rk_lock; int rk_refcnt; rd_kafka_type_t rk_type; struct timeval rk_tv_state_change; union { struct { char *topic; int32_t partition; uint64_t offset; uint64_t app_offset; } consumer; struct { int msg_cnt; /* current message count */ } producer; } rk_u; #define rk_consumer rk_u.consumer #define rk_producer rk_u.producer TAILQ_HEAD(, rd_kafka_timer_s) rk_timers; pthread_mutex_t rk_timers_lock; pthread_cond_t rk_timers_cond; void (*rk_log_cb) (const rd_kafka_t *rk, int level, const char *fac, const char *buf); int rk_log_level; pthread_t rk_thread; struct { char msg[512]; int err; /* errno */ } rk_err; }; #define rd_kafka_lock(rk) pthread_mutex_lock(&(rk)->rk_lock) #define rd_kafka_unlock(rk) pthread_mutex_unlock(&(rk)->rk_lock) /** * Debug contexts */ #define RD_KAFKA_DBG_GENERIC 0x1 #define RD_KAFKA_DBG_BROKER 0x2 #define RD_KAFKA_DBG_TOPIC 0x4 #define RD_KAFKA_DBG_METADATA 0x8 #define RD_KAFKA_DBG_PRODUCER 0x10 #define RD_KAFKA_DBG_QUEUE 0x20 #define RD_KAFKA_DBG_MSG 0x40 #define RD_KAFKA_DBG_ALL 0xff void rd_kafka_log_buf (const rd_kafka_t *rk, int level, const char *fac, const char *buf); void rd_kafka_log0 (const rd_kafka_t *rk, const char *extra, int level, const char *fac, const char *fmt, ...) __attribute__((format (printf, 5, 6))); #define rd_kafka_log(rk,level,fac,fmt...) rd_kafka_log0(rk,NULL,level,fac,fmt) #define rd_kafka_dbg(rk,ctx,fac,fmt...) do { \ if (unlikely((rk)->rk_conf.debug & RD_KAFKA_DBG_ ## ctx)) \ rd_kafka_log0(rk,NULL,LOG_DEBUG,fac,fmt); \ } while (0) #define rd_rkb_log(rkb,level,fac,fmt...) \ rd_kafka_log0((rkb)->rkb_rk, (rkb)->rkb_name, level, fac, fmt) #define rd_rkb_dbg(rkb,ctx,fac,fmt...) do { \ if (unlikely((rkb)->rkb_rk->rk_conf.debug & \ RD_KAFKA_DBG_ ## ctx)) \ rd_kafka_log0((rkb)->rkb_rk, (rkb)->rkb_name, \ LOG_DEBUG, fac, fmt); \ } while (0) void rd_kafka_q_init (rd_kafka_q_t *rkq); void rd_kafka_q_destroy (rd_kafka_q_t *rkq); /** * Enqueue the 'rko' op at the tail of the queue 'rkq'. * * Locality: any thread. */ static inline RD_UNUSED void rd_kafka_q_enq (rd_kafka_q_t *rkq, rd_kafka_op_t *rko) { pthread_mutex_lock(&rkq->rkq_lock); TAILQ_INSERT_TAIL(&rkq->rkq_q, rko, rko_link); (void)rd_atomic_add(&rkq->rkq_qlen, 1); pthread_cond_signal(&rkq->rkq_cond); pthread_mutex_unlock(&rkq->rkq_lock); } /** * Concat all elements of 'srcq' onto tail of 'rkq'. * 'dstq' will be be locked, but 'srcq' will not. * * Locality: any thread. */ static inline RD_UNUSED void rd_kafka_q_concat (rd_kafka_q_t *rkq, rd_kafka_q_t *srcq) { pthread_mutex_lock(&rkq->rkq_lock); TAILQ_CONCAT(&rkq->rkq_q, &srcq->rkq_q, rko_link); (void)rd_atomic_add(&rkq->rkq_qlen, srcq->rkq_qlen); pthread_cond_signal(&rkq->rkq_cond); pthread_mutex_unlock(&rkq->rkq_lock); } #define rd_kafka_q_len(rkq) ((rkq)->rkq_qlen) rd_kafka_op_t *rd_kafka_q_pop (rd_kafka_q_t *rkq, int timeout_ms); void rd_kafka_q_purge (rd_kafka_q_t *rkq); size_t rd_kafka_q_move_cnt (rd_kafka_q_t *dstq, rd_kafka_q_t *srcq, size_t cnt); void rd_kafka_op_destroy (rd_kafka_op_t *rko); rd_kafka_op_t *rd_kafka_op_new (rd_kafka_op_type_t type); void rd_kafka_op_reply2 (rd_kafka_t *rk, rd_kafka_op_t *rko); void rd_kafka_op_reply0 (rd_kafka_t *rk, rd_kafka_op_t *rko, rd_kafka_op_type_t type, rd_kafka_resp_err_t err, void *payload, int len); void rd_kafka_op_reply (rd_kafka_t *rk, rd_kafka_op_type_t type, rd_kafka_resp_err_t err, void *payload, int len); void rd_kafka_op_err (rd_kafka_t *rk, rd_kafka_resp_err_t err, const char *fmt, ...); #define rd_kafka_keep(rk) (void)rd_atomic_add(&(rk)->rk_refcnt, 1) void rd_kafka_destroy0 (rd_kafka_t *rk); typedef enum { _RK_GLOBAL = 0x1, _RK_PRODUCER = 0x2, _RK_CONSUMER = 0x4, _RK_TOPIC = 0x8 } rd_kafka_conf_scope_t; void rd_kafka_anyconf_destroy (int scope, void *conf); extern int rd_kafka_thread_cnt_curr; #define RD_KAFKA_SEND_END -1 int pthread_cond_timedwait_ms (pthread_cond_t *cond, pthread_mutex_t *mutex, int timeout_ms); librdkafka-0.8.3/rdkafka_msg.c000066400000000000000000000172041227620010100162240ustar00rootroot00000000000000/* * librdkafka - Apache Kafka C library * * Copyright (c) 2012,2013 Magnus Edenhill * All rights reserved. * * Redistribution and use in source and binary forms, with or without * modification, are permitted provided that the following conditions are met: * * 1. Redistributions of source code must retain the above copyright notice, * this list of conditions and the following disclaimer. * 2. Redistributions in binary form must reproduce the above copyright notice, * this list of conditions and the following disclaimer in the documentation * and/or other materials provided with the distribution. * * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE * POSSIBILITY OF SUCH DAMAGE. */ #include "rd.h" #include "rdkafka_int.h" #include "rdkafka_msg.h" #include "rdkafka_topic.h" #include "rdrand.h" #include "rdtime.h" #include "rdsysqueue.h" void rd_kafka_msg_destroy (rd_kafka_t *rk, rd_kafka_msg_t *rkm) { assert(rk->rk_producer.msg_cnt > 0); (void)rd_atomic_sub(&rk->rk_producer.msg_cnt, 1); if (rkm->rkm_flags & RD_KAFKA_MSG_F_FREE) free(rkm->rkm_payload); if (rkm->rkm_key) rd_kafkap_bytes_destroy(rkm->rkm_key); free(rkm); } /** * Produce: creates a new message, runs the partitioner and enqueues * into on the selected partition. * * Returns 0 on success or -1 on error. */ int rd_kafka_msg_new (rd_kafka_topic_t *rkt, int32_t force_partition, int msgflags, char *payload, size_t len, const void *key, size_t keylen, void *msg_opaque) { rd_kafka_msg_t *rkm; size_t mlen = sizeof(*rkm); rd_kafka_resp_err_t err; if (unlikely(len + keylen > rkt->rkt_rk->rk_conf.max_msg_size)) { errno = EMSGSIZE; return -1; } if (unlikely(rd_atomic_add(&rkt->rkt_rk->rk_producer.msg_cnt, 1) > rkt->rkt_rk->rk_conf.queue_buffering_max_msgs)) { (void)rd_atomic_sub(&rkt->rkt_rk->rk_producer.msg_cnt, 1); errno = ENOBUFS; return -1; } /* If we are to make a copy of the payload, allocate space for it too */ if (msgflags & RD_KAFKA_MSG_F_COPY) { msgflags &= ~RD_KAFKA_MSG_F_FREE; mlen += len; } /* Note: using malloc here, not calloc, so make sure all fields * are properly set up. */ rkm = malloc(mlen); rkm->rkm_len = len; rkm->rkm_flags = msgflags; rkm->rkm_opaque = msg_opaque; rkm->rkm_key = rd_kafkap_bytes_new(key, keylen); rkm->rkm_partition = force_partition; rkm->rkm_ts_timeout = rd_clock() + rkt->rkt_conf.message_timeout_ms * 1000; if (msgflags & RD_KAFKA_MSG_F_COPY) { /* Copy payload to space following the ..msg_t */ rkm->rkm_payload = (void *)(rkm+1); memcpy(rkm->rkm_payload, payload, len); } else { /* Just point to the provided payload. */ rkm->rkm_payload = payload; } err = rd_kafka_msg_partitioner(rkt, rkm, 1); if (likely(!err)) return 0; /* Handle partitioner failures: it only fails when the application * attempts to force a destination partition that does not exist * in the cluster. */ rd_kafka_msg_destroy(rkt->rkt_rk, rkm); /* Translate error codes to errnos. */ if (err == RD_KAFKA_RESP_ERR__UNKNOWN_PARTITION) errno = ESRCH; else if (err == RD_KAFKA_RESP_ERR__UNKNOWN_TOPIC) errno = ENOENT; else errno = EINVAL; /* NOTREACHED */ return -1; } /** * Scan 'rkmq' for messages that have timed out and remove them from * 'rkmq' and add to 'timedout'. */ int rd_kafka_msgq_age_scan (rd_kafka_msgq_t *rkmq, rd_kafka_msgq_t *timedout, rd_ts_t now) { rd_kafka_msg_t *rkm, *tmp; int cnt = timedout->rkmq_msg_cnt; /* Assume messages are added in time sequencial order */ TAILQ_FOREACH_SAFE(rkm, &rkmq->rkmq_msgs, rkm_link, tmp) { if (likely(rkm->rkm_ts_timeout > now)) break; rd_kafka_msgq_deq(rkmq, rkm, 1); rd_kafka_msgq_enq(timedout, rkm); } return timedout->rkmq_msg_cnt - cnt; } int32_t rd_kafka_msg_partitioner_random (const rd_kafka_topic_t *rkt, const void *key, size_t keylen, int32_t partition_cnt, void *rkt_opaque, void *msg_opaque) { int32_t p = rd_jitter(0, partition_cnt-1); if (unlikely(!rd_kafka_topic_partition_available(rkt, p))) return rd_jitter(0, partition_cnt-1); else return p; } /** * Assigns a message to a topic partition using a partitioner. * Returns RD_KAFKA_RESP_ERR__UNKNOWN_PARTITION or .._UNKNOWN_TOPIC if * partitioning failed, or 0 on success. */ int rd_kafka_msg_partitioner (rd_kafka_topic_t *rkt, rd_kafka_msg_t *rkm, int do_lock) { int32_t partition; rd_kafka_toppar_t *rktp_new; rd_kafka_resp_err_t err; if (do_lock) rd_kafka_topic_rdlock(rkt); /* Fast path for failing messages with forced partition * when the partition is not available. * Only fail the message if its forced partition does not * exist in the Kafka cluster, given that the topic's metadata * can be trusted (is not older than 3 times the metadata * refresh interval). */ if (unlikely((rkt->rkt_state == RD_KAFKA_TOPIC_S_UNKNOWN || (rkm->rkm_partition != RD_KAFKA_PARTITION_UA && (rkm->rkm_partition >= rkt->rkt_partition_cnt ))) && rd_clock() < rkt->rkt_ts_metadata + (rkt->rkt_rk->rk_conf.metadata_refresh_interval_ms * 3 * 1000))) { if (rkt->rkt_partition_cnt == 0) err = RD_KAFKA_RESP_ERR__UNKNOWN_TOPIC; else err = RD_KAFKA_RESP_ERR__UNKNOWN_PARTITION; if (do_lock) rd_kafka_topic_unlock(rkt); return err; } if (unlikely(rkt->rkt_partition_cnt == 0)) { partition = RD_KAFKA_PARTITION_UA; } else if (rkm->rkm_partition == RD_KAFKA_PARTITION_UA) partition = rkt->rkt_conf.partitioner(rkt, rkm->rkm_key->data, RD_KAFKAP_BYTES_LEN(rkm-> rkm_key), rkt->rkt_partition_cnt, rkt->rkt_conf.opaque, rkm->rkm_opaque); else /* Partition specified by the application */ partition = rkm->rkm_partition; if (partition >= rkt->rkt_partition_cnt) { /* Partition is unknown (locally) */ /* Temporary error, assign to UA partition for now */ rd_kafka_dbg(rkt->rkt_rk, TOPIC, "PART", "%.*s partition [%"PRId32"] not " "currently available", RD_KAFKAP_STR_PR(rkt->rkt_topic), partition); partition = RD_KAFKA_PARTITION_UA; /* FALLTHRU */ } if (0) rd_kafka_dbg(rkt->rkt_rk, MSG, "PART", "Message %p assigned to %.*s " "partition [%"PRId32"]/%"PRId32" " "(fixed [%"PRId32"])", rkm, RD_KAFKAP_STR_PR(rkt->rkt_topic), partition, rkt->rkt_partition_cnt, rkm->rkm_partition); /* Get new partition */ rktp_new = rd_kafka_toppar_get(rkt, partition, 0); if (likely(!rktp_new)) { /* Unknown topic or partition */ if (rkt->rkt_state == RD_KAFKA_TOPIC_S_UNKNOWN) err = RD_KAFKA_RESP_ERR__UNKNOWN_TOPIC; else err = RD_KAFKA_RESP_ERR__UNKNOWN_PARTITION; if (do_lock) rd_kafka_topic_unlock(rkt); return err; } /* Partition is available: enqueue msg on partition's queue */ rd_kafka_toppar_enq_msg(rktp_new, rkm); if (do_lock) rd_kafka_topic_unlock(rkt); rd_kafka_toppar_destroy(rktp_new); /* from _get() */ return 0; } librdkafka-0.8.3/rdkafka_msg.h000066400000000000000000000111771227620010100162340ustar00rootroot00000000000000/* * librdkafka - Apache Kafka C library * * Copyright (c) 2012,2013 Magnus Edenhill * All rights reserved. * * Redistribution and use in source and binary forms, with or without * modification, are permitted provided that the following conditions are met: * * 1. Redistributions of source code must retain the above copyright notice, * this list of conditions and the following disclaimer. * 2. Redistributions in binary form must reproduce the above copyright notice, * this list of conditions and the following disclaimer in the documentation * and/or other materials provided with the distribution. * * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE * POSSIBILITY OF SUCH DAMAGE. */ #pragma once #include "rdsysqueue.h" void rd_kafka_msg_destroy (rd_kafka_t *rk, rd_kafka_msg_t *rkm); int rd_kafka_msg_new (rd_kafka_topic_t *rkt, int32_t force_partition, int msgflags, char *payload, size_t len, const void *keydata, size_t keylen, void *msg_opaque); static inline RD_UNUSED void rd_kafka_msgq_init (rd_kafka_msgq_t *rkmq) { TAILQ_INIT(&rkmq->rkmq_msgs); rkmq->rkmq_msg_cnt = 0; rkmq->rkmq_msg_bytes = 0; } /** * Concat all elements of 'src' onto tail of 'dst'. * 'src' will be cleared. */ static inline RD_UNUSED void rd_kafka_msgq_concat (rd_kafka_msgq_t *dst, rd_kafka_msgq_t *src) { TAILQ_CONCAT(&dst->rkmq_msgs, &src->rkmq_msgs, rkm_link); (void)rd_atomic_add(&dst->rkmq_msg_cnt, src->rkmq_msg_cnt); (void)rd_atomic_add(&dst->rkmq_msg_bytes, src->rkmq_msg_bytes); rd_kafka_msgq_init(src); } /** * Move queue 'src' to 'dst' (overwrites dst) * Source will be cleared. */ static inline RD_UNUSED void rd_kafka_msgq_move (rd_kafka_msgq_t *dst, rd_kafka_msgq_t *src) { TAILQ_MOVE(&dst->rkmq_msgs, &src->rkmq_msgs, rkm_link); dst->rkmq_msg_cnt = src->rkmq_msg_cnt; dst->rkmq_msg_bytes = src->rkmq_msg_bytes; rd_kafka_msgq_init(src); } /** * Free all msgs in msgq and reinitialize the msgq. */ static inline RD_UNUSED void rd_kafka_msgq_purge (rd_kafka_t *rk, rd_kafka_msgq_t *rkmq) { rd_kafka_msg_t *rkm, *next; next = TAILQ_FIRST(&rkmq->rkmq_msgs); while (next) { rkm = next; next = TAILQ_NEXT(next, rkm_link); rd_kafka_msg_destroy(rk, rkm); } rd_kafka_msgq_init(rkmq); } /** * Remove message from message queue */ static inline RD_UNUSED rd_kafka_msg_t *rd_kafka_msgq_deq (rd_kafka_msgq_t *rkmq, rd_kafka_msg_t *rkm, int do_count) { if (likely(do_count)) { assert(rkmq->rkmq_msg_cnt > 0); assert(rkmq->rkmq_msg_bytes - rkm->rkm_len >= 0); (void)rd_atomic_sub(&rkmq->rkmq_msg_cnt, 1); (void)rd_atomic_sub(&rkmq->rkmq_msg_bytes, rkm->rkm_len); } TAILQ_REMOVE(&rkmq->rkmq_msgs, rkm, rkm_link); return rkm; } static inline RD_UNUSED rd_kafka_msg_t *rd_kafka_msgq_pop (rd_kafka_msgq_t *rkmq) { rd_kafka_msg_t *rkm; if (((rkm = TAILQ_FIRST(&rkmq->rkmq_msgs)))) rd_kafka_msgq_deq(rkmq, rkm, 1); return rkm; } /** * Insert message at head of message queue. */ static inline RD_UNUSED void rd_kafka_msgq_insert (rd_kafka_msgq_t *rkmq, rd_kafka_msg_t *rkm) { TAILQ_INSERT_HEAD(&rkmq->rkmq_msgs, rkm, rkm_link); (void)rd_atomic_add(&rkmq->rkmq_msg_cnt, 1); (void)rd_atomic_add(&rkmq->rkmq_msg_bytes, rkm->rkm_len); } /** * Append message to tail of message queue. */ static inline RD_UNUSED void rd_kafka_msgq_enq (rd_kafka_msgq_t *rkmq, rd_kafka_msg_t *rkm) { TAILQ_INSERT_TAIL(&rkmq->rkmq_msgs, rkm, rkm_link); (void)rd_atomic_add(&rkmq->rkmq_msg_cnt, 1); (void)rd_atomic_add(&rkmq->rkmq_msg_bytes, rkm->rkm_len); } /** * Scans a message queue for timed out messages and removes them from * 'rkmq' and adds them to 'timedout', returning the number of timed out * messages. * 'timedout' must be initialized. */ int rd_kafka_msgq_age_scan (rd_kafka_msgq_t *rkmq, rd_kafka_msgq_t *timedout, rd_ts_t now); int rd_kafka_msg_partitioner (rd_kafka_topic_t *rkt, rd_kafka_msg_t *rkm, int do_lock); librdkafka-0.8.3/rdkafka_offset.c000066400000000000000000000310051227620010100167170ustar00rootroot00000000000000/* * librdkafka - Apache Kafka C library * * Copyright (c) 2012,2013 Magnus Edenhill * All rights reserved. * * Redistribution and use in source and binary forms, with or without * modification, are permitted provided that the following conditions are met: * * 1. Redistributions of source code must retain the above copyright notice, * this list of conditions and the following disclaimer. * 2. Redistributions in binary form must reproduce the above copyright notice, * this list of conditions and the following disclaimer in the documentation * and/or other materials provided with the distribution. * * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE * POSSIBILITY OF SUCH DAMAGE. */ /** * This file implements the consumer offset storage. * It currently only supports local file storage, not zookeeper. * * Regardless of commit method (file, zookeeper, ..) this is how it works: * - When rdkafka, or the application, depending on if auto.offset.commit * is enabled or not, calls rd_kafka_offset_store() with an offset to store, * all it does is set rktp->rktp_stored_offset to this value. * This can happen from any thread and is locked by the rktp lock. * - The actual commit/write of the offset to its backing store (filesystem) * is performed by the main rdkafka thread and scheduled at the configured * auto.commit.interval.ms interval. * - The write is performed in the main rdkafka thread in a blocking manner * and once the write has succeeded rktp->rktp_commited_offset is updated * to the new value. * - If offset.store.sync.interval.ms is configured the main rdkafka thread * will also make sure to fsync() each offset file accordingly. */ #include #include #include #include #include "rdkafka.h" #include "rdkafka_int.h" #include "rdkafka_offset.h" #include "rdkafka_topic.h" /** * NOTE: toppar_lock(rktp) must be held */ static void rd_kafka_offset_file_close (rd_kafka_toppar_t *rktp) { if (rktp->rktp_offset_fd == -1) return; close(rktp->rktp_offset_fd); rktp->rktp_offset_fd = -1; } /** * NOTE: toppar_lock(rktp) must be held */ static int rd_kafka_offset_file_open (rd_kafka_toppar_t *rktp) { int fd; if ((fd = open(rktp->rktp_offset_path, O_CREAT|O_RDWR, 0644)) == -1) { rd_kafka_op_err(rktp->rktp_rkt->rkt_rk, RD_KAFKA_RESP_ERR__FS, "%s [%"PRId32"]: " "Failed to open offset file %s: %s", rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition, rktp->rktp_offset_path, strerror(errno)); return -1; } rktp->rktp_offset_fd = fd; return 0; } /** * NOTE: toppar_lock(rktp) must be held */ static int64_t rd_kafka_offset_file_read (rd_kafka_toppar_t *rktp) { char buf[22]; char *end; int64_t offset; int r; if (lseek(rktp->rktp_offset_fd, SEEK_SET, 0) == -1) { rd_kafka_op_err(rktp->rktp_rkt->rkt_rk, RD_KAFKA_RESP_ERR__FS, "%s [%"PRId32"]: " "Seek (for read) failed on offset file %s: %s", rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition, rktp->rktp_offset_path, strerror(errno)); rd_kafka_offset_file_close(rktp); return -1; } if ((r = read(rktp->rktp_offset_fd, buf, sizeof(buf)-1)) == -1) { rd_kafka_op_err(rktp->rktp_rkt->rkt_rk, RD_KAFKA_RESP_ERR__FS, "%s [%"PRId32"]: " "Failed to read offset file %s: %s", rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition, rktp->rktp_offset_path, strerror(errno)); rd_kafka_offset_file_close(rktp); return -1; } if (r == 0) { rd_kafka_dbg(rktp->rktp_rkt->rkt_rk, TOPIC, "OFFSET", "%s [%"PRId32"]: offset file (%s) is empty", rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition, rktp->rktp_offset_path); return -1; } buf[r] = '\0'; offset = strtoull(buf, &end, 10); if (buf == end) { rd_kafka_op_err(rktp->rktp_rkt->rkt_rk, RD_KAFKA_RESP_ERR__FS, "%s [%"PRId32"]: " "Unable to parse offset in %s", rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition, rktp->rktp_offset_path); return -1; } rd_kafka_dbg(rktp->rktp_rkt->rkt_rk, TOPIC, "OFFSET", "%s [%"PRId32"]: Read offset %"PRId64" from offset " "file (%s)", rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition, offset, rktp->rktp_offset_path); return offset; } /** * NOTE: rktp lock is not required. * Locality: rdkafka main thread */ static int rd_kafka_offset_file_commit (rd_kafka_toppar_t *rktp, int64_t offset) { rd_kafka_topic_t *rkt = rktp->rktp_rkt; int attempt; for (attempt = 0 ; attempt < 2 ; attempt++) { char buf[22]; int len; if (rktp->rktp_offset_fd == -1) if (rd_kafka_offset_file_open(rktp) == -1) continue; if (lseek(rktp->rktp_offset_fd, 0, SEEK_SET) == -1) { rd_kafka_op_err(rktp->rktp_rkt->rkt_rk, RD_KAFKA_RESP_ERR__FS, "%s [%"PRId32"]: " "Seek failed on offset file %s: %s", rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition, rktp->rktp_offset_path, strerror(errno)); rd_kafka_offset_file_close(rktp); continue; } len = snprintf(buf, sizeof(buf), "%"PRId64"\n", offset); if (write(rktp->rktp_offset_fd, buf, len) == -1) { rd_kafka_op_err(rktp->rktp_rkt->rkt_rk, RD_KAFKA_RESP_ERR__FS, "%s [%"PRId32"]: " "Failed to write offset %"PRId64" to " "offset file %s (fd %i): %s", rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition, offset, rktp->rktp_offset_path, rktp->rktp_offset_fd, strerror(errno)); rd_kafka_offset_file_close(rktp); continue; } if (ftruncate(rktp->rktp_offset_fd, len) == -1) ; /* Ignore truncate failures */ rd_kafka_dbg(rktp->rktp_rkt->rkt_rk, TOPIC, "OFFSET", "%s [%"PRId32"]: wrote offset %"PRId64" to " "file %s", rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition, offset, rktp->rktp_offset_path); rktp->rktp_commited_offset = offset; /* If sync interval is set to immediate we sync right away. */ if (rkt->rkt_conf.offset_store_sync_interval_ms == 0) fsync(rktp->rktp_offset_fd); return 0; } return -1; } /** * Store offset. * Typically called from application code. * * NOTE: No lucks must be held. */ rd_kafka_resp_err_t rd_kafka_offset_store (rd_kafka_topic_t *rkt, int32_t partition, int64_t offset) { rd_kafka_toppar_t *rktp; /* Find toppar */ rd_kafka_topic_rdlock(rkt); if (!(rktp = rd_kafka_toppar_get(rkt, partition, 0/*!ua_on_miss*/))) { rd_kafka_topic_unlock(rkt); return RD_KAFKA_RESP_ERR__UNKNOWN_PARTITION; } rd_kafka_topic_unlock(rkt); rd_kafka_offset_store0(rktp, offset, 1/*lock*/); rd_kafka_toppar_destroy(rktp); return RD_KAFKA_RESP_ERR_NO_ERROR; } /** * Offset file commit timer callback. */ static void rd_kafka_offset_file_commit_tmr_cb (rd_kafka_t *rk, void *arg) { rd_kafka_toppar_t *rktp = arg; rd_kafka_toppar_lock(rktp); rd_kafka_dbg(rk, TOPIC, "OFFSET", "%s [%"PRId32"]: periodic commit: " "stored offset %"PRId64" > commited offset %"PRId64" ?", rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition, rktp->rktp_stored_offset, rktp->rktp_commited_offset); if (rktp->rktp_stored_offset > rktp->rktp_commited_offset) rd_kafka_offset_file_commit(rktp, rktp->rktp_stored_offset); rd_kafka_toppar_unlock(rktp); } /** * Offset file sync timer callback */ static void rd_kafka_offset_file_sync_tmr_cb (rd_kafka_t *rk, void *arg) { rd_kafka_toppar_t *rktp = arg; rd_kafka_toppar_lock(rktp); if (rktp->rktp_offset_fd != -1) { rd_kafka_dbg(rk, TOPIC, "SYNC", "%s [%"PRId32"]: offset file sync", rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition); fsync(rktp->rktp_offset_fd); } rd_kafka_toppar_unlock(rktp); } /** * Decommissions the use of an offset file for a toppar. * The file content will not be touched and the file will not be removed. * * NOTE: toppar_lock(rktp) must be held. */ static void rd_kafka_offset_file_term (rd_kafka_toppar_t *rktp) { if (rktp->rktp_rkt->rkt_conf.offset_store_sync_interval_ms > 0) rd_kafka_timer_stop(rktp->rktp_rkt->rkt_rk, &rktp->rktp_offset_sync_tmr, 1/*lock*/); rd_kafka_timer_stop(rktp->rktp_rkt->rkt_rk, &rktp->rktp_offset_commit_tmr, 1/*lock*/); if (rktp->rktp_stored_offset > rktp->rktp_commited_offset) rd_kafka_offset_file_commit(rktp, rktp->rktp_stored_offset); rd_kafka_offset_file_close(rktp); free(rktp->rktp_offset_path); rktp->rktp_offset_path = NULL; } /** * Take action when the offset for a toppar becomes unusable. * NOTE: toppar_lock(rktp) must be held */ void rd_kafka_offset_reset (rd_kafka_toppar_t *rktp, int64_t err_offset, rd_kafka_resp_err_t err, const char *reason) { int64_t offset = RD_KAFKA_OFFSET_ERROR; rd_kafka_op_t *rko; switch (rktp->rktp_rkt->rkt_conf.auto_offset_reset) { case RD_KAFKA_OFFSET_END: case RD_KAFKA_OFFSET_BEGINNING: offset = rktp->rktp_rkt->rkt_conf.auto_offset_reset; rktp->rktp_query_offset = offset; rktp->rktp_fetch_state = RD_KAFKA_TOPPAR_FETCH_OFFSET_QUERY; break; case RD_KAFKA_OFFSET_ERROR: rko = rd_kafka_op_new(RD_KAFKA_OP_ERR); rko->rko_err = err; rko->rko_rkmessage.offset = err_offset; rko->rko_rkmessage.rkt = rktp->rktp_rkt; rko->rko_rkmessage.partition = rktp->rktp_partition; rko->rko_payload = strdup(reason); rko->rko_len = strlen(rko->rko_payload); rko->rko_flags |= RD_KAFKA_OP_F_FREE; rd_kafka_q_enq(&rktp->rktp_fetchq, rko); rktp->rktp_fetch_state = RD_KAFKA_TOPPAR_FETCH_NONE; break; } rd_kafka_dbg(rktp->rktp_rkt->rkt_rk, TOPIC, "OFFSET", "%s [%"PRId32"]: offset reset (at offset %"PRId64") " "to %"PRId64": %s: %s", rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition, err_offset, offset, reason, rd_kafka_err2str(err)); } /** * Prepare a toppar for using an offset file. * * NOTE: toppar_lock(rktp) must be held. */ static void rd_kafka_offset_file_init (rd_kafka_toppar_t *rktp) { struct stat st; char spath[4096]; const char *path = rktp->rktp_rkt->rkt_conf.offset_store_path; int64_t offset = -1; if (stat(path, &st) == 0 && S_ISDIR(st.st_mode)) { snprintf(spath, sizeof(spath), "%s%s%s-%"PRId32".offset", path, path[strlen(path)-1] == '/' ? "" : "/", rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition); path = spath; } rd_kafka_dbg(rktp->rktp_rkt->rkt_rk, TOPIC, "OFFSET", "%s [%"PRId32"] using offset file %s", rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition, path); rktp->rktp_offset_path = strdup(path); rd_kafka_timer_start(rktp->rktp_rkt->rkt_rk, &rktp->rktp_offset_commit_tmr, rktp->rktp_rkt->rkt_conf.auto_commit_interval_ms * 1000, rd_kafka_offset_file_commit_tmr_cb, rktp); if (rktp->rktp_rkt->rkt_conf.offset_store_sync_interval_ms > 0) rd_kafka_timer_start(rktp->rktp_rkt->rkt_rk, &rktp->rktp_offset_sync_tmr, rktp->rktp_rkt->rkt_conf. offset_store_sync_interval_ms * 1000, rd_kafka_offset_file_sync_tmr_cb, rktp); if (rd_kafka_offset_file_open(rktp) != -1) { /* Read offset from offset file. */ offset = rd_kafka_offset_file_read(rktp); } if (offset != -1) { /* Start fetching from offset */ rktp->rktp_commited_offset = offset; rktp->rktp_next_offset = offset; rktp->rktp_fetch_state = RD_KAFKA_TOPPAR_FETCH_ACTIVE; } else { /* Offset was not usable: perform offset reset logic */ rktp->rktp_commited_offset = 0; rd_kafka_offset_reset(rktp, RD_KAFKA_OFFSET_ERROR, RD_KAFKA_RESP_ERR__FS, "non-readable offset file"); } } /** * Terminates toppar's offset store. * NOTE: toppar_lock(rktp) must be held. */ void rd_kafka_offset_store_term (rd_kafka_toppar_t *rktp) { rd_kafka_offset_file_term(rktp); } /** * Initialize toppar's offset store. * NOTE: toppar_lock(rktp) must be held. */ void rd_kafka_offset_store_init (rd_kafka_toppar_t *rktp) { rd_kafka_offset_file_init(rktp); } librdkafka-0.8.3/rdkafka_offset.h000066400000000000000000000043631227620010100167330ustar00rootroot00000000000000/* * librdkafka - Apache Kafka C library * * Copyright (c) 2012,2013 Magnus Edenhill * All rights reserved. * * Redistribution and use in source and binary forms, with or without * modification, are permitted provided that the following conditions are met: * * 1. Redistributions of source code must retain the above copyright notice, * this list of conditions and the following disclaimer. * 2. Redistributions in binary form must reproduce the above copyright notice, * this list of conditions and the following disclaimer in the documentation * and/or other materials provided with the distribution. * * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE * POSSIBILITY OF SUCH DAMAGE. */ #pragma once /** * Stores the offset for the toppar 'rktp'. * The actual commit of the offset to backing store is performed * from the main rdkafka thread. * See head of rdkafka_offset.c for more information. * * NOTE: toppar_lock(rktp) must be held. */ static inline RD_UNUSED void rd_kafka_offset_store0 (rd_kafka_toppar_t *rktp, int64_t offset, int lock) { if (lock) rd_kafka_toppar_lock(rktp); rktp->rktp_stored_offset = offset; if (lock) rd_kafka_toppar_unlock(rktp); } rd_kafka_resp_err_t rd_kafka_offset_store (rd_kafka_topic_t *rkt, int32_t partition, int64_t offset); void rd_kafka_offset_store_term (rd_kafka_toppar_t *rktp); void rd_kafka_offset_store_init (rd_kafka_toppar_t *rktp); void rd_kafka_offset_reset (rd_kafka_toppar_t *rktp, int64_t err_offset, rd_kafka_resp_err_t err, const char *reason); librdkafka-0.8.3/rdkafka_proto.h000066400000000000000000000155431227620010100166120ustar00rootroot00000000000000/* * librdkafka - Apache Kafka C library * * Copyright (c) 2012,2013 Magnus Edenhill * All rights reserved. * * Redistribution and use in source and binary forms, with or without * modification, are permitted provided that the following conditions are met: * * 1. Redistributions of source code must retain the above copyright notice, * this list of conditions and the following disclaimer. * 2. Redistributions in binary form must reproduce the above copyright notice, * this list of conditions and the following disclaimer in the documentation * and/or other materials provided with the distribution. * * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE * POSSIBILITY OF SUCH DAMAGE. */ #pragma once /* * Kafka protocol definitions. */ #define RD_KAFKA_PORT 9092 #define RD_KAFKA_PORT_STR "9092" /** * Request header */ struct rd_kafkap_reqhdr { int32_t Size; int16_t ApiKey; #define RD_KAFKAP_Produce 0 #define RD_KAFKAP_Fetch 1 #define RD_KAFKAP_Offset 2 #define RD_KAFKAP_Metadata 3 #define RD_KAFKAP_LeaderAndIsr 4 #define RD_KAFKAP_StopReplica 5 #define RD_KAFKAP_OffsetCommit 6 #define RD_KAFKAP_OffsetFetch 7 int16_t ApiVersion; int32_t CorrId; /* ClientId follows */ } RD_PACKED; /** * Response header */ struct rd_kafkap_reshdr { int32_t Size; int32_t CorrId; } RD_PACKED; /** * * Kafka protocol string representation: { uint16, data.. } * */ typedef struct rd_kafkap_str_s { int16_t len; /* big endian */ char str[0]; /* allocated dynamically */ } RD_PACKED rd_kafkap_str_t; #define RD_KAFKAP_STR_LEN_NULL -1 /* Returns the actual size of a kafka protocol string representation. */ #define RD_KAFKAP_STR_SIZE(kstr) (int16_t)(sizeof((kstr)->len) + \ ((int16_t)ntohs((kstr)->len) == \ RD_KAFKAP_STR_LEN_NULL ? \ 0 : ntohs((kstr)->len))) /* Returns the length of the string of a kafka protocol string representation */ #define RD_KAFKAP_STR_LEN(kstr) (int)((ntohs((kstr)->len) == \ RD_KAFKAP_STR_LEN_NULL ? \ 0 : (int16_t)ntohs((kstr)->len))) /* Macro suitable for "%.*s" printing. */ #define RD_KAFKAP_STR_PR(kstr) \ ((int16_t)ntohs((kstr)->len) == RD_KAFKAP_STR_LEN_NULL ? \ 0 : (int)ntohs((kstr)->len)), (kstr)->str #define RD_KAFKAP_STR_IS_NULL(kstr) \ ((int16_t)ntohs((kstr)->len) == RD_KAFKAP_STR_LEN_NULL) static inline int rd_kafkap_str_cmp (const rd_kafkap_str_t *a, const rd_kafkap_str_t *b) RD_UNUSED; static inline int rd_kafkap_str_cmp (const rd_kafkap_str_t *a, const rd_kafkap_str_t *b) { if (a->len != b->len) return -1; return memcmp(a->str, b->str, ntohs(a->len)); } static inline int rd_kafkap_str_cmp_str (const rd_kafkap_str_t *a, const char *str) RD_UNUSED; static inline int rd_kafkap_str_cmp_str (const rd_kafkap_str_t *a, const char *str) { int len = strlen(str); if (ntohs(a->len) != len) return -1; return memcmp(a->str, str, ntohs(a->len)); } static inline rd_kafkap_str_t *rd_kafkap_str_new (const char *str) RD_UNUSED; static inline rd_kafkap_str_t *rd_kafkap_str_new (const char *str) { rd_kafkap_str_t *kstr; int len = 0; if (str) len = strlen(str); else len = 0; /* We allocate one more byte so we can null-terminate the string. * This null-termination is not included in the length so it * is not sent over the wire. */ kstr = malloc(sizeof(*kstr) + len + 1); if (str) { kstr->len = ntohs(len); memcpy(kstr->str, str, len+1); } else kstr->len = (int16_t)ntohs(RD_KAFKAP_STR_LEN_NULL); return kstr; } #define rd_kafkap_str_destroy(kstr) free(kstr) #define rd_kafkap_strdupa(kstr) strndupa((kstr)->str,\ RD_KAFKAP_STR_SIZE((kstr))) /** * * Kafka protocol bytes representation: { uint32, data.. } * */ typedef struct rd_kafkap_bytes_s { int32_t len; /* big endian */ char data[0]; /* allocated dynamically */ } RD_PACKED rd_kafkap_bytes_t; #define RD_KAFKAP_BYTES_LEN_NULL -1 /* Returns the actual size of a kafka protocol bytes representation. */ #define RD_KAFKAP_BYTES_SIZE(kbytes) (int32_t)(sizeof((kbytes)->len) + \ ((int32_t)ntohl((kbytes)->len)==\ RD_KAFKAP_BYTES_LEN_NULL ? \ 0 : ntohl((kbytes)->len))) /* Returns the length of the string of a kafka protocol bytes representation */ #define RD_KAFKAP_BYTES_LEN(kbytes) (int32_t)(((int32_t)ntohl((kbytes)->len) ==\ RD_KAFKAP_BYTES_LEN_NULL ? \ 0 : \ (int32_t)ntohl((kbytes)->len))) #define RD_KAFKAP_BYTES_IS_NULL(kbytes) \ ((int32_t)ntohl((kbytes)->len) == RD_KAFKAP_STR_LEN_NULL) static inline int rd_kafkap_bytes_cmp (const rd_kafkap_bytes_t *a, const rd_kafkap_bytes_t *b) RD_UNUSED; static inline int rd_kafkap_bytes_cmp (const rd_kafkap_bytes_t *a, const rd_kafkap_bytes_t *b) { if (a->len != b->len) return -1; return memcmp(a->data, b->data, ntohl(a->len)); } static inline int rd_kafkap_bytes_cmp_bytes (const rd_kafkap_bytes_t *a, const void *data, size_t datalen) RD_UNUSED; static inline int rd_kafkap_bytes_cmp_bytes (const rd_kafkap_bytes_t *a, const void *data, size_t datalen) { if (a->len != datalen) return -1; return memcmp(a->data, data, ntohl(a->len)); } static inline rd_kafkap_bytes_t *rd_kafkap_bytes_new (const void *data, size_t datalen) RD_UNUSED; static inline rd_kafkap_bytes_t *rd_kafkap_bytes_new (const void *data, size_t datalen) { rd_kafkap_bytes_t *kbytes; kbytes = malloc(sizeof(*kbytes) + datalen); if (data) { kbytes->len = ntohl(datalen); memcpy(kbytes->data, data, datalen); } else kbytes->len = (int32_t)ntohl(RD_KAFKAP_BYTES_LEN_NULL); return kbytes; } #define rd_kafkap_bytes_destroy(kbytes) free(kbytes) struct rd_kafkap_FetchRequest { int32_t ReplicaId; int32_t MaxWaitTime; int32_t MinBytes; int32_t TopicArrayCnt; } RD_PACKED; /* Non-protocol representation of a topic's metadata. */ struct rd_kafka_TopicMetadata { int16_t ErrorCode; rd_kafkap_str_t *Name; struct { int16_t ErrorCode; int32_t PartitionId; int32_t Leader; struct rd_kafka_broker_s *rkb; int32_t *Replicas; int32_t Replicas_cnt; int32_t *Isr; int32_t Isr_cnt; } *PartitionMetadata; int32_t PartitionMetadata_cnt; }; librdkafka-0.8.3/rdkafka_timer.c000066400000000000000000000110311227620010100165460ustar00rootroot00000000000000/* * librdkafka - Apache Kafka C library * * Copyright (c) 2012-2013, Magnus Edenhill * All rights reserved. * * Redistribution and use in source and binary forms, with or without * modification, are permitted provided that the following conditions are met: * * 1. Redistributions of source code must retain the above copyright notice, * this list of conditions and the following disclaimer. * 2. Redistributions in binary form must reproduce the above copyright notice, * this list of conditions and the following disclaimer in the documentation * and/or other materials provided with the distribution. * * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE * POSSIBILITY OF SUCH DAMAGE. */ #include "rdkafka_int.h" #include "rd.h" #include "rdtime.h" #include "rdsysqueue.h" #define rd_kafka_timers_lock(rk) pthread_mutex_lock(&(rk)->rk_timers_lock) #define rd_kafka_timers_unlock(rk) pthread_mutex_unlock(&(rk)->rk_timers_lock) static inline int rd_kafka_timer_started (const rd_kafka_timer_t *rtmr) { return rtmr->rtmr_next ? 1 : 0; } static int rd_kafka_timer_cmp (const void *_a, const void *_b) { const rd_kafka_timer_t *a = _a, *b = _b; return a->rtmr_next - b->rtmr_next; } static void rd_kafka_timer_unschedule (rd_kafka_t *rk, rd_kafka_timer_t *rtmr) { TAILQ_REMOVE(&rk->rk_timers, rtmr, rtmr_link); rtmr->rtmr_next = 0; } static void rd_kafka_timer_schedule (rd_kafka_t *rk, rd_kafka_timer_t *rtmr) { rd_kafka_timer_t *first; /* Timer has been stopped */ if (!rtmr->rtmr_interval) return; rtmr->rtmr_next = rd_clock() + rtmr->rtmr_interval; if (!(first = TAILQ_FIRST(&rk->rk_timers)) || first->rtmr_next > rtmr->rtmr_next) { TAILQ_INSERT_HEAD(&rk->rk_timers, rtmr, rtmr_link); pthread_cond_signal(&rk->rk_timers_cond); } else TAILQ_INSERT_SORTED(&rk->rk_timers, rtmr, rtmr_link, rd_kafka_timer_cmp); } /** * Stop a timer that may be started. * If called from inside a timer callback 'lock' must be 0, else 1. */ void rd_kafka_timer_stop (rd_kafka_t *rk, rd_kafka_timer_t *rtmr, int lock) { if (lock) rd_kafka_timers_lock(rk); if (!rd_kafka_timer_started(rtmr)) { if (lock) rd_kafka_timers_unlock(rk); return; } rd_kafka_timer_unschedule(rk, rtmr); rtmr->rtmr_interval = 0; if (lock) rd_kafka_timers_unlock(rk); } /** * Start the provided timer with the given interval. * Upon expiration of the interval the callback will be called in the * main rdkafka thread, after callback return the timer will be restarted. * * Use rd_kafka_timer_stop() to stop a timer. */ void rd_kafka_timer_start (rd_kafka_t *rk, rd_kafka_timer_t *rtmr, int interval, void (*callback) (rd_kafka_t *rk, void *arg), void *arg) { rd_kafka_timers_lock(rk); if (rd_kafka_timer_started(rtmr)) rd_kafka_timer_stop(rk, rtmr, 0/*!lock*/); rtmr->rtmr_interval = interval; rtmr->rtmr_callback = callback; rtmr->rtmr_arg = arg; rd_kafka_timer_schedule(rk, rtmr); rd_kafka_timers_unlock(rk); } /** * Dispatch timers. * Will block up to 'timeout' microseconds before returning. */ void rd_kafka_timers_run (rd_kafka_t *rk, int timeout) { rd_ts_t now = rd_clock(); rd_ts_t end = now + timeout; rd_kafka_timers_lock(rk); while (now <= end) { int64_t sleeptime; rd_kafka_timer_t *rtmr; if (likely((rtmr = TAILQ_FIRST(&rk->rk_timers)) != NULL)) sleeptime = rtmr->rtmr_next - now; else sleeptime = 100000000000000llu; if (sleeptime > 0) { if (sleeptime > (end - now)) sleeptime = end - now; pthread_cond_timedwait_ms(&rk->rk_timers_cond, &rk->rk_timers_lock, sleeptime / 1000); now = rd_clock(); } while ((rtmr = TAILQ_FIRST(&rk->rk_timers)) && rtmr->rtmr_next <= now) { rd_kafka_timer_unschedule(rk, rtmr); rtmr->rtmr_callback(rk, rtmr->rtmr_arg); rd_kafka_timer_schedule(rk, rtmr); } } rd_kafka_timers_unlock(rk); } librdkafka-0.8.3/rdkafka_timer.h000066400000000000000000000037501227620010100165640ustar00rootroot00000000000000/* * librdkafka - Apache Kafka C library * * Copyright (c) 2012-2013, Magnus Edenhill * All rights reserved. * * Redistribution and use in source and binary forms, with or without * modification, are permitted provided that the following conditions are met: * * 1. Redistributions of source code must retain the above copyright notice, * this list of conditions and the following disclaimer. * 2. Redistributions in binary form must reproduce the above copyright notice, * this list of conditions and the following disclaimer in the documentation * and/or other materials provided with the distribution. * * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE * POSSIBILITY OF SUCH DAMAGE. */ #pragma once #include "rd.h" typedef struct rd_kafka_timer_s { TAILQ_ENTRY(rd_kafka_timer_s) rtmr_link; rd_ts_t rtmr_next; int rtmr_interval; /* interval in microseconds */ void (*rtmr_callback) (struct rd_kafka_s *rk, void *arg); void *rtmr_arg; } rd_kafka_timer_t; void rd_kafka_timer_stop (rd_kafka_t *rk, rd_kafka_timer_t *rtmr, int lock); void rd_kafka_timer_start (rd_kafka_t *rk, rd_kafka_timer_t *rtmr, int interval, void (*callback) (struct rd_kafka_s *rk, void *arg), void *arg); void rd_kafka_timers_run (rd_kafka_t *rk, int timeout); librdkafka-0.8.3/rdkafka_topic.c000066400000000000000000000640261227620010100165600ustar00rootroot00000000000000/* * librdkafka - Apache Kafka C library * * Copyright (c) 2012,2013 Magnus Edenhill * All rights reserved. * * Redistribution and use in source and binary forms, with or without * modification, are permitted provided that the following conditions are met: * * 1. Redistributions of source code must retain the above copyright notice, * this list of conditions and the following disclaimer. * 2. Redistributions in binary form must reproduce the above copyright notice, * this list of conditions and the following disclaimer in the documentation * and/or other materials provided with the distribution. * * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE * POSSIBILITY OF SUCH DAMAGE. */ #include "rd.h" #include "rdkafka_int.h" #include "rdkafka_msg.h" #include "rdkafka_topic.h" #include "rdkafka_broker.h" #include "rdlog.h" #include "rdsysqueue.h" #include "rdtime.h" const char *rd_kafka_fetch_states[] = { "none", "offset-query", "offset-wait", "active" }; static rd_kafka_toppar_t *rd_kafka_toppar_new (rd_kafka_topic_t *rkt, int32_t partition) { rd_kafka_toppar_t *rktp; rktp = calloc(1, sizeof(*rktp)); rktp->rktp_partition = partition; rktp->rktp_rkt = rkt; rktp->rktp_fetch_state = RD_KAFKA_TOPPAR_FETCH_NONE; rktp->rktp_offset_fd = -1; rd_kafka_msgq_init(&rktp->rktp_msgq); rd_kafka_msgq_init(&rktp->rktp_xmit_msgq); pthread_mutex_init(&rktp->rktp_lock, NULL); rd_kafka_q_init(&rktp->rktp_fetchq); rd_kafka_toppar_keep(rktp); rd_kafka_topic_keep(rkt); return rktp; } void rd_kafka_toppar_destroy0 (rd_kafka_toppar_t *rktp) { /* Clear queues */ rd_kafka_dr_msgq(rktp->rktp_rkt->rkt_rk, &rktp->rktp_xmit_msgq, RD_KAFKA_RESP_ERR__DESTROY); rd_kafka_dr_msgq(rktp->rktp_rkt->rkt_rk, &rktp->rktp_msgq, RD_KAFKA_RESP_ERR__DESTROY); rd_kafka_q_purge(&rktp->rktp_fetchq); rd_kafka_topic_destroy0(rktp->rktp_rkt); pthread_mutex_destroy(&rktp->rktp_lock); free(rktp); } /** * Returns the appropriate toppar for a given rkt and partition. * The returned toppar has increased refcnt and must be unreffed by calling * rd_kafka_toppar_destroy(). * May return NULL. * * If 'ua_on_miss' is true the UA (unassigned) toppar is returned if * 'partition' was not known locally, else NULL is returned. * * NOTE: Caller must hold rd_kafka_topic_*lock() */ rd_kafka_toppar_t *rd_kafka_toppar_get (const rd_kafka_topic_t *rkt, int32_t partition, int ua_on_miss) { rd_kafka_toppar_t *rktp; if (partition >= 0 && partition < rkt->rkt_partition_cnt) rktp = rkt->rkt_p[partition]; else if (partition == RD_KAFKA_PARTITION_UA || ua_on_miss) rktp = rkt->rkt_ua; else return NULL; if (rktp) rd_kafka_toppar_keep(rktp); return rktp; } /** * Same as rd_kafka_toppar_get() but no need for locking and * looks up the topic first. */ rd_kafka_toppar_t *rd_kafka_toppar_get2 (rd_kafka_t *rk, const rd_kafkap_str_t *topic, int32_t partition, int ua_on_miss) { rd_kafka_topic_t *rkt; rd_kafka_toppar_t *rktp; if (unlikely(!(rkt = rd_kafka_topic_find0(rk, topic)))) return NULL; rd_kafka_topic_rdlock(rkt); rktp = rd_kafka_toppar_get(rkt, partition, ua_on_miss); rd_kafka_topic_unlock(rkt); rd_kafka_topic_destroy0(rkt); return rktp; } /** * Looks for partition 'i' in topic 'rkt's desired list. * * The desired partition list is the list of partitions that are desired * (e.g., by the consumer) but not yet seen on a broker. * As soon as the partition is seen on a broker the toppar is moved from * the desired list and onto the normal rkt_p array. * When the partition on the broker goes away a desired partition is put * back on the desired list. * * Locks: rd_kafka_topic_*lock() must be held. * Note: 'rktp' refcount is increased. */ rd_kafka_toppar_t *rd_kafka_toppar_desired_get (rd_kafka_topic_t *rkt, int32_t partition) { rd_kafka_toppar_t *rktp; TAILQ_FOREACH(rktp, &rkt->rkt_desp, rktp_rktlink) if (rktp->rktp_partition == partition) { rd_kafka_toppar_keep(rktp); return rktp; } return NULL; } /** * Adds 'partition' as a desired partition to topic 'rkt', or updates * an existing partition to be desired. * * Locks: rd_kafka_topic_wrlock() must be held. * NOTE: 'rktp' refcount is increased' */ rd_kafka_toppar_t *rd_kafka_toppar_desired_add (rd_kafka_topic_t *rkt, int32_t partition) { rd_kafka_toppar_t *rktp; if ((rktp = rd_kafka_toppar_get(rkt, partition, 0/*no_ua_on_miss*/))) { rd_kafka_toppar_lock(rktp); rd_kafka_dbg(rkt->rkt_rk, TOPIC, "DESP", "Setting topic %s [%"PRId32"] partition " "as desired", rkt->rkt_topic->str, rktp->rktp_partition); rktp->rktp_flags |= RD_KAFKA_TOPPAR_F_DESIRED; rd_kafka_toppar_unlock(rktp); return rktp; } if ((rktp = rd_kafka_toppar_desired_get(rkt, partition))) return rktp; rktp = rd_kafka_toppar_new(rkt, partition); rktp->rktp_flags |= RD_KAFKA_TOPPAR_F_DESIRED | RD_KAFKA_TOPPAR_F_UNKNOWN; rd_kafka_dbg(rkt->rkt_rk, TOPIC, "DESP", "Adding desired topic %s [%"PRId32"]", rkt->rkt_topic->str, rktp->rktp_partition); TAILQ_INSERT_TAIL(&rkt->rkt_desp, rktp, rktp_rktlink); return rktp; } /** * Unmarks an 'rktp' as desired. * * Locks: rd_kafka_topic_wrlock() must be held. */ void rd_kafka_toppar_desired_del (rd_kafka_toppar_t *rktp) { rd_kafka_toppar_lock(rktp); if (!(rktp->rktp_flags & RD_KAFKA_TOPPAR_F_DESIRED)) { rd_kafka_toppar_unlock(rktp); return; } rktp->rktp_flags &= ~RD_KAFKA_TOPPAR_F_DESIRED; if (rktp->rktp_flags & RD_KAFKA_TOPPAR_F_UNKNOWN) { rktp->rktp_flags &= ~RD_KAFKA_TOPPAR_F_UNKNOWN; TAILQ_REMOVE(&rktp->rktp_rkt->rkt_desp, rktp, rktp_rktlink); } rd_kafka_toppar_unlock(rktp); rd_kafka_dbg(rktp->rktp_rkt->rkt_rk, TOPIC, "DESP", "Removing (un)desired topic %s [%"PRId32"]", rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition); rd_kafka_toppar_destroy(rktp); } /** * Move all messages from toppar 'src' to 'dst'. * This is used when messages migrate between partitions. * * NOTE: Both dst and src must be locked. */ static void rd_kafka_toppar_move_msgs (rd_kafka_toppar_t *dst, rd_kafka_toppar_t *src) { rd_kafka_toppar_keep(src); rd_kafka_msgq_concat(&dst->rktp_msgq, &src->rktp_msgq); rd_kafka_toppar_destroy(src); } /** * Insert message at head of 'rktp' message queue. * This is typically for non-data flash messages. */ void rd_kafka_toppar_insert_msg (rd_kafka_toppar_t *rktp, rd_kafka_msg_t *rkm) { rd_kafka_toppar_lock(rktp); rd_kafka_msgq_insert(&rktp->rktp_msgq, rkm); rd_kafka_toppar_unlock(rktp); } /** * Append message at tail of 'rktp' message queue. */ void rd_kafka_toppar_enq_msg (rd_kafka_toppar_t *rktp, rd_kafka_msg_t *rkm) { rd_kafka_toppar_lock(rktp); rd_kafka_msgq_enq(&rktp->rktp_msgq, rkm); rd_kafka_toppar_unlock(rktp); } /** * Dequeue message from 'rktp' message queue. */ void rd_kafka_toppar_deq_msg (rd_kafka_toppar_t *rktp, rd_kafka_msg_t *rkm) { rd_kafka_toppar_lock(rktp); rd_kafka_msgq_deq(&rktp->rktp_msgq, rkm, 1); rd_kafka_toppar_unlock(rktp); } /** * Inserts all messages from 'rkmq' at head of toppar 'rktp's queue. * 'rkmq' will be cleared. */ void rd_kafka_toppar_insert_msgq (rd_kafka_toppar_t *rktp, rd_kafka_msgq_t *rkmq) { rd_kafka_toppar_lock(rktp); rd_kafka_msgq_concat(rkmq, &rktp->rktp_msgq); rd_kafka_msgq_move(&rktp->rktp_msgq, rkmq); rd_kafka_toppar_unlock(rktp); } /** * Move all messages in 'rkmq' to the unassigned partition, if any. * Returns 0 on success or -1 if there was no UA partition. */ int rd_kafka_toppar_ua_move (rd_kafka_topic_t *rkt, rd_kafka_msgq_t *rkmq) { rd_kafka_toppar_t *rktp_ua; rd_kafka_topic_rdlock(rkt); rktp_ua = rd_kafka_toppar_get(rkt, RD_KAFKA_PARTITION_UA, 0); rd_kafka_topic_unlock(rkt); if (unlikely(rktp_ua == NULL)) return -1; rd_kafka_msgq_concat(&rktp_ua->rktp_msgq, rkmq); rd_kafka_toppar_destroy(rktp_ua); return 0; } void rd_kafka_topic_destroy0 (rd_kafka_topic_t *rkt) { if (likely(rd_atomic_sub(&rkt->rkt_refcnt, 1) > 0)) return; assert(rkt->rkt_refcnt == 0); if (rkt->rkt_topic) rd_kafkap_str_destroy(rkt->rkt_topic); rd_kafka_lock(rkt->rkt_rk); TAILQ_REMOVE(&rkt->rkt_rk->rk_topics, rkt, rkt_link); rkt->rkt_rk->rk_topic_cnt--; rd_kafka_unlock(rkt->rkt_rk); rd_kafka_destroy0(rkt->rkt_rk); rd_kafka_anyconf_destroy(_RK_TOPIC, &rkt->rkt_conf); pthread_rwlock_destroy(&rkt->rkt_lock); free(rkt); } void rd_kafka_topic_destroy (rd_kafka_topic_t *rkt) { return rd_kafka_topic_destroy0(rkt); } /** * Finds and returns a topic based on its name, or NULL if not found. * The 'rkt' refcount is increased by one and the caller must call * rd_kafka_topic_destroy() when it is done with the topic to decrease * the refcount. * * Locality: any thread */ rd_kafka_topic_t *rd_kafka_topic_find (rd_kafka_t *rk, const char *topic) { rd_kafka_topic_t *rkt; rd_kafka_lock(rk); TAILQ_FOREACH(rkt, &rk->rk_topics, rkt_link) { if (!rd_kafkap_str_cmp_str(rkt->rkt_topic, topic)) { rd_kafka_topic_keep(rkt); break; } } rd_kafka_unlock(rk); return rkt; } /** * Same semantics as ..find() but takes a Kafka protocol string instead. */ rd_kafka_topic_t *rd_kafka_topic_find0 (rd_kafka_t *rk, const rd_kafkap_str_t *topic) { rd_kafka_topic_t *rkt; rd_kafka_lock(rk); TAILQ_FOREACH(rkt, &rk->rk_topics, rkt_link) { if (!rd_kafkap_str_cmp(rkt->rkt_topic, topic)) { rd_kafka_topic_keep(rkt); break; } } rd_kafka_unlock(rk); return rkt; } /** * Create new topic handle. * * Locality: application thread */ rd_kafka_topic_t *rd_kafka_topic_new (rd_kafka_t *rk, const char *topic, rd_kafka_topic_conf_t *conf) { rd_kafka_topic_t *rkt; /* Verify configuration */ if (!topic || (conf && (conf->message_timeout_ms <= 0 || conf->request_timeout_ms <= 0))) { errno = EINVAL; return NULL; } if ((rkt = rd_kafka_topic_find(rk, topic))) return rkt; rkt = calloc(1, sizeof(*rkt)); rkt->rkt_topic = rd_kafkap_str_new(topic); rkt->rkt_rk = rk; if (!conf) conf = rd_kafka_topic_conf_new(); rkt->rkt_conf = *conf; free(conf); /* Default partitioner: random */ if (!rkt->rkt_conf.partitioner) rkt->rkt_conf.partitioner = rd_kafka_msg_partitioner_random; rd_kafka_dbg(rk, TOPIC, "TOPIC", "New local topic: %.*s", RD_KAFKAP_STR_PR(rkt->rkt_topic)); TAILQ_INIT(&rkt->rkt_desp); rd_kafka_topic_keep(rkt); rd_kafka_keep(rk); pthread_rwlock_init(&rkt->rkt_lock, NULL); /* Create unassigned partition */ rkt->rkt_ua = rd_kafka_toppar_new(rkt, RD_KAFKA_PARTITION_UA); rd_kafka_lock(rk); TAILQ_INSERT_TAIL(&rk->rk_topics, rkt, rkt_link); rk->rk_topic_cnt++; rd_kafka_unlock(rk); /* Query for the topic leader (async) */ rd_kafka_topic_leader_query(rk, rkt); return rkt; } /** * Returns the name of a topic. * NOTE: * The topic Kafka String representation is crafted with an extra byte * at the end for the Nul that is not included in the length, this way * we can use the topic's String directly. * This is not true for Kafka Strings read from the network. */ const char *rd_kafka_topic_name (const rd_kafka_topic_t *rkt) { return rkt->rkt_topic->str; } /** * Delegates broker 'rkb' as leader for toppar 'rktp'. * 'rkb' may be NULL to undelegate leader. * * Locks: Caller must have rd_kafka_topic_wrlock(rktp->rktp_rkt) * AND rd_kafka_toppar_lock(rktp) held. */ void rd_kafka_toppar_broker_delegate (rd_kafka_toppar_t *rktp, rd_kafka_broker_t *rkb) { if (rktp->rktp_leader == rkb) return; rd_kafka_toppar_keep(rktp); if (rktp->rktp_leader) { rd_kafka_broker_t *old_rkb = rktp->rktp_leader; rd_kafka_dbg(rktp->rktp_rkt->rkt_rk, TOPIC, "BRKDELGT", "Broker %s no longer leader " "for topic %.*s [%"PRId32"]", rktp->rktp_leader->rkb_name, RD_KAFKAP_STR_PR(rktp->rktp_rkt->rkt_topic), rktp->rktp_partition); rd_kafka_broker_toppars_wrlock(old_rkb); TAILQ_REMOVE(&old_rkb->rkb_toppars, rktp, rktp_rkblink); old_rkb->rkb_toppar_cnt--; rktp->rktp_leader = NULL; rd_kafka_broker_toppars_unlock(old_rkb); rd_kafka_toppar_destroy(rktp); rd_kafka_broker_destroy(old_rkb); } if (rkb) { rd_kafka_dbg(rktp->rktp_rkt->rkt_rk, TOPIC, "BRKDELGT", "Broker %s is now leader for topic %.*s " "[%"PRId32"] with %i messages " "(%"PRIu64" bytes) queued", rkb->rkb_name, RD_KAFKAP_STR_PR(rktp->rktp_rkt->rkt_topic), rktp->rktp_partition, rktp->rktp_msgq.rkmq_msg_cnt, rktp->rktp_msgq.rkmq_msg_bytes); rd_kafka_broker_toppars_wrlock(rkb); rd_kafka_toppar_keep(rktp); TAILQ_INSERT_TAIL(&rkb->rkb_toppars, rktp, rktp_rkblink); rkb->rkb_toppar_cnt++; rktp->rktp_leader = rkb; rd_kafka_broker_keep(rkb); rd_kafka_broker_toppars_unlock(rkb); } else { rd_kafka_dbg(rktp->rktp_rkt->rkt_rk, TOPIC, "BRKDELGT", "No broker is leader for topic %.*s [%"PRId32"]", RD_KAFKAP_STR_PR(rktp->rktp_rkt->rkt_topic), rktp->rktp_partition); } rd_kafka_toppar_destroy(rktp); /* from keep() above */ } /** * Update the leader for a topic+partition. * Returns 1 if the leader was changed, else 0. * NOTE: rd_kafka_topic_wrlock(rkt) MUST be held. */ static int rd_kafka_topic_leader_update (rd_kafka_topic_t *rkt, int32_t partition, int32_t leader, rd_kafka_broker_t *rkb) { rd_kafka_t *rk = rkt->rkt_rk; rd_kafka_toppar_t *rktp; rktp = rd_kafka_toppar_get(rkt, partition, 0); assert(rktp); if (!rkb) { int had_leader = rktp->rktp_leader ? 1 : 0; if (leader == -1) /* Topic lost its leader */; else rd_kafka_log(rk, LOG_NOTICE, "TOPICBRK", "Topic %s [%"PRId32"] migrated to unknown " "broker %"PRId32": " "requesting metadata update", rkt->rkt_topic->str, partition, leader); rd_kafka_toppar_broker_delegate(rktp, NULL); /* Query for the topic leader (async) */ if (had_leader) rd_kafka_topic_leader_query(rk, rkt); rd_kafka_toppar_unlock(rktp); rd_kafka_toppar_destroy(rktp); /* from get() */ return had_leader ? 1 : 0; } if (rktp->rktp_leader) { if (rktp->rktp_leader == rkb) { /* No change in broker */ rd_kafka_dbg(rk, TOPIC, "TOPICUPD", "No leader change for topic %s " "[%"PRId32"] with leader %"PRId32, rkt->rkt_topic->str, partition, leader); rd_kafka_toppar_destroy(rktp); /* from get() */ return 0; } rd_kafka_dbg(rk, TOPIC, "TOPICUPD", "Topic %s [%"PRId32"] migrated from " "broker %"PRId32" to %"PRId32, rkt->rkt_topic->str, partition, rktp->rktp_leader->rkb_nodeid, rkb->rkb_nodeid); } rd_kafka_toppar_broker_delegate(rktp, rkb); rd_kafka_toppar_destroy(rktp); /* from get() */ return 1; } /** * Remove all partitions from a topic, including the ua. */ void rd_kafka_topic_partitions_remove (rd_kafka_topic_t *rkt) { rd_kafka_toppar_t *rktp; int i; rd_kafka_topic_keep(rkt); rd_kafka_topic_wrlock(rkt); /* Remove all partitions */ for (i = 0 ; i < rkt->rkt_partition_cnt ; i++) { if (!(rktp = rd_kafka_toppar_get(rkt, i, 0))) continue; rd_kafka_toppar_lock(rktp); rd_kafka_msgq_purge(rkt->rkt_rk, &rktp->rktp_msgq); rd_kafka_toppar_unlock(rktp); rd_kafka_toppar_destroy(rktp); /* _get() */ rd_kafka_toppar_destroy(rktp); /* remove partition */ } if (rkt->rkt_p) free(rkt->rkt_p); rkt->rkt_p = NULL; rkt->rkt_partition_cnt = 0; if ((rktp = rd_kafka_toppar_get(rkt, RD_KAFKA_PARTITION_UA, 0))) { rd_kafka_toppar_lock(rktp); rd_kafka_msgq_purge(rkt->rkt_rk, &rktp->rktp_msgq); rkt->rkt_ua = NULL; rd_kafka_toppar_unlock(rktp); rd_kafka_toppar_destroy(rktp); /* for get() */ rd_kafka_toppar_destroy(rktp); /* for final destruction */ } rd_kafka_topic_unlock(rkt); rd_kafka_topic_destroy0(rkt); } /** * Update the number of partitions for a topic and takes according actions. * Returns 1 if the partition count changed, else 0. * NOTE: rd_kafka_topic_wrlock(rkt) MUST be held. */ static int rd_kafka_topic_partition_cnt_update (rd_kafka_topic_t *rkt, int32_t partition_cnt) { rd_kafka_t *rk = rkt->rkt_rk; rd_kafka_toppar_t **rktps; rd_kafka_toppar_t *rktp_ua; rd_kafka_toppar_t *rktp; int32_t i; if (rkt->rkt_partition_cnt == partition_cnt) { rd_kafka_dbg(rk, TOPIC, "PARTCNT", "No change in partition count for topic %s", rkt->rkt_topic->str); return 0; /* No change in partition count */ } if (unlikely(rkt->rkt_partition_cnt != 0)) rd_kafka_log(rk, LOG_NOTICE, "PARTCNT", "Topic %s partition count changed " "from %"PRId32" to %"PRId32, rkt->rkt_topic->str, rkt->rkt_partition_cnt, partition_cnt); else rd_kafka_dbg(rk, TOPIC, "PARTCNT", "Topic %s partition count changed " "from %"PRId32" to %"PRId32, rkt->rkt_topic->str, rkt->rkt_partition_cnt, partition_cnt); /* Create and assign new partition list */ if (partition_cnt > 0) rktps = calloc(partition_cnt, sizeof(*rktps)); else rktps = NULL; for (i = 0 ; i < partition_cnt ; i++) { if (i >= rkt->rkt_partition_cnt) { /* New partition. Check if its in the list of * desired partitions first. */ if ((rktp = rd_kafka_toppar_desired_get(rkt, i))) { /* Remove from desp list since the partition * is now known. */ rd_kafka_toppar_lock(rktp); rktp->rktp_flags &= ~RD_KAFKA_TOPPAR_F_UNKNOWN; rd_kafka_toppar_unlock(rktp); TAILQ_REMOVE(&rkt->rkt_desp, rktp, rktp_rktlink); } else rktp = rd_kafka_toppar_new(rkt, i); rktps[i] = rktp; } else { /* Move existing partition */ rktps[i] = rkt->rkt_p[i]; } } rktp_ua = rd_kafka_toppar_get(rkt, RD_KAFKA_PARTITION_UA, 0); /* Remove excessive partitions if partition count decreased. */ for (; i < rkt->rkt_partition_cnt ; i++) { rktp = rkt->rkt_p[i]; /* Partition has gone away, move messages to UA or drop */ if (likely(rktp_ua != NULL)) rd_kafka_toppar_move_msgs(rktp_ua, rktp); else rd_kafka_msgq_purge(rkt->rkt_rk, &rktp->rktp_msgq); /* If this is a desired partition move it back on to * the desired list. */ rd_kafka_toppar_lock(rktp); if (rktp->rktp_flags & RD_KAFKA_TOPPAR_F_DESIRED) { /* Reinsert on desp list since the partition * is no longer known. */ assert(!(rktp->rktp_flags & RD_KAFKA_TOPPAR_F_UNKNOWN)); rktp->rktp_flags |= RD_KAFKA_TOPPAR_F_UNKNOWN; TAILQ_INSERT_TAIL(&rkt->rkt_desp, rktp, rktp_rktlink); } rd_kafka_toppar_unlock(rktp); rd_kafka_toppar_destroy(rktp); } if (likely(rktp_ua != NULL)) rd_kafka_toppar_destroy(rktp_ua); /* .._get() above */ if (rkt->rkt_p) free(rkt->rkt_p); rkt->rkt_p = rktps; rkt->rkt_partition_cnt = partition_cnt; return 1; } /** * Assign messages on the UA partition to available partitions. * Locks: rd_kafka_topic_*lock() must be held. */ static void rd_kafka_topic_assign_uas (rd_kafka_topic_t *rkt) { rd_kafka_t *rk = rkt->rkt_rk; rd_kafka_toppar_t *rktp_ua; rd_kafka_msg_t *rkm, *tmp; rd_kafka_msgq_t uas = RD_KAFKA_MSGQ_INITIALIZER(uas); rd_kafka_msgq_t failed = RD_KAFKA_MSGQ_INITIALIZER(failed); int cnt; rktp_ua = rd_kafka_toppar_get(rkt, RD_KAFKA_PARTITION_UA, 0); if (unlikely(!rktp_ua)) { rd_kafka_dbg(rk, TOPIC, "ASSIGNUA", "No UnAssigned partition available for %s", rkt->rkt_topic->str); return; } /* Assign all unassigned messages to new topics. */ rd_kafka_dbg(rk, TOPIC, "PARTCNT", "Partitioning %i unassigned messages in topic %.*s to " "%"PRId32" partitions", rktp_ua->rktp_msgq.rkmq_msg_cnt, RD_KAFKAP_STR_PR(rkt->rkt_topic), rkt->rkt_partition_cnt); rd_kafka_toppar_lock(rktp_ua); rd_kafka_msgq_move(&uas, &rktp_ua->rktp_msgq); cnt = uas.rkmq_msg_cnt; rd_kafka_toppar_unlock(rktp_ua); TAILQ_FOREACH_SAFE(rkm, &uas.rkmq_msgs, rkm_link, tmp) { /* Fast-path for failing messages with forced partition */ if (rkm->rkm_partition != RD_KAFKA_PARTITION_UA && rkm->rkm_partition >= rkt->rkt_partition_cnt && rkt->rkt_state != RD_KAFKA_TOPIC_S_INIT) { rd_kafka_msgq_enq(&failed, rkm); continue; } if (unlikely(rd_kafka_msg_partitioner(rkt, rkm, 0) != 0)) { /* Desired partition not available */ rd_kafka_msgq_enq(&failed, rkm); } } rd_kafka_dbg(rk, TOPIC, "UAS", "%i/%i messages were partitioned in topic %s", cnt - failed.rkmq_msg_cnt, cnt, rkt->rkt_topic->str); if (failed.rkmq_msg_cnt > 0) { /* Fail the messages */ rd_kafka_dbg(rk, TOPIC, "UAS", "%i/%i messages failed partitioning in topic %s", uas.rkmq_msg_cnt, cnt, rkt->rkt_topic->str); rd_kafka_dr_msgq(rk, &failed, rkt->rkt_state == RD_KAFKA_TOPIC_S_UNKNOWN ? RD_KAFKA_RESP_ERR__UNKNOWN_TOPIC : RD_KAFKA_RESP_ERR__UNKNOWN_PARTITION); } rd_kafka_toppar_destroy(rktp_ua); /* from get() */ } /** * Received metadata request contained no information about topic 'rkt' * and thus indicates the topic is not available in the cluster. */ void rd_kafka_topic_metadata_none (rd_kafka_topic_t *rkt) { rd_kafka_topic_wrlock(rkt); rkt->rkt_ts_metadata = rd_clock(); rkt->rkt_state = RD_KAFKA_TOPIC_S_UNKNOWN; /* Update number of partitions */ rd_kafka_topic_partition_cnt_update(rkt, 0); /* Purge messages with forced partition */ rd_kafka_topic_assign_uas(rkt); rd_kafka_topic_unlock(rkt); } /** * Update a topic from metadata. * Returns 1 if the number of partitions changed, 0 if not, and -1 if the * topic is unknown. */ int rd_kafka_topic_metadata_update (rd_kafka_broker_t *rkb, const struct rd_kafka_TopicMetadata *tm) { rd_kafka_topic_t *rkt; int upd = 0; int j; if (!(rkt = rd_kafka_topic_find(rkb->rkb_rk, rd_kafkap_strdupa(tm->Name)))) return -1; /* Ignore topics that we dont have locally. */ if (tm->ErrorCode != RD_KAFKA_RESP_ERR_NO_ERROR) rd_rkb_dbg(rkb, TOPIC, "METADATA", "Error in metadata reply for " "topic %s (PartCnt %"PRId32"): %s", rkt->rkt_topic->str, tm->PartitionMetadata_cnt, rd_kafka_err2str(tm->ErrorCode)); /* Look up brokers before acquiring rkt lock to preserve lock order */ rd_kafka_lock(rkb->rkb_rk); for (j = 0 ; j < tm->PartitionMetadata_cnt ; j++) { if (tm->PartitionMetadata[j].Leader == -1) { tm->PartitionMetadata[j].rkb = NULL; continue; } tm->PartitionMetadata[j].rkb = rd_kafka_broker_find_by_nodeid(rkb->rkb_rk, tm->PartitionMetadata[j]. Leader); } rd_kafka_unlock(rkb->rkb_rk); rd_kafka_topic_wrlock(rkt); rkt->rkt_ts_metadata = rd_clock(); /* Set topic state */ if (tm->ErrorCode == RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_OR_PART) rkt->rkt_state = RD_KAFKA_TOPIC_S_UNKNOWN; else rkt->rkt_state = RD_KAFKA_TOPIC_S_EXISTS; /* Update number of partitions */ upd += rd_kafka_topic_partition_cnt_update(rkt, tm->PartitionMetadata_cnt); /* Update leader for each partition */ for (j = 0 ; j < tm->PartitionMetadata_cnt ; j++) { rd_rkb_dbg(rkb, METADATA, "METADATA", " Topic %s partition %"PRId32" Leader %"PRId32, rkt->rkt_topic->str, tm->PartitionMetadata[j].PartitionId, tm->PartitionMetadata[j].Leader); /* Update leader for partition */ upd += rd_kafka_topic_leader_update(rkt, tm->PartitionMetadata[j]. PartitionId, tm->PartitionMetadata[j]. Leader, tm->PartitionMetadata[j]. rkb); } /* Try to assign unassigned messages to new partitions, or fail them */ if (upd > 0 || rkt->rkt_state == RD_KAFKA_TOPIC_S_UNKNOWN) rd_kafka_topic_assign_uas(rkt); rd_kafka_topic_unlock(rkt); rd_kafka_topic_destroy0(rkt); /* from find() */ /* Drop broker references */ for (j = 0 ; j < tm->PartitionMetadata_cnt ; j++) if (tm->PartitionMetadata[j].rkb) rd_kafka_broker_destroy(tm->PartitionMetadata[j].rkb); return upd; } /** * Scan all topics and partitions for: * - timed out messages. * - topics that needs to be created on the broker. */ int rd_kafka_topic_scan_all (rd_kafka_t *rk, rd_ts_t now) { rd_kafka_topic_t *rkt; rd_kafka_toppar_t *rktp; rd_kafka_msgq_t timedout; int tpcnt = 0; int totcnt; rd_kafka_msgq_init(&timedout); rd_kafka_lock(rk); TAILQ_FOREACH(rkt, &rk->rk_topics, rkt_link) { int p; rd_kafka_topic_rdlock(rkt); if (rkt->rkt_partition_cnt == 0) { /* If this partition is unknown by brokers try * to create it by sending a topic-specific * metadata request. * This requires "auto.create.topics.enable=true" * on the brokers. */ rd_kafka_topic_unlock(rkt); rd_kafka_topic_leader_query0(rk, rkt, 0/*no_rk_lock*/); rd_kafka_topic_rdlock(rkt); } for (p = RD_KAFKA_PARTITION_UA ; p < rkt->rkt_partition_cnt ; p++) { if (!(rktp = rd_kafka_toppar_get(rkt, p, 0))) continue; rd_kafka_toppar_lock(rktp); /* Scan toppar's message queue for timeouts */ if (rd_kafka_msgq_age_scan(&rktp->rktp_msgq, &timedout, now) > 0) tpcnt++; rd_kafka_toppar_unlock(rktp); rd_kafka_toppar_destroy(rktp); } rd_kafka_topic_unlock(rkt); } rd_kafka_unlock(rk); if ((totcnt = timedout.rkmq_msg_cnt) > 0) { rd_kafka_dbg(rk, MSG, "TIMEOUT", "%i message(s) from %i toppar(s) timed out", timedout.rkmq_msg_cnt, tpcnt); rd_kafka_dr_msgq(rk, &timedout, RD_KAFKA_RESP_ERR__MSG_TIMED_OUT); } return totcnt; } /** * Locks: rd_kafka_topic_*lock() must be held. */ int rd_kafka_topic_partition_available (const rd_kafka_topic_t *rkt, int32_t partition) { int avail; rd_kafka_toppar_t *rktp; rktp = rd_kafka_toppar_get(rkt, partition, 0/*no ua-on-miss*/); if (unlikely(!rktp)) return 0; rd_kafka_toppar_lock(rktp); avail = rktp->rktp_leader ? 1 : 0; rd_kafka_toppar_unlock(rktp); rd_kafka_toppar_destroy(rktp); return avail; } librdkafka-0.8.3/rdkafka_topic.h000066400000000000000000000061351227620010100165620ustar00rootroot00000000000000/* * librdkafka - Apache Kafka C library * * Copyright (c) 2012,2013 Magnus Edenhill * All rights reserved. * * Redistribution and use in source and binary forms, with or without * modification, are permitted provided that the following conditions are met: * * 1. Redistributions of source code must retain the above copyright notice, * this list of conditions and the following disclaimer. * 2. Redistributions in binary form must reproduce the above copyright notice, * this list of conditions and the following disclaimer in the documentation * and/or other materials provided with the distribution. * * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE * POSSIBILITY OF SUCH DAMAGE. */ #pragma once extern const char *rd_kafka_fetch_states[]; void rd_kafka_toppar_destroy0 (rd_kafka_toppar_t *rktp); void rd_kafka_toppar_insert_msg (rd_kafka_toppar_t *rktp, rd_kafka_msg_t *rkm); void rd_kafka_toppar_enq_msg (rd_kafka_toppar_t *rktp, rd_kafka_msg_t *rkm); void rd_kafka_toppar_deq_msg (rd_kafka_toppar_t *rktp, rd_kafka_msg_t *rkm); void rd_kafka_toppar_insert_msgq (rd_kafka_toppar_t *rktp, rd_kafka_msgq_t *rkmq); #define rd_kafka_topic_keep(rkt) (void)rd_atomic_add(&(rkt->rkt_refcnt), 1) void rd_kafka_topic_destroy0 (rd_kafka_topic_t *rkt); rd_kafka_toppar_t *rd_kafka_toppar_get (const rd_kafka_topic_t *rkt, int32_t partition, int ua_on_miss); rd_kafka_toppar_t *rd_kafka_toppar_get2 (rd_kafka_t *rk, const rd_kafkap_str_t *topic, int32_t partition, int ua_on_miss); rd_kafka_toppar_t *rd_kafka_toppar_desired_get (rd_kafka_topic_t *rkt, int32_t partition); rd_kafka_toppar_t *rd_kafka_toppar_desired_add (rd_kafka_topic_t *rkt, int32_t partition); void rd_kafka_toppar_desired_del (rd_kafka_toppar_t *rktp); rd_kafka_topic_t *rd_kafka_topic_find (rd_kafka_t *rk, const char *topic); rd_kafka_topic_t *rd_kafka_topic_find0 (rd_kafka_t *rk, const rd_kafkap_str_t *topic); int rd_kafka_toppar_ua_move (rd_kafka_topic_t *rkt, rd_kafka_msgq_t *rkmq); void rd_kafka_toppar_broker_delegate (rd_kafka_toppar_t *rktp, rd_kafka_broker_t *rkb); void rd_kafka_topic_partitions_remove (rd_kafka_topic_t *rkt); void rd_kafka_topic_metadata_none (rd_kafka_topic_t *rkt); int rd_kafka_topic_metadata_update (rd_kafka_broker_t *rkb, const struct rd_kafka_TopicMetadata *tm); int rd_kafka_topic_scan_all (rd_kafka_t *rk, rd_ts_t now); librdkafka-0.8.3/rdlog.c000077500000000000000000000041111227620010100150560ustar00rootroot00000000000000/* * librd - Rapid Development C library * * Copyright (c) 2012-2013, Magnus Edenhill * All rights reserved. * * Redistribution and use in source and binary forms, with or without * modification, are permitted provided that the following conditions are met: * * 1. Redistributions of source code must retain the above copyright notice, * this list of conditions and the following disclaimer. * 2. Redistributions in binary form must reproduce the above copyright notice, * this list of conditions and the following disclaimer in the documentation * and/or other materials provided with the distribution. * * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE * POSSIBILITY OF SUCH DAMAGE. */ #include #include #include #include "rd.h" #include "rdthread.h" #include "rdlog.h" void rd_hexdump (FILE *fp, const char *name, const void *ptr, size_t len) { const char *p = (const char *)ptr; int of = 0; if (name) fprintf(fp, "%s hexdump (%zu bytes):\n", name, len); for (of = 0 ; of < len ; of += 16) { char hexen[16*3+1]; char charen[16+1]; int hof = 0; int cof = 0; int i; for (i = of ; i < of + 16 && i < len ; i++) { hof += sprintf(hexen+hof, "%02x ", p[i] & 0xff); cof += sprintf(charen+cof, "%c", isprint((int)p[i]) ? p[i] : '.'); } fprintf(fp, "%08x: %-48s %-16s\n", of, hexen, charen); } } librdkafka-0.8.3/rdlog.h000077500000000000000000000027641227620010100150770ustar00rootroot00000000000000/* * librd - Rapid Development C library * * Copyright (c) 2012-2013, Magnus Edenhill * All rights reserved. * * Redistribution and use in source and binary forms, with or without * modification, are permitted provided that the following conditions are met: * * 1. Redistributions of source code must retain the above copyright notice, * this list of conditions and the following disclaimer. * 2. Redistributions in binary form must reproduce the above copyright notice, * this list of conditions and the following disclaimer in the documentation * and/or other materials provided with the distribution. * * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE * POSSIBILITY OF SUCH DAMAGE. */ #pragma once void rd_hexdump (FILE *fp, const char *name, const void *ptr, size_t len); librdkafka-0.8.3/rdqueue.c000066400000000000000000000106261227620010100154260ustar00rootroot00000000000000/* * librd - Rapid Development C library * * Copyright (c) 2012-2013, Magnus Edenhill * All rights reserved. * * Redistribution and use in source and binary forms, with or without * modification, are permitted provided that the following conditions are met: * * 1. Redistributions of source code must retain the above copyright notice, * this list of conditions and the following disclaimer. * 2. Redistributions in binary form must reproduce the above copyright notice, * this list of conditions and the following disclaimer in the documentation * and/or other materials provided with the distribution. * * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE * POSSIBILITY OF SUCH DAMAGE. */ #include "rd.h" #include "rdthread.h" #include "rdqueue.h" void rd_fifoq_destroy (rd_fifoq_t *rfq) { rd_fifoq_elm_t *rfqe; rd_mutex_lock(&rfq->rfq_lock); while ((rfqe = TAILQ_FIRST(&rfq->rfq_q))) { TAILQ_REMOVE(&rfq->rfq_q, rfqe, rfqe_link); free(rfqe); } rd_mutex_unlock(&rfq->rfq_lock); } rd_fifoq_t *rd_fifoq_init (rd_fifoq_t *rfq) { if (!rfq) rfq = calloc(1, sizeof(*rfq)); TAILQ_INIT(&rfq->rfq_q); rd_mutex_init(&rfq->rfq_lock); rd_cond_init(&rfq->rfq_cond, NULL); rfq->rfq_inited = 1; return rfq; } void rd_fifoq_set_max_size (rd_fifoq_t *rfq, int max_size, int taildrop) { rd_mutex_lock(&rfq->rfq_lock); rfq->rfq_max_size = max_size; rfq->rfq_taildrop = !!taildrop; rd_mutex_unlock(&rfq->rfq_lock); } /** * Adds 'ptr' to FIFO queue. * The optional '*ptr_purged' will be set to the purged element's ptr * if the max_size settings of the fifo has been exceeded; i.e., it will * contain the pushed-out element's ptr so that the application can * update that object's state. */ void rd_fifoq_add0 (rd_fifoq_t *rfq, void *ptr, void **ptr_purged) { rd_fifoq_elm_t *rfqe; if (ptr_purged) *ptr_purged = NULL; assert(rfq->rfq_inited); rfqe = malloc(sizeof(*rfqe)); rfqe->rfqe_refcnt = 2; /* one for rfq, one for caller */ rfqe->rfqe_ptr = ptr; rd_mutex_lock(&rfq->rfq_lock); if (rfq->rfq_max_size != 0 && rfq->rfq_cnt >= rfq->rfq_max_size) { rd_fifoq_elm_t *purge; /* Queue has reached max size, drop an entry. */ if (rfq->rfq_taildrop) purge = TAILQ_LAST(&rfq->rfq_q, rd_fifoq_elm_head_s); else purge = TAILQ_FIRST(&rfq->rfq_q); if (ptr_purged) *ptr_purged = purge->rfqe_ptr; rfq->rfq_cnt--; TAILQ_REMOVE(&rfq->rfq_q, purge, rfqe_link); if (purge->rfqe_refcnt == 1) { /* Only fifoq's refcount remained, * this entry is no longer desired on the fifo, * ignore and remove it. */ rd_fifoq_elm_release0(rfq, purge); } } TAILQ_INSERT_TAIL(&rfq->rfq_q, rfqe, rfqe_link); rfq->rfq_cnt++; rd_cond_signal(&rfq->rfq_cond); rd_mutex_unlock(&rfq->rfq_lock); } rd_fifoq_elm_t *rd_fifoq_pop0 (rd_fifoq_t *rfq, int nowait, int timeout_ms) { rd_fifoq_elm_t *rfqe; /* Pop the next valid element from the FIFO. */ do { rd_mutex_lock(&rfq->rfq_lock); while (!(rfqe = TAILQ_FIRST(&rfq->rfq_q))) { if (nowait) { rd_mutex_unlock(&rfq->rfq_lock); return NULL; } if (timeout_ms) { if (rd_cond_timedwait_ms(&rfq->rfq_cond, &rfq->rfq_lock, timeout_ms) == ETIMEDOUT) { rd_mutex_unlock(&rfq->rfq_lock); return NULL; } } else rd_cond_wait(&rfq->rfq_cond, &rfq->rfq_lock); } assert(rfq->rfq_cnt > 0); rfq->rfq_cnt--; TAILQ_REMOVE(&rfq->rfq_q, rfqe, rfqe_link); if (rfqe->rfqe_refcnt == 1) { /* Only fifoq's refcount remains, * this entry is no longer desired on the fifo, * ignore and remove it. */ rd_fifoq_elm_release0(rfq, rfqe); continue; } break; } while (rfqe == NULL); rd_fifoq_elm_release0(rfq, rfqe); rd_mutex_unlock(&rfq->rfq_lock); return rfqe; } librdkafka-0.8.3/rdqueue.h000066400000000000000000000071171227620010100154340ustar00rootroot00000000000000/* * librd - Rapid Development C library * * Copyright (c) 2012-2013, Magnus Edenhill * All rights reserved. * * Redistribution and use in source and binary forms, with or without * modification, are permitted provided that the following conditions are met: * * 1. Redistributions of source code must retain the above copyright notice, * this list of conditions and the following disclaimer. * 2. Redistributions in binary form must reproduce the above copyright notice, * this list of conditions and the following disclaimer in the documentation * and/or other materials provided with the distribution. * * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE * POSSIBILITY OF SUCH DAMAGE. */ #pragma once #include "rd.h" #include "rdtypes.h" #include "rdsysqueue.h" /** * Thread-safe FIFO queues. * Typical usage is for work-queues (see tests/0001-fifoq.c for an example) * * * It is up the object code to properly lock the object itself. * * * Usage: * * Caller thread: * -- Add to fifoq: * rd_fifoq_add(&my_fifoq, myobj); * * In worker thread: * -- Dequeue and process: * while (1) { * rfqe = rd_fifoq_pop_wait(&my_fifoq); * myobj = rfqe->ptr; * perform_work(myobj); * rd_fifoq_elm_release(rfqe); * } */ typedef struct rd_fifoq_elm_s { TAILQ_ENTRY(rd_fifoq_elm_s) rfqe_link; int rfqe_refcnt; void *rfqe_ptr; } rd_fifoq_elm_t; TAILQ_HEAD(rd_fifoq_elm_head_s, rd_fifoq_elm_s); typedef struct rd_fifoq_s { TAILQ_HEAD(, rd_fifoq_elm_s) rfq_q; rd_mutex_t rfq_lock; rd_cond_t rfq_cond; int rfq_cnt; int rfq_max_size; int rfq_taildrop; int rfq_inited; } rd_fifoq_t; void rd_fifoq_destroy (rd_fifoq_t *rfg); rd_fifoq_t *rd_fifoq_init (rd_fifoq_t *rfq); #define RD_FIFOQ_INITIALIZER(rfq) \ { \ .rfq_q = TAILQ_HEAD_INITIALIZER((rfq).rfq_q), \ .rfq_lock = RD_MUTEX_INITIALIZER, \ .rfq_cond = RD_COND_INITIALIZER, \ .rfq_inited = 1 \ } void rd_fifoq_set_max_size (rd_fifoq_t *rfq, int max_size, int taildrop); void rd_fifoq_add0 (rd_fifoq_t *rfq, void *ptr, void **ptr_purged); #define rd_fifoq_add(rfq,ptr) rd_fifoq_add0(rfq,ptr,NULL) #define rd_fifoq_add_purge(rfq,ptr,ptr_purged) \ rd_fifoq_add0(rfq,ptr,(void **)ptr_purged) rd_fifoq_elm_t *rd_fifoq_pop0 (rd_fifoq_t *rfq, int no_wait, int timeout_ms); #define rd_fifoq_pop_wait(rfq) rd_fifoq_pop0(rfq, 0, 0) #define rd_fifoq_pop_timedwait(rfq,tmo) rd_fifoq_pop0(rfq, 0, tmo) #define rd_fifoq_pop(rfq) rd_fifoq_pop0(rfq, 1, 0) static inline void rd_fifoq_elm_release0 (rd_fifoq_t *rfq, rd_fifoq_elm_t *rfqe) { if (rd_atomic_sub(&rfqe->rfqe_refcnt, 1) > 0) return; free(rfqe); } #define rd_fifoq_elm_release(RFQ,RFQE) do { \ rd_mutex_lock(&(RFQ)->rfq_lock); \ rd_fifoq_elm_release0(RFQ, RFQE); \ rd_mutex_unlock(&(RFQ)->rfq_lock); \ } while (0) librdkafka-0.8.3/rdrand.c000066400000000000000000000036341227620010100152270ustar00rootroot00000000000000/* * librd - Rapid Development C library * * Copyright (c) 2012, Magnus Edenhill * All rights reserved. * * Redistribution and use in source and binary forms, with or without * modification, are permitted provided that the following conditions are met: * * 1. Redistributions of source code must retain the above copyright notice, * this list of conditions and the following disclaimer. * 2. Redistributions in binary form must reproduce the above copyright notice, * this list of conditions and the following disclaimer in the documentation * and/or other materials provided with the distribution. * * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE * POSSIBILITY OF SUCH DAMAGE. */ #include "rd.h" #include "rdrand.h" void rd_array_shuffle (void *base, size_t nmemb, size_t entry_size) { int i; void *tmp = alloca(entry_size); /* FIXME: Optimized version for word-sized entries. */ for (i = nmemb - 1 ; i > 0 ; i--) { int j = rd_jitter(0, i); if (unlikely(i == j)) continue; memcpy(tmp, (char *)base + (i*entry_size), entry_size); memcpy((char *)base+(i*entry_size), (char *)base+(j*entry_size), entry_size); memcpy((char *)base+(j*entry_size), tmp, entry_size); } } librdkafka-0.8.3/rdrand.h000066400000000000000000000034531227620010100152330ustar00rootroot00000000000000/* * librd - Rapid Development C library * * Copyright (c) 2012, Magnus Edenhill * All rights reserved. * * Redistribution and use in source and binary forms, with or without * modification, are permitted provided that the following conditions are met: * * 1. Redistributions of source code must retain the above copyright notice, * this list of conditions and the following disclaimer. * 2. Redistributions in binary form must reproduce the above copyright notice, * this list of conditions and the following disclaimer in the documentation * and/or other materials provided with the distribution. * * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE * POSSIBILITY OF SUCH DAMAGE. */ #pragma once /** * Returns a random (using rand(3)) number between 'low'..'high' (inclusive). */ static inline int rd_jitter (int low, int high) RD_UNUSED; static inline int rd_jitter (int low, int high) { return (low + (rand() % (high+1))); } /** * Shuffles (randomizes) an array using the modern Fisher-Yates algorithm. */ void rd_array_shuffle (void *base, size_t nmemb, size_t entry_size); librdkafka-0.8.3/rdsignal.h000066400000000000000000000036701227620010100155650ustar00rootroot00000000000000/* * librd - Rapid Development C library * * Copyright (c) 2012-2013, Magnus Edenhill * All rights reserved. * * Redistribution and use in source and binary forms, with or without * modification, are permitted provided that the following conditions are met: * * 1. Redistributions of source code must retain the above copyright notice, * this list of conditions and the following disclaimer. * 2. Redistributions in binary form must reproduce the above copyright notice, * this list of conditions and the following disclaimer in the documentation * and/or other materials provided with the distribution. * * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE * POSSIBILITY OF SUCH DAMAGE. */ #pragma once #include #define RD_SIG_ALL -1 #define RD_SIG_END -2 extern sigset_t rd_intr_sigset; extern int rd_intr_blocked; static inline void rd_intr_block (void) RD_UNUSED; static inline void rd_intr_block (void) { if (rd_intr_blocked++) return; sigprocmask(SIG_BLOCK, &rd_intr_sigset, NULL); } static inline void rd_intr_unblock (void) RD_UNUSED; static inline void rd_intr_unblock (void) { assert(rd_intr_blocked > 0); if (--rd_intr_blocked) return; sigprocmask(SIG_UNBLOCK, &rd_intr_sigset, NULL); } librdkafka-0.8.3/rdsysqueue.h000066400000000000000000000244071227620010100161740ustar00rootroot00000000000000/* * librd - Rapid Development C library * * Copyright (c) 2012-2013, Magnus Edenhill * Copyright (c) 2012-2013, Andreas Öman * All rights reserved. * * Redistribution and use in source and binary forms, with or without * modification, are permitted provided that the following conditions are met: * * 1. Redistributions of source code must retain the above copyright notice, * this list of conditions and the following disclaimer. * 2. Redistributions in binary form must reproduce the above copyright notice, * this list of conditions and the following disclaimer in the documentation * and/or other materials provided with the distribution. * * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE * POSSIBILITY OF SUCH DAMAGE. */ /* * * Redistribution and use in source and binary forms, with or without * modification, are permitted provided that the following conditions * are met: * 1. Redistributions of source code must retain the above copyright * notice, this list of conditions and the following disclaimer. * 2. Redistributions in binary form must reproduce the above copyright * notice, this list of conditions and the following disclaimer in the * documentation and/or other materials provided with the distribution. * * THIS SOFTWARE IS PROVIDED BY THE NETBSD FOUNDATION, INC. AND CONTRIBUTORS * ``AS IS'' AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED * TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR * PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE FOUNDATION OR CONTRIBUTORS * BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE * POSSIBILITY OF SUCH DAMAGE. */ #pragma once #include /* * Complete missing LIST-ops */ #ifndef LIST_FOREACH #define LIST_FOREACH(var, head, field) \ for ((var) = ((head)->lh_first); \ (var); \ (var) = ((var)->field.le_next)) #endif #ifndef LIST_EMPTY #define LIST_EMPTY(head) ((head)->lh_first == NULL) #endif #ifndef LIST_FIRST #define LIST_FIRST(head) ((head)->lh_first) #endif #ifndef LIST_NEXT #define LIST_NEXT(elm, field) ((elm)->field.le_next) #endif #ifndef LIST_INSERT_BEFORE #define LIST_INSERT_BEFORE(listelm, elm, field) do { \ (elm)->field.le_prev = (listelm)->field.le_prev; \ (elm)->field.le_next = (listelm); \ *(listelm)->field.le_prev = (elm); \ (listelm)->field.le_prev = &(elm)->field.le_next; \ } while (/*CONSTCOND*/0) #endif /* * Complete missing TAILQ-ops */ #ifndef TAILQ_INSERT_BEFORE #define TAILQ_INSERT_BEFORE(listelm, elm, field) do { \ (elm)->field.tqe_prev = (listelm)->field.tqe_prev; \ (elm)->field.tqe_next = (listelm); \ *(listelm)->field.tqe_prev = (elm); \ (listelm)->field.tqe_prev = &(elm)->field.tqe_next; \ } while (0) #endif #ifndef TAILQ_FOREACH #define TAILQ_FOREACH(var, head, field) \ for ((var) = ((head)->tqh_first); (var); (var) = ((var)->field.tqe_next)) #endif #ifndef TAILQ_FIRST #define TAILQ_FIRST(head) ((head)->tqh_first) #endif #ifndef TAILQ_NEXT #define TAILQ_NEXT(elm, field) ((elm)->field.tqe_next) #endif #ifndef TAILQ_LAST #define TAILQ_LAST(head, headname) \ (*(((struct headname *)((head)->tqh_last))->tqh_last)) #endif #ifndef TAILQ_PREV #define TAILQ_PREV(elm, headname, field) \ (*(((struct headname *)((elm)->field.tqe_prev))->tqh_last)) #endif #ifndef TAILQ_FOREACH_SAFE /* * TAILQ_FOREACH_SAFE() provides a traversal where the current iterated element * may be freed or unlinked. * It does not allow freeing or modifying any other element in the list, * at least not the next element. */ #define TAILQ_FOREACH_SAFE(elm,head,field,tmpelm) \ for ((elm) = TAILQ_FIRST(head) ; \ (elm) && ((tmpelm) = TAILQ_NEXT((elm), field), 1) ; \ (elm) = (tmpelm)) #endif /* * In Mac OS 10.4 and earlier TAILQ_FOREACH_REVERSE was defined * differently, redefined it. */ #ifdef __ENVIRONMENT_MAC_OS_X_VERSION_MIN_REQUIRED__ #if __ENVIRONMENT_MAC_OS_X_VERSION_MIN_REQUIRED__ < 1050 #undef TAILQ_FOREACH_REVERSE #endif #endif #ifndef TAILQ_FOREACH_REVERSE #define TAILQ_FOREACH_REVERSE(var, head, headname, field) \ for ((var) = (*(((struct headname *)((head)->tqh_last))->tqh_last)); \ (var); \ (var) = (*(((struct headname *)((var)->field.tqe_prev))->tqh_last))) #endif /** * Treat the TAILQ as a circular list and return the previous/next entry, * possibly wrapping to the end/beginning. */ #define TAILQ_CIRC_PREV(var, head, headname, field) \ ((var) != TAILQ_FIRST(head) ? \ TAILQ_PREV(var, headname, field) : \ TAILQ_LAST(head, headname)) #define TAILQ_CIRC_NEXT(var, head, headname, field) \ ((var) != TAILQ_LAST(head, headname) ? \ TAILQ_NEXT(var, field) : \ TAILQ_FIRST(head)) /* * Some extra functions for LIST manipulation */ #define LIST_MOVE(newhead, oldhead, field) do { \ if((oldhead)->lh_first) { \ (oldhead)->lh_first->field.le_prev = &(newhead)->lh_first; \ } \ (newhead)->lh_first = (oldhead)->lh_first; \ } while (0) #define LIST_INSERT_SORTED(head, elm, field, cmpfunc) do { \ if(LIST_EMPTY(head)) { \ LIST_INSERT_HEAD(head, elm, field); \ } else { \ typeof(elm) _tmp; \ LIST_FOREACH(_tmp,head,field) { \ if(cmpfunc(elm,_tmp) <= 0) { \ LIST_INSERT_BEFORE(_tmp,elm,field); \ break; \ } \ if(!LIST_NEXT(_tmp,field)) { \ LIST_INSERT_AFTER(_tmp,elm,field); \ break; \ } \ } \ } \ } while(0) #ifndef TAILQ_INSERT_SORTED #define TAILQ_INSERT_SORTED(head, elm, field, cmpfunc) do { \ if(TAILQ_FIRST(head) == NULL) { \ TAILQ_INSERT_HEAD(head, elm, field); \ } else { \ typeof(elm) _tmp; \ TAILQ_FOREACH(_tmp,head,field) { \ if(cmpfunc(elm,_tmp) <= 0) { \ TAILQ_INSERT_BEFORE(_tmp,elm,field); \ break; \ } \ if(!TAILQ_NEXT(_tmp,field)) { \ TAILQ_INSERT_AFTER(head,_tmp,elm,field); \ break; \ } \ } \ } \ } while(0) #endif #define TAILQ_MOVE(newhead, oldhead, field) do { \ if(TAILQ_FIRST(oldhead)) { \ TAILQ_FIRST(oldhead)->field.tqe_prev = &(newhead)->tqh_first; \ } \ (newhead)->tqh_first = (oldhead)->tqh_first; \ (newhead)->tqh_last = (oldhead)->tqh_last; \ } while (/*CONSTCOND*/0) #ifndef TAILQ_CONCAT #define TAILQ_CONCAT(dhead, shead, field) do { \ if (!TAILQ_EMPTY(shead)) { \ *(dhead)->tqh_last = (shead)->tqh_first; \ (shead)->tqh_first->field.tqe_prev = \ (dhead)->tqh_last; \ (dhead)->tqh_last = (shead)->tqh_last; \ TAILQ_INIT((shead)); \ } \ } while (0) #endif #ifndef SIMPLEQ_HEAD #define SIMPLEQ_HEAD(name, type) \ struct name { \ struct type *sqh_first; \ struct type **sqh_last; \ } #endif #ifndef SIMPLEQ_ENTRY #define SIMPLEQ_ENTRY(type) \ struct { \ struct type *sqe_next; \ } #endif #ifndef SIMPLEQ_FIRST #define SIMPLEQ_FIRST(head) ((head)->sqh_first) #endif #ifndef SIMPLEQ_REMOVE_HEAD #define SIMPLEQ_REMOVE_HEAD(head, field) do { \ if (((head)->sqh_first = (head)->sqh_first->field.sqe_next) == NULL) \ (head)->sqh_last = &(head)->sqh_first; \ } while (0) #endif #ifndef SIMPLEQ_INSERT_TAIL #define SIMPLEQ_INSERT_TAIL(head, elm, field) do { \ (elm)->field.sqe_next = NULL; \ *(head)->sqh_last = (elm); \ (head)->sqh_last = &(elm)->field.sqe_next; \ } while (0) #endif #ifndef SIMPLEQ_INIT #define SIMPLEQ_INIT(head) do { \ (head)->sqh_first = NULL; \ (head)->sqh_last = &(head)->sqh_first; \ } while (0) #endif #ifndef SIMPLEQ_INSERT_HEAD #define SIMPLEQ_INSERT_HEAD(head, elm, field) do { \ if (((elm)->field.sqe_next = (head)->sqh_first) == NULL) \ (head)->sqh_last = &(elm)->field.sqe_next; \ (head)->sqh_first = (elm); \ } while (0) #endif #ifndef SIMPLEQ_FOREACH #define SIMPLEQ_FOREACH(var, head, field) \ for((var) = SIMPLEQ_FIRST(head); \ (var) != SIMPLEQ_END(head); \ (var) = SIMPLEQ_NEXT(var, field)) #endif #ifndef SIMPLEQ_INSERT_AFTER #define SIMPLEQ_INSERT_AFTER(head, listelm, elm, field) do { \ if (((elm)->field.sqe_next = (listelm)->field.sqe_next) == NULL) \ (head)->sqh_last = &(elm)->field.sqe_next; \ (listelm)->field.sqe_next = (elm); \ } while (0) #endif #ifndef SIMPLEQ_END #define SIMPLEQ_END(head) NULL #endif #ifndef SIMPLEQ_NEXT #define SIMPLEQ_NEXT(elm, field) ((elm)->field.sqe_next) #endif #ifndef SIMPLEQ_HEAD_INITIALIZER #define SIMPLEQ_HEAD_INITIALIZER(head) \ { NULL, &(head).sqh_first } #endif #ifndef SIMPLEQ_EMPTY #define SIMPLEQ_EMPTY(head) (SIMPLEQ_FIRST(head) == SIMPLEQ_END(head)) #endif librdkafka-0.8.3/rdthread.c000066400000000000000000000112631227620010100155470ustar00rootroot00000000000000/* * librd - Rapid Development C library * * Copyright (c) 2012-2013, Magnus Edenhill * All rights reserved. * * Redistribution and use in source and binary forms, with or without * modification, are permitted provided that the following conditions are met: * * 1. Redistributions of source code must retain the above copyright notice, * this list of conditions and the following disclaimer. * 2. Redistributions in binary form must reproduce the above copyright notice, * this list of conditions and the following disclaimer in the documentation * and/or other materials provided with the distribution. * * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE * POSSIBILITY OF SUCH DAMAGE. */ #include "rd.h" #include "rdthread.h" #include "rdqueue.h" #include "rdevent.h" #include "rdlog.h" #ifdef __linux__ #include #endif #include rd_thread_t *rd_mainthread; __thread rd_thread_t *rd_currthread; void rd_thread_init (void) { pthread_t thr = pthread_self(); rd_mainthread = rd_thread_create0("main", &thr); rd_currthread = rd_mainthread; } int rd_thread_poll (int timeout_ms) { rd_fifoq_elm_t *rfqe; int cnt = 0; int nowait = timeout_ms == 0; while ((rfqe = rd_fifoq_pop0(&rd_currthread->rdt_eventq, nowait, timeout_ms))) { rd_thread_event_t *rte = rfqe->rfqe_ptr; rd_thread_event_call(rte); rd_fifoq_elm_release(&rd_currthread->rdt_eventq, rfqe); cnt++; } return cnt; } static void rd_thread_destroy (rd_thread_t *rdt) { assert(rdt->rdt_state != RD_THREAD_S_RUNNING); if (rdt->rdt_name) free(rdt->rdt_name); rd_fifoq_destroy(&rdt->rdt_eventq); free(rdt); } void rd_thread_cleanup (void) { } void rd_thread_dispatch (void) { while (rd_currthread->rdt_state == RD_THREAD_S_RUNNING) { /* FIXME: Proper conding for all thread inputs. */ rd_thread_poll(100); } rd_thread_cleanup(); } static void *rd_thread_start_routine (void *arg) { rd_thread_t *rdt = arg; void *ret; /* By default with block the user-defined signals. */ rd_thread_sigmask(SIG_BLOCK, SIGUSR1, SIGUSR2, RD_SIG_END); rd_currthread = rdt; ret = rdt->rdt_start(rdt->rdt_start_arg); rd_thread_cleanup(); rd_thread_destroy(rdt); return ret; } rd_thread_t *rd_thread_create0 (const char *name, pthread_t *pthread) { rd_thread_t *rdt; rdt = calloc(1, sizeof(*rdt)); if (name) rdt->rdt_name = strdup(name); rdt->rdt_state = RD_THREAD_S_RUNNING; rd_fifoq_init(&rdt->rdt_eventq); if (pthread) rdt->rdt_thread = *pthread; return rdt; } int rd_thread_create (rd_thread_t **rdt, const char *name, const pthread_attr_t *attr, void *(*start_routine)(void*), void *arg) { rd_thread_t *rdt0; rdt0 = rd_thread_create0(name, NULL); rdt0->rdt_start = start_routine; rdt0->rdt_start_arg = arg; if (rdt) *rdt = rdt0; /* FIXME: We should block all signals until pthread_create returns. */ if (pthread_create(&rdt0->rdt_thread, attr, rd_thread_start_routine, rdt0)) { int errno_save = errno; rd_thread_destroy(rdt0); if (rdt) *rdt = NULL; errno = errno_save; return -1; } #ifdef PR_SET_NAME prctl(PR_SET_NAME, (char *)rdt0->rdt_name, 0, 0, 0); #endif return 0; } int rd_threads_create (const char *nameprefix, int threadcount, const pthread_attr_t *attr, void *(*start_routine)(void*), void *arg) { int i; char *name = alloca(strlen(nameprefix) + 4); int failed = 0; if (threadcount >= 1000) { errno = E2BIG; return -1; } for (i = 0 ; i < threadcount ; i++) { sprintf(name, "%s%i", nameprefix, i); if (!rd_thread_create(NULL, name, attr, start_routine, arg)) failed++; } if (failed == threadcount) return -1; return threadcount - failed; } int rd_thread_sigmask (int how, ...) { va_list ap; sigset_t set; int sig; sigemptyset(&set); va_start(ap, how); while ((sig = va_arg(ap, int)) != RD_SIG_END) { if (sig == RD_SIG_ALL) sigfillset(&set); else sigaddset(&set, sig); } va_end(ap); return pthread_sigmask(how, &set, NULL); } librdkafka-0.8.3/rdthread.h000066400000000000000000000153311227620010100155540ustar00rootroot00000000000000/* * librd - Rapid Development C library * * Copyright (c) 2012-2013, Magnus Edenhill * All rights reserved. * * Redistribution and use in source and binary forms, with or without * modification, are permitted provided that the following conditions are met: * * 1. Redistributions of source code must retain the above copyright notice, * this list of conditions and the following disclaimer. * 2. Redistributions in binary form must reproduce the above copyright notice, * this list of conditions and the following disclaimer in the documentation * and/or other materials provided with the distribution. * * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE * POSSIBILITY OF SUCH DAMAGE. */ #pragma once #include #include "rdqueue.h" #include "rdtime.h" #include "rdsignal.h" typedef struct rd_thread_s { pthread_t rdt_thread; char *rdt_name; void *(*rdt_start)(void *); void *rdt_start_arg; enum { RD_THREAD_S_NONE, RD_THREAD_S_RUNNING, RD_THREAD_S_EXITING, RD_THREAD_S_DEAD, } rdt_state; rd_fifoq_t rdt_eventq; } rd_thread_t; extern rd_thread_t *rd_mainthread; extern __thread rd_thread_t *rd_currthread; /** * Slow controlled thread exit through rd_thread_dispatch(). * Called by exiting thread. */ #define rd_thread_exit() do { \ rd_currthread_get(); \ assert(rd_currthread->rdt_state == RD_THREAD_S_RUNNING || \ rd_currthread->rdt_state == RD_THREAD_S_EXITING); \ (rd_currthread->rdt_state = RD_THREAD_S_EXITING); \ } while (0) /** * Slow controlled thread exit through rd_thread_dispatch(). * Called by other thread. */ #define rd_thread_kill(rdt) do { \ (rdt)->rdt_state = RD_THREAD_S_EXITING; \ } while (0) static inline int rd_thread_kill_join (rd_thread_t *rdt, void **retval) RD_UNUSED; static inline int rd_thread_kill_join (rd_thread_t *rdt, void **retval) { pthread_t pthread; pthread = rdt->rdt_thread; rd_thread_kill(rdt); return pthread_join(pthread, retval); } /** * Clean up / free resources allocated to the current thread. * Use prior to thread destruction. * * Locality: the thread itself */ void rd_thread_cleanup (void); rd_thread_t *rd_thread_create0 (const char *name, pthread_t *pthread); /** * Creates and starts a new thread and returns its rd_thread_t handle. * The new thread handle is assigned to '*rdt' (if 'rdt' is non-null). * * Same semantics as pthread_create() with the following exceptions: * - User-defined signals are by default blocked by the new thread: * SIGUSR1, SIGUSR2. * Use rd_thread_sigmask() to unblock. */ int rd_thread_create (rd_thread_t **rdt, const char *name, const pthread_attr_t *attr, void *(*start_routine)(void*), void *arg); /** * Wrapper around rd_thread_create() for creating multiple ('threadcount') * threads. * * Useful for creating a set of worker threads or similar. */ int rd_threads_create (const char *nameprefix, int threadcount, const pthread_attr_t *attr, void *(*start_routine)(void*), void *arg); /** * This creates the rd_currthread handle for threads that where not * created through rd_thread_create(). */ static inline rd_thread_t *rd_currthread_get (void) RD_UNUSED; static inline rd_thread_t *rd_currthread_get (void) { if (unlikely(!rd_currthread)) { pthread_t thr = pthread_self(); char thrname[16]; snprintf(thrname, sizeof(thrname), "%p", (void *)thr); rd_currthread = rd_thread_create0(thrname, &thr); } return rd_currthread; } /** * va-arg wrapper for pthread_sigmask(). * The va-arg-list must be terminated with RD_SIG_END. * RD_SIG_ALL means all signals. * * Example: * rd_thread_sigmask(SIG_BLOCK, RD_SIG_ALL, RD_SIG_END); * or * rd_thread_sigmask(SIG_SETMASK, SIGUSR1, SIGUSR2, SIGIO, RD_SIG_END); * * Returns the return value from pthread_sigmask(). */ int rd_thread_sigmask (int how, ...); #define rd_assert_inthread(rdt) assert(rd_currthread == (rdt)) #define rd_assert_inpthread(pthread) assert(pthread_self() == (pthread)) #define RD_MUTEX_INITIALIZER PTHREAD_MUTEX_INITIALIZER #define rd_mutex_init(MTX) pthread_mutex_init(MTX, NULL) #define rd_mutex_destroy(MTX) pthread_mutex_destroy(MTX) #define rd_mutex_lock(MTX) pthread_mutex_lock(MTX) #define rd_mutex_unlock(MTX) pthread_mutex_unlock(MTX) #define RD_MUTEX_LOCKED(MTX,CODE...) do { \ rd_mutex_lock(MTX); \ CODE; \ rd_mutex_unlock(MTX); \ } while (0) #define rd_rwlock_init(RWL) pthread_rwlock_init(RWL,NULL) #define rd_rwlock_destroy(RWL) pthread_rwlock_destroy(RWL) #define rd_rwlock_rdlock(RWL) pthread_rwlock_rdlock(RWL) #define rd_rwlock_wrlock(RWL) pthread_rwlock_wrlock(RWL) #define rd_rwlock_unlock(RWL) pthread_rwlock_unlock(RWL) #define RD_RWLOCK_RDLOCKED(RWL,CODE...) do { \ rd_rwlock_rdlock(RWL); \ CODE; \ rd_rwlock_unlock(RWL); \ } while (0) #define RD_RWLOCK_WRLOCKED(RWL,CODE...) do { \ rd_rwlock_wrlock(RWL); \ CODE; \ rd_rwlock_unlock(RWL); \ } while (0) #define rd_cond_init(COND,ATTR) pthread_cond_init(COND,ATTR) #define RD_COND_INITIALIZER PTHREAD_COND_INITIALIZER #define rd_cond_signal(COND) pthread_cond_signal(COND) #define rd_cond_timedwait(COND,MTX,TS) pthread_cond_timedwait(COND,MTX,TS) #define rd_cond_wait(COND,MTX) pthread_cond_wait(COND,MTX) /** * Wrapper for pthread_cond_timedwait() that makes it simpler to use * for delta timeouts. * `timeout_ms' is the delta timeout in milliseconds. */ static int rd_cond_timedwait_ms (rd_cond_t *cond, rd_mutex_t *mutex, int timeout_ms) RD_UNUSED; static int rd_cond_timedwait_ms (rd_cond_t *cond, rd_mutex_t *mutex, int timeout_ms) { struct timeval tv; struct timespec ts; gettimeofday(&tv, NULL); TIMEVAL_TO_TIMESPEC(&tv, &ts); ts.tv_sec += timeout_ms / 1000; ts.tv_nsec += (timeout_ms % 1000) * 1000000; if (ts.tv_nsec > 1000000000) { ts.tv_sec++; ts.tv_nsec -= 1000000000; } return rd_cond_timedwait(cond, mutex, &ts); } int rd_thread_poll (int timeout_ms); void rd_thread_dispatch (void); void rd_thread_init (void); librdkafka-0.8.3/rdtime.h000066400000000000000000000054711227620010100152470ustar00rootroot00000000000000/* * librd - Rapid Development C library * * Copyright (c) 2012, Magnus Edenhill * All rights reserved. * * Redistribution and use in source and binary forms, with or without * modification, are permitted provided that the following conditions are met: * * 1. Redistributions of source code must retain the above copyright notice, * this list of conditions and the following disclaimer. * 2. Redistributions in binary form must reproduce the above copyright notice, * this list of conditions and the following disclaimer in the documentation * and/or other materials provided with the distribution. * * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE * POSSIBILITY OF SUCH DAMAGE. */ #pragma once #ifndef TIMEVAL_TO_TIMESPEC #define TIMEVAL_TO_TIMESPEC(tv,ts) do { \ (ts)->tv_sec = (tv)->tv_sec; \ (ts)->tv_nsec = (tv)->tv_usec * 1000; \ } while (0) #define TIMESPEC_TO_TIMEVAL(tv, ts) do { \ (tv)->tv_sec = (ts)->tv_sec; \ (tv)->tv_usec = (ts)->tv_nsec / 1000; \ } while (0) #endif #define TIMESPEC_TO_TS(ts) \ (((rd_ts_t)(ts)->tv_sec * 1000000LLU) + ((ts)->tv_nsec / 1000)) #define TS_TO_TIMESPEC(ts,tsx) do { \ (ts)->tv_sec = (tsx) / 1000000; \ (ts)->tv_nsec = ((tsx) % 1000000) * 1000; \ if ((ts)->tv_nsec > 1000000000LLU) { \ (ts)->tv_sec++; \ (ts)->tv_nsec -= 1000000000LLU; \ } \ } while (0) #define TIMESPEC_CLEAR(ts) ((ts)->tv_sec = (ts)->tv_nsec = 0LLU) static inline rd_ts_t rd_clock (void) RD_UNUSED; static inline rd_ts_t rd_clock (void) { #ifdef __APPLE__ /* No monotonic clock on Darwin */ struct timeval tv; gettimeofday(&tv, NULL); return ((rd_ts_t)tv.tv_sec * 1000000LLU) + (rd_ts_t)tv.tv_usec; #else struct timespec ts; clock_gettime(CLOCK_MONOTONIC, &ts); return ((rd_ts_t)ts.tv_sec * 1000000LLU) + ((rd_ts_t)ts.tv_nsec / 1000LLU); #endif } /** * Thread-safe version of ctime() that strips the trailing newline. */ static inline const char *rd_ctime (const time_t *t) RD_UNUSED; static inline const char *rd_ctime (const time_t *t) { static __thread char ret[27]; ctime_r(t, ret); ret[25] = '\0'; return ret; } librdkafka-0.8.3/rdtypes.h000066400000000000000000000032111227620010100154430ustar00rootroot00000000000000/* * librd - Rapid Development C library * * Copyright (c) 2012, Magnus Edenhill * All rights reserved. * * Redistribution and use in source and binary forms, with or without * modification, are permitted provided that the following conditions are met: * * 1. Redistributions of source code must retain the above copyright notice, * this list of conditions and the following disclaimer. * 2. Redistributions in binary form must reproduce the above copyright notice, * this list of conditions and the following disclaimer in the documentation * and/or other materials provided with the distribution. * * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE * POSSIBILITY OF SUCH DAMAGE. */ #pragma once #include /* * Fundamental types */ typedef pthread_mutex_t rd_mutex_t; typedef pthread_rwlock_t rd_rwlock_t; typedef pthread_cond_t rd_cond_t; /* Timestamp (microseconds) */ typedef uint64_t rd_ts_t; librdkafka-0.8.3/snappy.c000066400000000000000000001304251227620010100152660ustar00rootroot00000000000000/* * C port of the snappy compressor from Google. * This is a very fast compressor with comparable compression to lzo. * Works best on 64bit little-endian, but should be good on others too. * Ported by Andi Kleen. * Uptodate with snappy 1.1.0 */ /* * Copyright 2005 Google Inc. All Rights Reserved. * * Redistribution and use in source and binary forms, with or without * modification, are permitted provided that the following conditions are * met: * * * Redistributions of source code must retain the above copyright * notice, this list of conditions and the following disclaimer. * * Redistributions in binary form must reproduce the above * copyright notice, this list of conditions and the following disclaimer * in the documentation and/or other materials provided with the * distribution. * * Neither the name of Google Inc. nor the names of its * contributors may be used to endorse or promote products derived from * this software without specific prior written permission. * * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. */ #ifdef __KERNEL__ #include #ifdef SG #include #endif #include #include #include #include #include #include #else #include "snappy.h" #include "snappy_compat.h" #endif #define CRASH_UNLESS(x) BUG_ON(!(x)) #define CHECK(cond) CRASH_UNLESS(cond) #define CHECK_LE(a, b) CRASH_UNLESS((a) <= (b)) #define CHECK_GE(a, b) CRASH_UNLESS((a) >= (b)) #define CHECK_EQ(a, b) CRASH_UNLESS((a) == (b)) #define CHECK_NE(a, b) CRASH_UNLESS((a) != (b)) #define CHECK_LT(a, b) CRASH_UNLESS((a) < (b)) #define CHECK_GT(a, b) CRASH_UNLESS((a) > (b)) #define UNALIGNED_LOAD16(_p) get_unaligned((u16 *)(_p)) #define UNALIGNED_LOAD32(_p) get_unaligned((u32 *)(_p)) #define UNALIGNED_LOAD64(_p) get_unaligned64((u64 *)(_p)) #define UNALIGNED_STORE16(_p, _val) put_unaligned(_val, (u16 *)(_p)) #define UNALIGNED_STORE32(_p, _val) put_unaligned(_val, (u32 *)(_p)) #define UNALIGNED_STORE64(_p, _val) put_unaligned64(_val, (u64 *)(_p)) /* * This can be more efficient than UNALIGNED_LOAD64 + UNALIGNED_STORE64 * on some platforms, in particular ARM. */ static inline void unaligned_copy64(const void *src, void *dst) { if (sizeof(void *) == 8) { UNALIGNED_STORE64(dst, UNALIGNED_LOAD64(src)); } else { const char *src_char = (const char *)(src); char *dst_char = (char *)(dst); UNALIGNED_STORE32(dst_char, UNALIGNED_LOAD32(src_char)); UNALIGNED_STORE32(dst_char + 4, UNALIGNED_LOAD32(src_char + 4)); } } #ifdef NDEBUG #define DCHECK(cond) do {} while(0) #define DCHECK_LE(a, b) do {} while(0) #define DCHECK_GE(a, b) do {} while(0) #define DCHECK_EQ(a, b) do {} while(0) #define DCHECK_NE(a, b) do {} while(0) #define DCHECK_LT(a, b) do {} while(0) #define DCHECK_GT(a, b) do {} while(0) #else #define DCHECK(cond) CHECK(cond) #define DCHECK_LE(a, b) CHECK_LE(a, b) #define DCHECK_GE(a, b) CHECK_GE(a, b) #define DCHECK_EQ(a, b) CHECK_EQ(a, b) #define DCHECK_NE(a, b) CHECK_NE(a, b) #define DCHECK_LT(a, b) CHECK_LT(a, b) #define DCHECK_GT(a, b) CHECK_GT(a, b) #endif static inline bool is_little_endian(void) { #ifdef __LITTLE_ENDIAN__ return true; #endif return false; } static inline int log2_floor(u32 n) { return n == 0 ? -1 : 31 ^ __builtin_clz(n); } static inline __attribute__((unused)) int find_lsb_set_non_zero(u32 n) { return __builtin_ctz(n); } static inline int find_lsb_set_non_zero64(u64 n) { return __builtin_ctzll(n); } #define kmax32 5 /* * Attempts to parse a varint32 from a prefix of the bytes in [ptr,limit-1]. * Never reads a character at or beyond limit. If a valid/terminated varint32 * was found in the range, stores it in *OUTPUT and returns a pointer just * past the last byte of the varint32. Else returns NULL. On success, * "result <= limit". */ static inline const char *varint_parse32_with_limit(const char *p, const char *l, u32 * OUTPUT) { const unsigned char *ptr = (const unsigned char *)(p); const unsigned char *limit = (const unsigned char *)(l); u32 b, result; if (ptr >= limit) return NULL; b = *(ptr++); result = b & 127; if (b < 128) goto done; if (ptr >= limit) return NULL; b = *(ptr++); result |= (b & 127) << 7; if (b < 128) goto done; if (ptr >= limit) return NULL; b = *(ptr++); result |= (b & 127) << 14; if (b < 128) goto done; if (ptr >= limit) return NULL; b = *(ptr++); result |= (b & 127) << 21; if (b < 128) goto done; if (ptr >= limit) return NULL; b = *(ptr++); result |= (b & 127) << 28; if (b < 16) goto done; return NULL; /* Value is too long to be a varint32 */ done: *OUTPUT = result; return (const char *)(ptr); } /* * REQUIRES "ptr" points to a buffer of length sufficient to hold "v". * EFFECTS Encodes "v" into "ptr" and returns a pointer to the * byte just past the last encoded byte. */ static inline char *varint_encode32(char *sptr, u32 v) { /* Operate on characters as unsigneds */ unsigned char *ptr = (unsigned char *)(sptr); static const int B = 128; if (v < (1 << 7)) { *(ptr++) = v; } else if (v < (1 << 14)) { *(ptr++) = v | B; *(ptr++) = v >> 7; } else if (v < (1 << 21)) { *(ptr++) = v | B; *(ptr++) = (v >> 7) | B; *(ptr++) = v >> 14; } else if (v < (1 << 28)) { *(ptr++) = v | B; *(ptr++) = (v >> 7) | B; *(ptr++) = (v >> 14) | B; *(ptr++) = v >> 21; } else { *(ptr++) = v | B; *(ptr++) = (v >> 7) | B; *(ptr++) = (v >> 14) | B; *(ptr++) = (v >> 21) | B; *(ptr++) = v >> 28; } return (char *)(ptr); } #ifdef SG struct source { struct iovec *iov; int iovlen; int curvec; int curoff; size_t total; }; /* Only valid at beginning when nothing is consumed */ static inline int available(struct source *s) { return s->total; } static inline const char *peek(struct source *s, size_t *len) { if (likely(s->curvec < s->iovlen)) { struct iovec *iv = &s->iov[s->curvec]; if (s->curoff < iv->iov_len) { *len = iv->iov_len - s->curoff; return (char *)iv->iov_base + s->curoff; } } *len = 0; return NULL; } static inline void skip(struct source *s, size_t n) { struct iovec *iv = &s->iov[s->curvec]; s->curoff += n; DCHECK_LE(s->curoff, iv->iov_len); if (s->curoff >= iv->iov_len && s->curvec + 1 < s->iovlen) { s->curoff = 0; s->curvec++; } } struct sink { struct iovec *iov; int iovlen; unsigned curvec; unsigned curoff; unsigned written; }; static inline void append(struct sink *s, const char *data, size_t n) { struct iovec *iov = &s->iov[s->curvec]; char *dst = (char *)iov->iov_base + s->curoff; size_t nlen = min_t(size_t, iov->iov_len - s->curoff, n); if (data != dst) memcpy(dst, data, nlen); s->written += n; s->curoff += nlen; while ((n -= nlen) > 0) { data += nlen; s->curvec++; DCHECK_LT(s->curvec, s->iovlen); iov++; nlen = min_t(size_t, iov->iov_len, n); memcpy(iov->iov_base, data, nlen); s->curoff = nlen; } } static inline void *sink_peek(struct sink *s, size_t n) { struct iovec *iov = &s->iov[s->curvec]; if (s->curvec < iov->iov_len && iov->iov_len - s->curoff >= n) return (char *)iov->iov_base + s->curoff; return NULL; } #else struct source { const char *ptr; size_t left; }; static inline int available(struct source *s) { return s->left; } static inline const char *peek(struct source *s, size_t * len) { *len = s->left; return s->ptr; } static inline void skip(struct source *s, size_t n) { s->left -= n; s->ptr += n; } struct sink { char *dest; }; static inline void append(struct sink *s, const char *data, size_t n) { if (data != s->dest) memcpy(s->dest, data, n); s->dest += n; } #define sink_peek(s, n) sink_peek_no_sg(s) static inline void *sink_peek_no_sg(const struct sink *s) { return s->dest; } #endif struct writer { char *base; char *op; char *op_limit; }; /* Called before decompression */ static inline void writer_set_expected_length(struct writer *w, size_t len) { w->op_limit = w->op + len; } /* Called after decompression */ static inline bool writer_check_length(struct writer *w) { return w->op == w->op_limit; } /* * Copy "len" bytes from "src" to "op", one byte at a time. Used for * handling COPY operations where the input and output regions may * overlap. For example, suppose: * src == "ab" * op == src + 2 * len == 20 * After IncrementalCopy(src, op, len), the result will have * eleven copies of "ab" * ababababababababababab * Note that this does not match the semantics of either memcpy() * or memmove(). */ static inline void incremental_copy(const char *src, char *op, ssize_t len) { DCHECK_GT(len, 0); do { *op++ = *src++; } while (--len > 0); } /* * Equivalent to IncrementalCopy except that it can write up to ten extra * bytes after the end of the copy, and that it is faster. * * The main part of this loop is a simple copy of eight bytes at a time until * we've copied (at least) the requested amount of bytes. However, if op and * src are less than eight bytes apart (indicating a repeating pattern of * length < 8), we first need to expand the pattern in order to get the correct * results. For instance, if the buffer looks like this, with the eight-byte * and patterns marked as intervals: * * abxxxxxxxxxxxx * [------] src * [------] op * * a single eight-byte copy from to will repeat the pattern once, * after which we can move two bytes without moving : * * ababxxxxxxxxxx * [------] src * [------] op * * and repeat the exercise until the two no longer overlap. * * This allows us to do very well in the special case of one single byte * repeated many times, without taking a big hit for more general cases. * * The worst case of extra writing past the end of the match occurs when * op - src == 1 and len == 1; the last copy will read from byte positions * [0..7] and write to [4..11], whereas it was only supposed to write to * position 1. Thus, ten excess bytes. */ #define kmax_increment_copy_overflow 10 static inline void incremental_copy_fast_path(const char *src, char *op, ssize_t len) { while (op - src < 8) { unaligned_copy64(src, op); len -= op - src; op += op - src; } while (len > 0) { unaligned_copy64(src, op); src += 8; op += 8; len -= 8; } } static inline bool writer_append_from_self(struct writer *w, u32 offset, u32 len) { char *const op = w->op; CHECK_LE(op, w->op_limit); const u32 space_left = w->op_limit - op; if (op - w->base <= offset - 1u) /* -1u catches offset==0 */ return false; if (len <= 16 && offset >= 8 && space_left >= 16) { /* Fast path, used for the majority (70-80%) of dynamic * invocations. */ unaligned_copy64(op - offset, op); unaligned_copy64(op - offset + 8, op + 8); } else { if (space_left >= len + kmax_increment_copy_overflow) { incremental_copy_fast_path(op - offset, op, len); } else { if (space_left < len) { return false; } incremental_copy(op - offset, op, len); } } w->op = op + len; return true; } static inline bool writer_append(struct writer *w, const char *ip, u32 len) { char *const op = w->op; CHECK_LE(op, w->op_limit); const u32 space_left = w->op_limit - op; if (space_left < len) return false; memcpy(op, ip, len); w->op = op + len; return true; } static inline bool writer_try_fast_append(struct writer *w, const char *ip, u32 available_bytes, u32 len) { char *const op = w->op; const int space_left = w->op_limit - op; if (len <= 16 && available_bytes >= 16 && space_left >= 16) { /* Fast path, used for the majority (~95%) of invocations */ unaligned_copy64(ip, op); unaligned_copy64(ip + 8, op + 8); w->op = op + len; return true; } return false; } /* * Any hash function will produce a valid compressed bitstream, but a good * hash function reduces the number of collisions and thus yields better * compression for compressible input, and more speed for incompressible * input. Of course, it doesn't hurt if the hash function is reasonably fast * either, as it gets called a lot. */ static inline u32 hash_bytes(u32 bytes, int shift) { u32 kmul = 0x1e35a7bd; return (bytes * kmul) >> shift; } static inline u32 hash(const char *p, int shift) { return hash_bytes(UNALIGNED_LOAD32(p), shift); } /* * Compressed data can be defined as: * compressed := item* literal* * item := literal* copy * * The trailing literal sequence has a space blowup of at most 62/60 * since a literal of length 60 needs one tag byte + one extra byte * for length information. * * Item blowup is trickier to measure. Suppose the "copy" op copies * 4 bytes of data. Because of a special check in the encoding code, * we produce a 4-byte copy only if the offset is < 65536. Therefore * the copy op takes 3 bytes to encode, and this type of item leads * to at most the 62/60 blowup for representing literals. * * Suppose the "copy" op copies 5 bytes of data. If the offset is big * enough, it will take 5 bytes to encode the copy op. Therefore the * worst case here is a one-byte literal followed by a five-byte copy. * I.e., 6 bytes of input turn into 7 bytes of "compressed" data. * * This last factor dominates the blowup, so the final estimate is: */ size_t snappy_max_compressed_length(size_t source_len) { return 32 + source_len + source_len / 6; } EXPORT_SYMBOL(snappy_max_compressed_length); enum { LITERAL = 0, COPY_1_BYTE_OFFSET = 1, /* 3 bit length + 3 bits of offset in opcode */ COPY_2_BYTE_OFFSET = 2, COPY_4_BYTE_OFFSET = 3 }; static inline char *emit_literal(char *op, const char *literal, int len, bool allow_fast_path) { int n = len - 1; /* Zero-length literals are disallowed */ if (n < 60) { /* Fits in tag byte */ *op++ = LITERAL | (n << 2); /* * The vast majority of copies are below 16 bytes, for which a * call to memcpy is overkill. This fast path can sometimes * copy up to 15 bytes too much, but that is okay in the * main loop, since we have a bit to go on for both sides: * * - The input will always have kInputMarginBytes = 15 extra * available bytes, as long as we're in the main loop, and * if not, allow_fast_path = false. * - The output will always have 32 spare bytes (see * MaxCompressedLength). */ if (allow_fast_path && len <= 16) { unaligned_copy64(literal, op); unaligned_copy64(literal + 8, op + 8); return op + len; } } else { /* Encode in upcoming bytes */ char *base = op; int count = 0; op++; while (n > 0) { *op++ = n & 0xff; n >>= 8; count++; } DCHECK(count >= 1); DCHECK(count <= 4); *base = LITERAL | ((59 + count) << 2); } memcpy(op, literal, len); return op + len; } static inline char *emit_copy_less_than64(char *op, int offset, int len) { DCHECK_LE(len, 64); DCHECK_GE(len, 4); DCHECK_LT(offset, 65536); if ((len < 12) && (offset < 2048)) { int len_minus_4 = len - 4; DCHECK(len_minus_4 < 8); /* Must fit in 3 bits */ *op++ = COPY_1_BYTE_OFFSET + ((len_minus_4) << 2) + ((offset >> 8) << 5); *op++ = offset & 0xff; } else { *op++ = COPY_2_BYTE_OFFSET + ((len - 1) << 2); put_unaligned_le16(offset, op); op += 2; } return op; } static inline char *emit_copy(char *op, int offset, int len) { /* * Emit 64 byte copies but make sure to keep at least four bytes * reserved */ while (len >= 68) { op = emit_copy_less_than64(op, offset, 64); len -= 64; } /* * Emit an extra 60 byte copy if have too much data to fit in * one copy */ if (len > 64) { op = emit_copy_less_than64(op, offset, 60); len -= 60; } /* Emit remainder */ op = emit_copy_less_than64(op, offset, len); return op; } /** * snappy_uncompressed_length - return length of uncompressed output. * @start: compressed buffer * @n: length of compressed buffer. * @result: Write the length of the uncompressed output here. * * Returns true when successfull, otherwise false. */ bool snappy_uncompressed_length(const char *start, size_t n, size_t * result) { u32 v = 0; const char *limit = start + n; if (varint_parse32_with_limit(start, limit, &v) != NULL) { *result = v; return true; } else { return false; } } EXPORT_SYMBOL(snappy_uncompressed_length); /* * The size of a compression block. Note that many parts of the compression * code assumes that kBlockSize <= 65536; in particular, the hash table * can only store 16-bit offsets, and EmitCopy() also assumes the offset * is 65535 bytes or less. Note also that if you change this, it will * affect the framing format * Note that there might be older data around that is compressed with larger * block sizes, so the decompression code should not rely on the * non-existence of long backreferences. */ #define kblock_log 16 #define kblock_size (1 << kblock_log) /* * This value could be halfed or quartered to save memory * at the cost of slightly worse compression. */ #define kmax_hash_table_bits 14 #define kmax_hash_table_size (1U << kmax_hash_table_bits) /* * Use smaller hash table when input.size() is smaller, since we * fill the table, incurring O(hash table size) overhead for * compression, and if the input is short, we won't need that * many hash table entries anyway. */ static u16 *get_hash_table(struct snappy_env *env, size_t input_size, int *table_size) { unsigned htsize = 256; DCHECK(kmax_hash_table_size >= 256); while (htsize < kmax_hash_table_size && htsize < input_size) htsize <<= 1; CHECK_EQ(0, htsize & (htsize - 1)); CHECK_LE(htsize, kmax_hash_table_size); u16 *table; table = env->hash_table; *table_size = htsize; memset(table, 0, htsize * sizeof(*table)); return table; } /* * Return the largest n such that * * s1[0,n-1] == s2[0,n-1] * and n <= (s2_limit - s2). * * Does not read *s2_limit or beyond. * Does not read *(s1 + (s2_limit - s2)) or beyond. * Requires that s2_limit >= s2. * * Separate implementation for x86_64, for speed. Uses the fact that * x86_64 is little endian. */ #if defined(__LITTLE_ENDIAN__) && BITS_PER_LONG == 64 static inline int find_match_length(const char *s1, const char *s2, const char *s2_limit) { int matched = 0; DCHECK_GE(s2_limit, s2); /* * Find out how long the match is. We loop over the data 64 bits at a * time until we find a 64-bit block that doesn't match; then we find * the first non-matching bit and use that to calculate the total * length of the match. */ while (likely(s2 <= s2_limit - 8)) { if (unlikely (UNALIGNED_LOAD64(s2) == UNALIGNED_LOAD64(s1 + matched))) { s2 += 8; matched += 8; } else { /* * On current (mid-2008) Opteron models there * is a 3% more efficient code sequence to * find the first non-matching byte. However, * what follows is ~10% better on Intel Core 2 * and newer, and we expect AMD's bsf * instruction to improve. */ u64 x = UNALIGNED_LOAD64(s2) ^ UNALIGNED_LOAD64(s1 + matched); int matching_bits = find_lsb_set_non_zero64(x); matched += matching_bits >> 3; return matched; } } while (likely(s2 < s2_limit)) { if (likely(s1[matched] == *s2)) { ++s2; ++matched; } else { return matched; } } return matched; } #else static inline int find_match_length(const char *s1, const char *s2, const char *s2_limit) { /* Implementation based on the x86-64 version, above. */ DCHECK_GE(s2_limit, s2); int matched = 0; while (s2 <= s2_limit - 4 && UNALIGNED_LOAD32(s2) == UNALIGNED_LOAD32(s1 + matched)) { s2 += 4; matched += 4; } if (is_little_endian() && s2 <= s2_limit - 4) { u32 x = UNALIGNED_LOAD32(s2) ^ UNALIGNED_LOAD32(s1 + matched); int matching_bits = find_lsb_set_non_zero(x); matched += matching_bits >> 3; } else { while ((s2 < s2_limit) && (s1[matched] == *s2)) { ++s2; ++matched; } } return matched; } #endif /* * For 0 <= offset <= 4, GetU32AtOffset(GetEightBytesAt(p), offset) will * equal UNALIGNED_LOAD32(p + offset). Motivation: On x86-64 hardware we have * empirically found that overlapping loads such as * UNALIGNED_LOAD32(p) ... UNALIGNED_LOAD32(p+1) ... UNALIGNED_LOAD32(p+2) * are slower than UNALIGNED_LOAD64(p) followed by shifts and casts to u32. * * We have different versions for 64- and 32-bit; ideally we would avoid the * two functions and just inline the UNALIGNED_LOAD64 call into * GetUint32AtOffset, but GCC (at least not as of 4.6) is seemingly not clever * enough to avoid loading the value multiple times then. For 64-bit, the load * is done when GetEightBytesAt() is called, whereas for 32-bit, the load is * done at GetUint32AtOffset() time. */ #if BITS_PER_LONG == 64 typedef u64 eight_bytes_reference; static inline eight_bytes_reference get_eight_bytes_at(const char* ptr) { return UNALIGNED_LOAD64(ptr); } static inline u32 get_u32_at_offset(u64 v, int offset) { DCHECK_GE(offset, 0); DCHECK_LE(offset, 4); return v >> (is_little_endian()? 8 * offset : 32 - 8 * offset); } #else typedef const char *eight_bytes_reference; static inline eight_bytes_reference get_eight_bytes_at(const char* ptr) { return ptr; } static inline u32 get_u32_at_offset(const char *v, int offset) { DCHECK_GE(offset, 0); DCHECK_LE(offset, 4); return UNALIGNED_LOAD32(v + offset); } #endif /* * Flat array compression that does not emit the "uncompressed length" * prefix. Compresses "input" string to the "*op" buffer. * * REQUIRES: "input" is at most "kBlockSize" bytes long. * REQUIRES: "op" points to an array of memory that is at least * "MaxCompressedLength(input.size())" in size. * REQUIRES: All elements in "table[0..table_size-1]" are initialized to zero. * REQUIRES: "table_size" is a power of two * * Returns an "end" pointer into "op" buffer. * "end - op" is the compressed size of "input". */ static char *compress_fragment(const char *const input, const size_t input_size, char *op, u16 * table, const unsigned table_size) { /* "ip" is the input pointer, and "op" is the output pointer. */ const char *ip = input; CHECK_LE(input_size, kblock_size); CHECK_EQ(table_size & (table_size - 1), 0); const int shift = 32 - log2_floor(table_size); DCHECK_EQ(UINT_MAX >> shift, table_size - 1); const char *ip_end = input + input_size; const char *baseip = ip; /* * Bytes in [next_emit, ip) will be emitted as literal bytes. Or * [next_emit, ip_end) after the main loop. */ const char *next_emit = ip; const unsigned kinput_margin_bytes = 15; if (likely(input_size >= kinput_margin_bytes)) { const char *const ip_limit = input + input_size - kinput_margin_bytes; u32 next_hash; for (next_hash = hash(++ip, shift);;) { DCHECK_LT(next_emit, ip); /* * The body of this loop calls EmitLiteral once and then EmitCopy one or * more times. (The exception is that when we're close to exhausting * the input we goto emit_remainder.) * * In the first iteration of this loop we're just starting, so * there's nothing to copy, so calling EmitLiteral once is * necessary. And we only start a new iteration when the * current iteration has determined that a call to EmitLiteral will * precede the next call to EmitCopy (if any). * * Step 1: Scan forward in the input looking for a 4-byte-long match. * If we get close to exhausting the input then goto emit_remainder. * * Heuristic match skipping: If 32 bytes are scanned with no matches * found, start looking only at every other byte. If 32 more bytes are * scanned, look at every third byte, etc.. When a match is found, * immediately go back to looking at every byte. This is a small loss * (~5% performance, ~0.1% density) for lcompressible data due to more * bookkeeping, but for non-compressible data (such as JPEG) it's a huge * win since the compressor quickly "realizes" the data is incompressible * and doesn't bother looking for matches everywhere. * * The "skip" variable keeps track of how many bytes there are since the * last match; dividing it by 32 (ie. right-shifting by five) gives the * number of bytes to move ahead for each iteration. */ u32 skip_bytes = 32; const char *next_ip = ip; const char *candidate; do { ip = next_ip; u32 hval = next_hash; DCHECK_EQ(hval, hash(ip, shift)); u32 bytes_between_hash_lookups = skip_bytes++ >> 5; next_ip = ip + bytes_between_hash_lookups; if (unlikely(next_ip > ip_limit)) { goto emit_remainder; } next_hash = hash(next_ip, shift); candidate = baseip + table[hval]; DCHECK_GE(candidate, baseip); DCHECK_LT(candidate, ip); table[hval] = ip - baseip; } while (likely(UNALIGNED_LOAD32(ip) != UNALIGNED_LOAD32(candidate))); /* * Step 2: A 4-byte match has been found. We'll later see if more * than 4 bytes match. But, prior to the match, input * bytes [next_emit, ip) are unmatched. Emit them as "literal bytes." */ DCHECK_LE(next_emit + 16, ip_end); op = emit_literal(op, next_emit, ip - next_emit, true); /* * Step 3: Call EmitCopy, and then see if another EmitCopy could * be our next move. Repeat until we find no match for the * input immediately after what was consumed by the last EmitCopy call. * * If we exit this loop normally then we need to call EmitLiteral next, * though we don't yet know how big the literal will be. We handle that * by proceeding to the next iteration of the main loop. We also can exit * this loop via goto if we get close to exhausting the input. */ eight_bytes_reference input_bytes; u32 candidate_bytes = 0; do { /* * We have a 4-byte match at ip, and no need to emit any * "literal bytes" prior to ip. */ const char *base = ip; int matched = 4 + find_match_length(candidate + 4, ip + 4, ip_end); ip += matched; int offset = base - candidate; DCHECK_EQ(0, memcmp(base, candidate, matched)); op = emit_copy(op, offset, matched); /* * We could immediately start working at ip now, but to improve * compression we first update table[Hash(ip - 1, ...)]. */ const char *insert_tail = ip - 1; next_emit = ip; if (unlikely(ip >= ip_limit)) { goto emit_remainder; } input_bytes = get_eight_bytes_at(insert_tail); u32 prev_hash = hash_bytes(get_u32_at_offset (input_bytes, 0), shift); table[prev_hash] = ip - baseip - 1; u32 cur_hash = hash_bytes(get_u32_at_offset (input_bytes, 1), shift); candidate = baseip + table[cur_hash]; candidate_bytes = UNALIGNED_LOAD32(candidate); table[cur_hash] = ip - baseip; } while (get_u32_at_offset(input_bytes, 1) == candidate_bytes); next_hash = hash_bytes(get_u32_at_offset(input_bytes, 2), shift); ++ip; } } emit_remainder: /* Emit the remaining bytes as a literal */ if (next_emit < ip_end) op = emit_literal(op, next_emit, ip_end - next_emit, false); return op; } /* * ----------------------------------------------------------------------- * Lookup table for decompression code. Generated by ComputeTable() below. * ----------------------------------------------------------------------- */ /* Mapping from i in range [0,4] to a mask to extract the bottom 8*i bits */ static const u32 wordmask[] = { 0u, 0xffu, 0xffffu, 0xffffffu, 0xffffffffu }; /* * Data stored per entry in lookup table: * Range Bits-used Description * ------------------------------------ * 1..64 0..7 Literal/copy length encoded in opcode byte * 0..7 8..10 Copy offset encoded in opcode byte / 256 * 0..4 11..13 Extra bytes after opcode * * We use eight bits for the length even though 7 would have sufficed * because of efficiency reasons: * (1) Extracting a byte is faster than a bit-field * (2) It properly aligns copy offset so we do not need a <<8 */ static const u16 char_table[256] = { 0x0001, 0x0804, 0x1001, 0x2001, 0x0002, 0x0805, 0x1002, 0x2002, 0x0003, 0x0806, 0x1003, 0x2003, 0x0004, 0x0807, 0x1004, 0x2004, 0x0005, 0x0808, 0x1005, 0x2005, 0x0006, 0x0809, 0x1006, 0x2006, 0x0007, 0x080a, 0x1007, 0x2007, 0x0008, 0x080b, 0x1008, 0x2008, 0x0009, 0x0904, 0x1009, 0x2009, 0x000a, 0x0905, 0x100a, 0x200a, 0x000b, 0x0906, 0x100b, 0x200b, 0x000c, 0x0907, 0x100c, 0x200c, 0x000d, 0x0908, 0x100d, 0x200d, 0x000e, 0x0909, 0x100e, 0x200e, 0x000f, 0x090a, 0x100f, 0x200f, 0x0010, 0x090b, 0x1010, 0x2010, 0x0011, 0x0a04, 0x1011, 0x2011, 0x0012, 0x0a05, 0x1012, 0x2012, 0x0013, 0x0a06, 0x1013, 0x2013, 0x0014, 0x0a07, 0x1014, 0x2014, 0x0015, 0x0a08, 0x1015, 0x2015, 0x0016, 0x0a09, 0x1016, 0x2016, 0x0017, 0x0a0a, 0x1017, 0x2017, 0x0018, 0x0a0b, 0x1018, 0x2018, 0x0019, 0x0b04, 0x1019, 0x2019, 0x001a, 0x0b05, 0x101a, 0x201a, 0x001b, 0x0b06, 0x101b, 0x201b, 0x001c, 0x0b07, 0x101c, 0x201c, 0x001d, 0x0b08, 0x101d, 0x201d, 0x001e, 0x0b09, 0x101e, 0x201e, 0x001f, 0x0b0a, 0x101f, 0x201f, 0x0020, 0x0b0b, 0x1020, 0x2020, 0x0021, 0x0c04, 0x1021, 0x2021, 0x0022, 0x0c05, 0x1022, 0x2022, 0x0023, 0x0c06, 0x1023, 0x2023, 0x0024, 0x0c07, 0x1024, 0x2024, 0x0025, 0x0c08, 0x1025, 0x2025, 0x0026, 0x0c09, 0x1026, 0x2026, 0x0027, 0x0c0a, 0x1027, 0x2027, 0x0028, 0x0c0b, 0x1028, 0x2028, 0x0029, 0x0d04, 0x1029, 0x2029, 0x002a, 0x0d05, 0x102a, 0x202a, 0x002b, 0x0d06, 0x102b, 0x202b, 0x002c, 0x0d07, 0x102c, 0x202c, 0x002d, 0x0d08, 0x102d, 0x202d, 0x002e, 0x0d09, 0x102e, 0x202e, 0x002f, 0x0d0a, 0x102f, 0x202f, 0x0030, 0x0d0b, 0x1030, 0x2030, 0x0031, 0x0e04, 0x1031, 0x2031, 0x0032, 0x0e05, 0x1032, 0x2032, 0x0033, 0x0e06, 0x1033, 0x2033, 0x0034, 0x0e07, 0x1034, 0x2034, 0x0035, 0x0e08, 0x1035, 0x2035, 0x0036, 0x0e09, 0x1036, 0x2036, 0x0037, 0x0e0a, 0x1037, 0x2037, 0x0038, 0x0e0b, 0x1038, 0x2038, 0x0039, 0x0f04, 0x1039, 0x2039, 0x003a, 0x0f05, 0x103a, 0x203a, 0x003b, 0x0f06, 0x103b, 0x203b, 0x003c, 0x0f07, 0x103c, 0x203c, 0x0801, 0x0f08, 0x103d, 0x203d, 0x1001, 0x0f09, 0x103e, 0x203e, 0x1801, 0x0f0a, 0x103f, 0x203f, 0x2001, 0x0f0b, 0x1040, 0x2040 }; struct snappy_decompressor { struct source *reader; /* Underlying source of bytes to decompress */ const char *ip; /* Points to next buffered byte */ const char *ip_limit; /* Points just past buffered bytes */ u32 peeked; /* Bytes peeked from reader (need to skip) */ bool eof; /* Hit end of input without an error? */ char scratch[5]; /* Temporary buffer for peekfast boundaries */ }; static void init_snappy_decompressor(struct snappy_decompressor *d, struct source *reader) { d->reader = reader; d->ip = NULL; d->ip_limit = NULL; d->peeked = 0; d->eof = false; } static void exit_snappy_decompressor(struct snappy_decompressor *d) { skip(d->reader, d->peeked); } /* * Read the uncompressed length stored at the start of the compressed data. * On succcess, stores the length in *result and returns true. * On failure, returns false. */ static bool read_uncompressed_length(struct snappy_decompressor *d, u32 * result) { DCHECK(d->ip == NULL); /* * Must not have read anything yet * Length is encoded in 1..5 bytes */ *result = 0; u32 shift = 0; while (true) { if (shift >= 32) return false; size_t n; const char *ip = peek(d->reader, &n); if (n == 0) return false; const unsigned char c = *(const unsigned char *)(ip); skip(d->reader, 1); *result |= (u32) (c & 0x7f) << shift; if (c < 128) { break; } shift += 7; } return true; } static bool refill_tag(struct snappy_decompressor *d); /* * Process the next item found in the input. * Returns true if successful, false on error or end of input. */ static void decompress_all_tags(struct snappy_decompressor *d, struct writer *writer) { const char *ip = d->ip; /* * We could have put this refill fragment only at the beginning of the loop. * However, duplicating it at the end of each branch gives the compiler more * scope to optimize the expression based on the local * context, which overall increases speed. */ #define MAYBE_REFILL() \ if (d->ip_limit - ip < 5) { \ d->ip = ip; \ if (!refill_tag(d)) return; \ ip = d->ip; \ } MAYBE_REFILL(); for (;;) { if (d->ip_limit - ip < 5) { d->ip = ip; if (!refill_tag(d)) return; ip = d->ip; } const unsigned char c = *(const unsigned char *)(ip++); if ((c & 0x3) == LITERAL) { u32 literal_length = (c >> 2) + 1; if (writer_try_fast_append(writer, ip, d->ip_limit - ip, literal_length)) { DCHECK_LT(literal_length, 61); ip += literal_length; MAYBE_REFILL(); continue; } if (unlikely(literal_length >= 61)) { /* Long literal */ const u32 literal_ll = literal_length - 60; literal_length = (get_unaligned_le32(ip) & wordmask[literal_ll]) + 1; ip += literal_ll; } u32 avail = d->ip_limit - ip; while (avail < literal_length) { if (!writer_append(writer, ip, avail)) return; literal_length -= avail; skip(d->reader, d->peeked); size_t n; ip = peek(d->reader, &n); avail = n; d->peeked = avail; if (avail == 0) return; /* Premature end of input */ d->ip_limit = ip + avail; } if (!writer_append(writer, ip, literal_length)) return; ip += literal_length; MAYBE_REFILL(); } else { const u32 entry = char_table[c]; const u32 trailer = get_unaligned_le32(ip) & wordmask[entry >> 11]; const u32 length = entry & 0xff; ip += entry >> 11; /* * copy_offset/256 is encoded in bits 8..10. * By just fetching those bits, we get * copy_offset (since the bit-field starts at * bit 8). */ const u32 copy_offset = entry & 0x700; if (!writer_append_from_self(writer, copy_offset + trailer, length)) return; MAYBE_REFILL(); } } } #undef MAYBE_REFILL static bool refill_tag(struct snappy_decompressor *d) { const char *ip = d->ip; if (ip == d->ip_limit) { size_t n; /* Fetch a new fragment from the reader */ skip(d->reader, d->peeked); /* All peeked bytes are used up */ ip = peek(d->reader, &n); d->peeked = n; if (n == 0) { d->eof = true; return false; } d->ip_limit = ip + n; } /* Read the tag character */ DCHECK_LT(ip, d->ip_limit); const unsigned char c = *(const unsigned char *)(ip); const u32 entry = char_table[c]; const u32 needed = (entry >> 11) + 1; /* +1 byte for 'c' */ DCHECK_LE(needed, sizeof(d->scratch)); /* Read more bytes from reader if needed */ u32 nbuf = d->ip_limit - ip; if (nbuf < needed) { /* * Stitch together bytes from ip and reader to form the word * contents. We store the needed bytes in "scratch". They * will be consumed immediately by the caller since we do not * read more than we need. */ memmove(d->scratch, ip, nbuf); skip(d->reader, d->peeked); /* All peeked bytes are used up */ d->peeked = 0; while (nbuf < needed) { size_t length; const char *src = peek(d->reader, &length); if (length == 0) return false; u32 to_add = min_t(u32, needed - nbuf, length); memcpy(d->scratch + nbuf, src, to_add); nbuf += to_add; skip(d->reader, to_add); } DCHECK_EQ(nbuf, needed); d->ip = d->scratch; d->ip_limit = d->scratch + needed; } else if (nbuf < 5) { /* * Have enough bytes, but move into scratch so that we do not * read past end of input */ memmove(d->scratch, ip, nbuf); skip(d->reader, d->peeked); /* All peeked bytes are used up */ d->peeked = 0; d->ip = d->scratch; d->ip_limit = d->scratch + nbuf; } else { /* Pass pointer to buffer returned by reader. */ d->ip = ip; } return true; } static int internal_uncompress(struct source *r, struct writer *writer, u32 max_len) { struct snappy_decompressor decompressor; u32 uncompressed_len = 0; init_snappy_decompressor(&decompressor, r); if (!read_uncompressed_length(&decompressor, &uncompressed_len)) return -EIO; /* Protect against possible DoS attack */ if ((u64) (uncompressed_len) > max_len) return -EIO; writer_set_expected_length(writer, uncompressed_len); /* Process the entire input */ decompress_all_tags(&decompressor, writer); exit_snappy_decompressor(&decompressor); if (decompressor.eof && writer_check_length(writer)) return 0; return -EIO; } static inline int compress(struct snappy_env *env, struct source *reader, struct sink *writer) { int err; size_t written = 0; int N = available(reader); char ulength[kmax32]; char *p = varint_encode32(ulength, N); append(writer, ulength, p - ulength); written += (p - ulength); while (N > 0) { /* Get next block to compress (without copying if possible) */ size_t fragment_size; const char *fragment = peek(reader, &fragment_size); if (fragment_size == 0) { err = -EIO; goto out; } const unsigned num_to_read = min_t(int, N, kblock_size); size_t bytes_read = fragment_size; int pending_advance = 0; if (bytes_read >= num_to_read) { /* Buffer returned by reader is large enough */ pending_advance = num_to_read; fragment_size = num_to_read; } else { memcpy(env->scratch, fragment, bytes_read); skip(reader, bytes_read); while (bytes_read < num_to_read) { fragment = peek(reader, &fragment_size); size_t n = min_t(size_t, fragment_size, num_to_read - bytes_read); memcpy((char *)(env->scratch) + bytes_read, fragment, n); bytes_read += n; skip(reader, n); } DCHECK_EQ(bytes_read, num_to_read); fragment = env->scratch; fragment_size = num_to_read; } if (fragment_size < num_to_read) return -EIO; /* Get encoding table for compression */ int table_size; u16 *table = get_hash_table(env, num_to_read, &table_size); /* Compress input_fragment and append to dest */ char *dest; dest = sink_peek(writer, snappy_max_compressed_length(num_to_read)); if (!dest) { /* * Need a scratch buffer for the output, * because the byte sink doesn't have enough * in one piece. */ dest = env->scratch_output; } char *end = compress_fragment(fragment, fragment_size, dest, table, table_size); append(writer, dest, end - dest); written += (end - dest); N -= num_to_read; skip(reader, pending_advance); } err = 0; out: return err; } #ifdef SG int snappy_compress_iov(struct snappy_env *env, struct iovec *iov_in, int iov_in_len, size_t input_length, struct iovec *iov_out, int *iov_out_len, size_t *compressed_length) { struct source reader = { .iov = iov_in, .iovlen = iov_in_len, .total = input_length }; struct sink writer = { .iov = iov_out, .iovlen = *iov_out_len, }; int err = compress(env, &reader, &writer); *iov_out_len = writer.curvec + 1; /* Compute how many bytes were added */ *compressed_length = writer.written; return err; } EXPORT_SYMBOL(snappy_compress_iov); /** * snappy_compress - Compress a buffer using the snappy compressor. * @env: Preallocated environment * @input: Input buffer * @input_length: Length of input_buffer * @compressed: Output buffer for compressed data * @compressed_length: The real length of the output written here. * * Return 0 on success, otherwise an negative error code. * * The output buffer must be at least * snappy_max_compressed_length(input_length) bytes long. * * Requires a preallocated environment from snappy_init_env. * The environment does not keep state over individual calls * of this function, just preallocates the memory. */ int snappy_compress(struct snappy_env *env, const char *input, size_t input_length, char *compressed, size_t *compressed_length) { struct iovec iov_in = { .iov_base = (char *)input, .iov_len = input_length, }; struct iovec iov_out = { .iov_base = compressed, .iov_len = 0xffffffff, }; int out = 1; return snappy_compress_iov(env, &iov_in, 1, input_length, &iov_out, &out, compressed_length); } EXPORT_SYMBOL(snappy_compress); int snappy_uncompress_iov(struct iovec *iov_in, int iov_in_len, size_t input_len, char *uncompressed) { struct source reader = { .iov = iov_in, .iovlen = iov_in_len, .total = input_len }; struct writer output = { .base = uncompressed, .op = uncompressed }; return internal_uncompress(&reader, &output, 0xffffffff); } EXPORT_SYMBOL(snappy_uncompress_iov); /** * snappy_uncompress - Uncompress a snappy compressed buffer * @compressed: Input buffer with compressed data * @n: length of compressed buffer * @uncompressed: buffer for uncompressed data * * The uncompressed data buffer must be at least * snappy_uncompressed_length(compressed) bytes long. * * Return 0 on success, otherwise an negative error code. */ int snappy_uncompress(const char *compressed, size_t n, char *uncompressed) { struct iovec iov = { .iov_base = (char *)compressed, .iov_len = n }; return snappy_uncompress_iov(&iov, 1, n, uncompressed); } EXPORT_SYMBOL(snappy_uncompress); #else /** * snappy_compress - Compress a buffer using the snappy compressor. * @env: Preallocated environment * @input: Input buffer * @input_length: Length of input_buffer * @compressed: Output buffer for compressed data * @compressed_length: The real length of the output written here. * * Return 0 on success, otherwise an negative error code. * * The output buffer must be at least * snappy_max_compressed_length(input_length) bytes long. * * Requires a preallocated environment from snappy_init_env. * The environment does not keep state over individual calls * of this function, just preallocates the memory. */ int snappy_compress(struct snappy_env *env, const char *input, size_t input_length, char *compressed, size_t *compressed_length) { struct source reader = { .ptr = input, .left = input_length }; struct sink writer = { .dest = compressed, }; int err = compress(env, &reader, &writer); /* Compute how many bytes were added */ *compressed_length = (writer.dest - compressed); return err; } EXPORT_SYMBOL(snappy_compress); /** * snappy_uncompress - Uncompress a snappy compressed buffer * @compressed: Input buffer with compressed data * @n: length of compressed buffer * @uncompressed: buffer for uncompressed data * * The uncompressed data buffer must be at least * snappy_uncompressed_length(compressed) bytes long. * * Return 0 on success, otherwise an negative error code. */ int snappy_uncompress(const char *compressed, size_t n, char *uncompressed) { struct source reader = { .ptr = compressed, .left = n }; struct writer output = { .base = uncompressed, .op = uncompressed }; return internal_uncompress(&reader, &output, 0xffffffff); } EXPORT_SYMBOL(snappy_uncompress); #endif #ifdef SG /** * snappy_init_env_sg - Allocate snappy compression environment * @env: Environment to preallocate * @sg: Input environment ever does scather gather * * If false is passed to sg then multiple entries in an iovec * are not legal. * Returns 0 on success, otherwise negative errno. * Must run in process context. */ int snappy_init_env_sg(struct snappy_env *env, bool sg) { env->hash_table = vmalloc(sizeof(u16) * kmax_hash_table_size); if (!env->hash_table) goto error; if (sg) { env->scratch = vmalloc(kblock_size); if (!env->scratch) goto error; env->scratch_output = vmalloc(snappy_max_compressed_length(kblock_size)); if (!env->scratch_output) goto error; } return 0; error: snappy_free_env(env); return -ENOMEM; } EXPORT_SYMBOL(snappy_init_env_sg); #endif /** * snappy_init_env - Allocate snappy compression environment * @env: Environment to preallocate * * Passing multiple entries in an iovec is not allowed * on the environment allocated here. * Returns 0 on success, otherwise negative errno. * Must run in process context. */ int snappy_init_env(struct snappy_env *env) { env->hash_table = vmalloc(sizeof(u16) * kmax_hash_table_size); if (!env->hash_table) return -ENOMEM; return 0; } EXPORT_SYMBOL(snappy_init_env); /** * snappy_free_env - Free an snappy compression environment * @env: Environment to free. * * Must run in process context. */ void snappy_free_env(struct snappy_env *env) { vfree(env->hash_table); #ifdef SG vfree(env->scratch); vfree(env->scratch_output); #endif memset(env, 0, sizeof(struct snappy_env)); } EXPORT_SYMBOL(snappy_free_env); librdkafka-0.8.3/snappy.h000066400000000000000000000021061227620010100152650ustar00rootroot00000000000000#ifndef _LINUX_SNAPPY_H #define _LINUX_SNAPPY_H 1 #include #include /* Only needed for compression. This preallocates the worst case */ struct snappy_env { unsigned short *hash_table; void *scratch; void *scratch_output; }; struct iovec; int snappy_init_env(struct snappy_env *env); int snappy_init_env_sg(struct snappy_env *env, bool sg); void snappy_free_env(struct snappy_env *env); int snappy_uncompress_iov(struct iovec *iov_in, int iov_in_len, size_t input_len, char *uncompressed); int snappy_uncompress(const char *compressed, size_t n, char *uncompressed); int snappy_compress(struct snappy_env *env, const char *input, size_t input_length, char *compressed, size_t *compressed_length); int snappy_compress_iov(struct snappy_env *env, struct iovec *iov_in, int iov_in_len, size_t input_length, struct iovec *iov_out, int *iov_out_len, size_t *compressed_length); bool snappy_uncompressed_length(const char *buf, size_t len, size_t *result); size_t snappy_max_compressed_length(size_t source_len); #endif librdkafka-0.8.3/snappy_compat.h000066400000000000000000000036461227620010100166420ustar00rootroot00000000000000#include #include #include #include #include #include #include #include "endian_compat.h" #if defined(__arm__) && \ !defined(__ARM_ARCH_4__) && \ !defined(__ARM_ARCH_4T__) && \ !defined(__ARM_ARCH_5__) && \ !defined(__ARM_ARCH_5T__) && \ !defined(__ARM_ARCH_5TE__) && \ !defined(__ARM_ARCH_5TEJ__) && \ !defined(__ARM_ARCH_6__) && \ !defined(__ARM_ARCH_6J__) && \ !defined(__ARM_ARCH_6K__) && \ !defined(__ARM_ARCH_6Z__) && \ !defined(__ARM_ARCH_6ZK__) && \ !defined(__ARM_ARCH_6T2__) #define UNALIGNED64_REALLYS_SLOW 1 #endif #ifndef htole16 # if __BYTE_ORDER == __LITTLE_ENDIAN # define htole16(x) (x) # define le32toh(x) (x) # else # define htole16(x) __bswap_16 (x) # define le32toh(x) __bswap_32 (x) #endif #endif typedef unsigned char u8; typedef unsigned short u16; typedef unsigned u32; typedef unsigned long long u64; #define BUG_ON(x) assert(!(x)) #define get_unaligned(x) (*(x)) #define get_unaligned_le32(x) (le32toh(*(u32 *)(x))) #define put_unaligned(v,x) (*(x) = (v)) #define put_unaligned_le16(v,x) (*(u16 *)(x) = htole16(v)) /* You may want to define this on various ARM architectures */ #ifdef UNALIGNED64_REALLYS_SLOW static inline u64 get_unaligned64(const void *p) { u64 t; memcpy(&t, p, 8); return t; } static inline u64 put_unaligned64(u64 t, void *p) { memcpy(p, &t, 8); return t; } #else #define get_unaligned64(x) get_unaligned(x) #define put_unaligned64(x,p) put_unaligned(x,p) #endif #define vmalloc(x) malloc(x) #define vfree(x) free(x) #define EXPORT_SYMBOL(x) #define ARRAY_SIZE(x) (sizeof(x) / sizeof(*(x))) #define likely(x) __builtin_expect((x), 1) #define unlikely(x) __builtin_expect((x), 0) #define min_t(t,x,y) ((x) < (y) ? (x) : (y)) #define max_t(t,x,y) ((x) > (y) ? (x) : (y)) #if __BYTE_ORDER == __LITTLE_ENDIAN #define __LITTLE_ENDIAN__ 1 #endif #define BITS_PER_LONG (__SIZEOF_LONG__ * 8) librdkafka-0.8.3/tests/000077500000000000000000000000001227620010100147455ustar00rootroot00000000000000librdkafka-0.8.3/tests/.gitignore000066400000000000000000000000211227620010100167260ustar00rootroot00000000000000*.test test.conf librdkafka-0.8.3/tests/0001-multiobj.c000066400000000000000000000067631227620010100173300ustar00rootroot00000000000000/* * librdkafka - Apache Kafka C library * * Copyright (c) 2012-2013, Magnus Edenhill * All rights reserved. * * Redistribution and use in source and binary forms, with or without * modification, are permitted provided that the following conditions are met: * * 1. Redistributions of source code must retain the above copyright notice, * this list of conditions and the following disclaimer. * 2. Redistributions in binary form must reproduce the above copyright notice, * this list of conditions and the following disclaimer in the documentation * and/or other materials provided with the distribution. * * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE * POSSIBILITY OF SUCH DAMAGE. */ /** * Tests multiple rd_kafka_t object creations and destructions. * Issue #20 */ #define _GNU_SOURCE #include #include #include "test.h" /* Typical include path would be , but this program * is built from within the librdkafka source tree and thus differs. */ #include "rdkafka.h" /* for Kafka driver */ int main (int argc, char **argv) { char *topic = "rdkafkatest1"; int partition = RD_KAFKA_PARTITION_UA; /* random */ int i; const int NUM_ITER = 100; struct rlimit rlim = {}; /* * Put some limits to catch bad cleanups by librdkafka (issue #20) */ /* File descriptors. One or two per broker. */ rlim.rlim_cur = rlim.rlim_max = NUM_ITER * 5; setrlimit(RLIMIT_NOFILE, &rlim); /* best effort, fails under valgrind */ TEST_SAY("Creating and destroying %i kafka instances\n", NUM_ITER); /* Create, use and destroy NUM_ITER kafka instances. */ for (i = 0 ; i < NUM_ITER ; i++) { rd_kafka_t *rk; rd_kafka_topic_t *rkt; rd_kafka_conf_t *conf; rd_kafka_topic_conf_t *topic_conf; char errstr[512]; char msg[128]; test_conf_init(&conf, &topic_conf, 30); rk = rd_kafka_new(RD_KAFKA_PRODUCER, conf, errstr, sizeof(errstr)); if (!rk) TEST_FAIL("Failed to create rdkafka instance #%i: %s\n", i, errstr); rkt = rd_kafka_topic_new(rk, topic, topic_conf); if (!rkt) TEST_FAIL("Failed to create topic for " "rdkafka instance #%i: %s\n", i, strerror(errno)); snprintf(msg, sizeof(msg), "%s test message for iteration #%i", argv[0], i); /* Produce a message */ rd_kafka_produce(rkt, partition, RD_KAFKA_MSG_F_COPY, msg, strlen(msg), NULL, 0, NULL); /* Wait for it to be sent (and possibly acked) */ while (rd_kafka_outq_len(rk) > 0) rd_kafka_poll(rk, 50); /* Destroy topic */ rd_kafka_topic_destroy(rkt); /* Destroy rdkafka instance */ rd_kafka_destroy(rk); } /* Wait for everything to be cleaned up since broker destroys are * handled in its own thread. */ test_wait_exit(10); /* If we havent failed at this point then * there were no threads leaked */ return 0; } librdkafka-0.8.3/tests/0002-unkpart.c000066400000000000000000000111451227620010100171560ustar00rootroot00000000000000/* * librdkafka - Apache Kafka C library * * Copyright (c) 2012-2013, Magnus Edenhill * All rights reserved. * * Redistribution and use in source and binary forms, with or without * modification, are permitted provided that the following conditions are met: * * 1. Redistributions of source code must retain the above copyright notice, * this list of conditions and the following disclaimer. * 2. Redistributions in binary form must reproduce the above copyright notice, * this list of conditions and the following disclaimer in the documentation * and/or other materials provided with the distribution. * * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE * POSSIBILITY OF SUCH DAMAGE. */ /** * Tests that producing to unknown partitions fails. * Issue #39 */ #define _GNU_SOURCE #include #include #include "test.h" /* Typical include path would be , but this program * is built from within the librdkafka source tree and thus differs. */ #include "rdkafka.h" /* for Kafka driver */ static int msgs_wait = 0; /* bitmask */ /** * Delivery report callback. * Called for each message once to signal its delivery status. */ static void dr_cb (rd_kafka_t *rk, void *payload, size_t len, rd_kafka_resp_err_t err, void *opaque, void *msg_opaque) { int msgid = *(int *)msg_opaque; free(msg_opaque); if (!(msgs_wait & (1 << msgid))) TEST_FAIL("Unwanted delivery report for message #%i " "(waiting for 0x%x)\n", msgid, msgs_wait); TEST_SAY("Delivery report for message #%i: %s\n", msgid, rd_kafka_err2str(err)); msgs_wait &= ~(1 << msgid); if (err != RD_KAFKA_RESP_ERR__UNKNOWN_PARTITION) TEST_FAIL("Message #%i failed with unexpected error %s\n", msgid, rd_kafka_err2str(err)); } int main (int argc, char **argv) { char *topic = "rdkafkatest1"; int partition = 99; /* non-existent */ int r; rd_kafka_t *rk; rd_kafka_topic_t *rkt; rd_kafka_conf_t *conf; rd_kafka_topic_conf_t *topic_conf; char errstr[512]; char msg[128]; int msgcnt = 10; int i; test_conf_init(&conf, &topic_conf, 10); /* Set delivery report callback */ rd_kafka_conf_set_dr_cb(conf, dr_cb); /* Create kafka instance */ rk = rd_kafka_new(RD_KAFKA_PRODUCER, conf, errstr, sizeof(errstr)); if (!rk) TEST_FAIL("Failed to create rdkafka instance: %s\n", errstr); TEST_SAY("Created kafka instance %s\n", rd_kafka_name(rk)); rkt = rd_kafka_topic_new(rk, topic, topic_conf); if (!rkt) TEST_FAIL("Failed to create topic: %s\n", strerror(errno)); /* Produce a message */ for (i = 0 ; i < msgcnt ; i++) { int *msgidp = malloc(sizeof(*msgidp)); *msgidp = i; snprintf(msg, sizeof(msg), "%s test message #%i", argv[0], i); r = rd_kafka_produce(rkt, partition, RD_KAFKA_MSG_F_COPY, msg, strlen(msg), NULL, 0, msgidp); if (r == -1) { if (errno == ESRCH) TEST_SAY("Failed to produce message #%i: " "unknown partition: good!\n", i); else TEST_FAIL("Failed to produce message #%i: %s\n", i, strerror(errno)); } else { if (i > 5) TEST_FAIL("Message #%i produced: " "should've failed\n", i); msgs_wait |= (1 << i); } /* After half the messages: sleep to allow the metadata * to be fetched from broker and update the actual partition * count: this will make subsequent produce() calls fail * immediately. */ if (i == 5) sleep(2); } /* Wait for messages to time out */ while (rd_kafka_outq_len(rk) > 0) rd_kafka_poll(rk, 50); if (msgs_wait != 0) TEST_FAIL("Still waiting for messages: 0x%x\n", msgs_wait); /* Destroy topic */ rd_kafka_topic_destroy(rkt); /* Destroy rdkafka instance */ TEST_SAY("Destroying kafka instance %s\n", rd_kafka_name(rk)); rd_kafka_destroy(rk); /* Wait for everything to be cleaned up since broker destroys are * handled in its own thread. */ test_wait_exit(10); /* If we havent failed at this point then * there were no threads leaked */ return 0; } librdkafka-0.8.3/tests/0003-msgmaxsize.c000066400000000000000000000112151227620010100176600ustar00rootroot00000000000000/* * librdkafka - Apache Kafka C library * * Copyright (c) 2012-2013, Magnus Edenhill * All rights reserved. * * Redistribution and use in source and binary forms, with or without * modification, are permitted provided that the following conditions are met: * * 1. Redistributions of source code must retain the above copyright notice, * this list of conditions and the following disclaimer. * 2. Redistributions in binary form must reproduce the above copyright notice, * this list of conditions and the following disclaimer in the documentation * and/or other materials provided with the distribution. * * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE * POSSIBILITY OF SUCH DAMAGE. */ /** * Tests "message.bytes.max" * Issue #24 */ #define _GNU_SOURCE #include #include #include "test.h" /* Typical include path would be , but this program * is built from within the librdkafka source tree and thus differs. */ #include "rdkafka.h" /* for Kafka driver */ static int msgs_wait = 0; /* bitmask */ /** * Delivery report callback. * Called for each message once to signal its delivery status. */ static void dr_cb (rd_kafka_t *rk, void *payload, size_t len, rd_kafka_resp_err_t err, void *opaque, void *msg_opaque) { int msgid = *(int *)msg_opaque; free(msg_opaque); if (err) TEST_FAIL("Unexpected delivery error for message #%i: %s\n", msgid, rd_kafka_err2str(err)); if (!(msgs_wait & (1 << msgid))) TEST_FAIL("Unwanted delivery report for message #%i " "(waiting for 0x%x)\n", msgid, msgs_wait); TEST_SAY("Delivery report for message #%i: %s\n", msgid, rd_kafka_err2str(err)); msgs_wait &= ~(1 << msgid); } int main (int argc, char **argv) { char *topic = "rdkafkatest1"; int partition = 0; int r; rd_kafka_t *rk; rd_kafka_topic_t *rkt; rd_kafka_conf_t *conf; rd_kafka_topic_conf_t *topic_conf; char errstr[512]; char msg[100000]; int msgcnt = 10; int i; test_conf_init(&conf, &topic_conf, 10); /* Set a small maximum message size. */ if (rd_kafka_conf_set(conf, "message.max.bytes", "100000", errstr, sizeof(errstr)) != RD_KAFKA_CONF_OK) TEST_FAIL("%s\n", errstr); /* Set delivery report callback */ rd_kafka_conf_set_dr_cb(conf, dr_cb); /* Create kafka instance */ rk = rd_kafka_new(RD_KAFKA_PRODUCER, conf, errstr, sizeof(errstr)); if (!rk) TEST_FAIL("Failed to create rdkafka instance: %s\n", errstr); TEST_SAY("Created kafka instance %s\n", rd_kafka_name(rk)); rkt = rd_kafka_topic_new(rk, topic, topic_conf); if (!rkt) TEST_FAIL("Failed to create topic: %s\n", strerror(errno)); memset(msg, 0, sizeof(msg)); /* Produce 'msgcnt' messages, size odd ones larger than max.bytes, * and even ones smaller than max.bytes. */ for (i = 0 ; i < msgcnt ; i++) { int *msgidp = malloc(sizeof(*msgidp)); size_t len; int toobig = i & 1; *msgidp = i; if (toobig) { /* Too big */ len = 200000; } else { /* Good size */ len = 5000; msgs_wait |= (1 << i); } snprintf(msg, sizeof(msg), "%s test message #%i", argv[0], i); r = rd_kafka_produce(rkt, partition, RD_KAFKA_MSG_F_COPY, msg, len, NULL, 0, msgidp); if (toobig) { if (r != -1) TEST_FAIL("Succeeded to produce too " "large message #%i\n", i); free(msgidp); } else if (r == -1) TEST_FAIL("Failed to produce message #%i: %s\n", i, strerror(errno)); } /* Wait for messages to be delivered. */ while (rd_kafka_outq_len(rk) > 0) rd_kafka_poll(rk, 50); if (msgs_wait != 0) TEST_FAIL("Still waiting for messages: 0x%x\n", msgs_wait); /* Destroy topic */ rd_kafka_topic_destroy(rkt); /* Destroy rdkafka instance */ TEST_SAY("Destroying kafka instance %s\n", rd_kafka_name(rk)); rd_kafka_destroy(rk); /* Wait for everything to be cleaned up since broker destroys are * handled in its own thread. */ test_wait_exit(10); /* If we havent failed at this point then * there were no threads leaked */ return 0; } librdkafka-0.8.3/tests/0004-conf.c000066400000000000000000000150651227620010100164260ustar00rootroot00000000000000/* * librdkafka - Apache Kafka C library * * Copyright (c) 2012-2013, Magnus Edenhill * All rights reserved. * * Redistribution and use in source and binary forms, with or without * modification, are permitted provided that the following conditions are met: * * 1. Redistributions of source code must retain the above copyright notice, * this list of conditions and the following disclaimer. * 2. Redistributions in binary form must reproduce the above copyright notice, * this list of conditions and the following disclaimer in the documentation * and/or other materials provided with the distribution. * * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE * POSSIBILITY OF SUCH DAMAGE. */ /** * Tests various config related things */ #define _GNU_SOURCE #include #include #include "test.h" /* Typical include path would be , but this program * is built from within the librdkafka source tree and thus differs. */ #include "rdkafka.h" /* for Kafka driver */ static void dr_cb (rd_kafka_t *rk, void *payload, size_t len, rd_kafka_resp_err_t err, void *opaque, void *msg_opaque) { } static void error_cb (rd_kafka_t *rk, int err, const char *reason, void *opaque) { } static int32_t partitioner (const rd_kafka_topic_t *rkt, const void *keydata, size_t keylen, int32_t partition_cnt, void *rkt_opaque, void *msg_opaque) { return 0; } static void conf_verify (int line, const char **arr, size_t cnt, const char **confs) { int i, j; for (i = 0 ; confs[i] ; i += 2) { for (j = 0 ; j < cnt ; j += 2) { if (!strcmp(confs[i], arr[j])) { if (strcmp(confs[i+1], arr[j+1])) TEST_FAIL("%i: Property %s mismatch: " "expected %s != retrieved %s", line, confs[i], confs[i+1], arr[j+1]); } if (j == cnt) TEST_FAIL("%i: " "Property %s not found in config\n", line, confs[i]); } } } static void conf_cmp (const char *desc, const char **a, size_t acnt, const char **b, size_t bcnt) { int i; if (acnt != bcnt) TEST_FAIL("%s config compare: count %zd != %zd mismatch", desc, acnt, bcnt); for (i = 0 ; i < acnt ; i += 2) { if (strcmp(a[i], b[i])) TEST_FAIL("%s conf mismatch: %s != %s", desc, a[i], b[i]); else if (strcmp(a[i+1], b[i+1])) TEST_FAIL("%s conf value mismatch for %s: %s != %s", desc, a[i], a[i+1], b[i+1]); } } int main (int argc, char **argv) { char *topic = "rdkafkatest1"; rd_kafka_t *rk; rd_kafka_topic_t *rkt; rd_kafka_conf_t *ignore_conf, *conf, *conf2; rd_kafka_topic_conf_t *ignore_topic_conf, *tconf, *tconf2; char errstr[512]; const char **arr_orig, **arr_dup; size_t cnt_orig, cnt_dup; int i; static const char *gconfs[] = { "message.max.bytes", "12345", /* int property */ "client.id", "my id", /* string property */ "debug", "topic,metadata", /* S2F property */ "compression.codec", "gzip", /* S2I property */ NULL }; static const char *tconfs[] = { "request.required.acks", "-1", /* int */ "auto.commit.enable", "false", /* bool */ "auto.offset.reset", "error", /* S2I */ "offset.store.path", "my/path", /* string */ NULL }; test_conf_init(&ignore_conf, &ignore_topic_conf, 2); rd_kafka_conf_destroy(ignore_conf); rd_kafka_topic_conf_destroy(ignore_topic_conf); /* Set up a global config object */ conf = rd_kafka_conf_new(); rd_kafka_conf_set_dr_cb(conf, dr_cb); rd_kafka_conf_set_error_cb(conf, error_cb); for (i = 0 ; gconfs[i] ; i += 2) { if (rd_kafka_conf_set(conf, gconfs[i], gconfs[i+1], errstr, sizeof(errstr)) != RD_KAFKA_CONF_OK) TEST_FAIL("%s\n", errstr); } /* Set up a topic config object */ tconf = rd_kafka_topic_conf_new(); rd_kafka_topic_conf_set_partitioner_cb(tconf, partitioner); rd_kafka_topic_conf_set_opaque(tconf, (void *)0xbeef); for (i = 0 ; tconfs[i] ; i += 2) { if (rd_kafka_topic_conf_set(tconf, tconfs[i], tconfs[i+1], errstr, sizeof(errstr)) != RD_KAFKA_CONF_OK) TEST_FAIL("%s\n", errstr); } /* Verify global config */ arr_orig = rd_kafka_conf_dump(conf, &cnt_orig); conf_verify(__LINE__, arr_orig, cnt_orig, gconfs); /* Verify copied global config */ conf2 = rd_kafka_conf_dup(conf); arr_dup = rd_kafka_conf_dump(conf2, &cnt_dup); conf_verify(__LINE__, arr_dup, cnt_dup, gconfs); conf_cmp("global", arr_orig, cnt_orig, arr_dup, cnt_dup); rd_kafka_conf_dump_free(arr_orig, cnt_orig); rd_kafka_conf_dump_free(arr_dup, cnt_dup); /* Verify topic config */ arr_orig = rd_kafka_topic_conf_dump(tconf, &cnt_orig); conf_verify(__LINE__, arr_orig, cnt_orig, tconfs); /* Verify copied topic config */ tconf2 = rd_kafka_topic_conf_dup(tconf); arr_dup = rd_kafka_topic_conf_dump(tconf2, &cnt_dup); conf_verify(__LINE__, arr_dup, cnt_dup, tconfs); conf_cmp("topic", arr_orig, cnt_orig, arr_dup, cnt_dup); rd_kafka_conf_dump_free(arr_orig, cnt_orig); rd_kafka_conf_dump_free(arr_dup, cnt_dup); /* * Create kafka instances using original and copied confs */ /* original */ rk = rd_kafka_new(RD_KAFKA_PRODUCER, conf, errstr, sizeof(errstr)); if (!rk) TEST_FAIL("Failed to create rdkafka instance: %s\n", errstr); rkt = rd_kafka_topic_new(rk, topic, tconf); if (!rkt) TEST_FAIL("Failed to create topic: %s\n", strerror(errno)); rd_kafka_topic_destroy(rkt); rd_kafka_destroy(rk); /* copied */ rk = rd_kafka_new(RD_KAFKA_PRODUCER, conf2, errstr, sizeof(errstr)); if (!rk) TEST_FAIL("Failed to create rdkafka instance: %s\n", errstr); rkt = rd_kafka_topic_new(rk, topic, tconf2); if (!rkt) TEST_FAIL("Failed to create topic: %s\n", strerror(errno)); rd_kafka_topic_destroy(rkt); rd_kafka_destroy(rk); /* Wait for everything to be cleaned up since broker destroys are * handled in its own thread. */ test_wait_exit(2); /* If we havent failed at this point then * there were no threads leaked */ return 0; } librdkafka-0.8.3/tests/0005-order.c000066400000000000000000000102071227620010100166060ustar00rootroot00000000000000/* * librdkafka - Apache Kafka C library * * Copyright (c) 2012-2013, Magnus Edenhill * All rights reserved. * * Redistribution and use in source and binary forms, with or without * modification, are permitted provided that the following conditions are met: * * 1. Redistributions of source code must retain the above copyright notice, * this list of conditions and the following disclaimer. * 2. Redistributions in binary form must reproduce the above copyright notice, * this list of conditions and the following disclaimer in the documentation * and/or other materials provided with the distribution. * * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE * POSSIBILITY OF SUCH DAMAGE. */ /** * Tests messages are produced in order. */ #define _GNU_SOURCE #include #include #include "test.h" /* Typical include path would be , but this program * is built from within the librdkafka source tree and thus differs. */ #include "rdkafka.h" /* for Kafka driver */ static int msgid_next = 0; static int fails = 0; /** * Delivery reported callback. * Called for each message once to signal its delivery status. */ static void dr_cb (rd_kafka_t *rk, void *payload, size_t len, rd_kafka_resp_err_t err, void *opaque, void *msg_opaque) { int msgid = *(int *)msg_opaque; free(msg_opaque); if (err != RD_KAFKA_RESP_ERR_NO_ERROR) TEST_FAIL("Message delivery failed: %s\n", rd_kafka_err2str(err)); if (msgid != msgid_next) { fails++; TEST_FAIL("Delivered msg %i, expected %i\n", msgid, msgid_next); return; } msgid_next = msgid+1; } int main (int argc, char **argv) { char *topic = "rdkafkatest1"; int partition = 0; int r; rd_kafka_t *rk; rd_kafka_topic_t *rkt; rd_kafka_conf_t *conf; rd_kafka_topic_conf_t *topic_conf; char errstr[512]; char msg[128]; int msgcnt = 100000; int i; test_conf_init(&conf, &topic_conf, 10); /* Set delivery report callback */ rd_kafka_conf_set_dr_cb(conf, dr_cb); /* Create kafka instance */ rk = rd_kafka_new(RD_KAFKA_PRODUCER, conf, errstr, sizeof(errstr)); if (!rk) TEST_FAIL("Failed to create rdkafka instance: %s\n", errstr); TEST_SAY("Created kafka instance %s\n", rd_kafka_name(rk)); rkt = rd_kafka_topic_new(rk, topic, topic_conf); if (!rkt) TEST_FAIL("Failed to create topic: %s\n", strerror(errno)); /* Produce a message */ for (i = 0 ; i < msgcnt ; i++) { int *msgidp = malloc(sizeof(*msgidp)); *msgidp = i; snprintf(msg, sizeof(msg), "%s test message #%i", argv[0], i); r = rd_kafka_produce(rkt, partition, RD_KAFKA_MSG_F_COPY, msg, strlen(msg), NULL, 0, msgidp); if (r == -1) TEST_FAIL("Failed to produce message #%i: %s\n", i, strerror(errno)); } TEST_SAY("Produced %i messages, waiting for deliveries\n", msgcnt); /* Wait for messages to time out */ while (rd_kafka_outq_len(rk) > 0) rd_kafka_poll(rk, 50); if (fails) TEST_FAIL("%i failures, see previous errors", fails); if (msgid_next != msgcnt) TEST_FAIL("Still waiting for messages: next %i != end %i\n", msgid_next, msgcnt); /* Destroy topic */ rd_kafka_topic_destroy(rkt); /* Destroy rdkafka instance */ TEST_SAY("Destroying kafka instance %s\n", rd_kafka_name(rk)); rd_kafka_destroy(rk); /* Wait for everything to be cleaned up since broker destroys are * handled in its own thread. */ test_wait_exit(10); /* If we havent failed at this point then * there were no threads leaked */ return 0; } librdkafka-0.8.3/tests/0006-symbols.c000066400000000000000000000104641227620010100171710ustar00rootroot00000000000000/* * librdkafka - Apache Kafka C library * * Copyright (c) 2012-2013, Magnus Edenhill * All rights reserved. * * Redistribution and use in source and binary forms, with or without * modification, are permitted provided that the following conditions are met: * * 1. Redistributions of source code must retain the above copyright notice, * this list of conditions and the following disclaimer. * 2. Redistributions in binary form must reproduce the above copyright notice, * this list of conditions and the following disclaimer in the documentation * and/or other materials provided with the distribution. * * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE * POSSIBILITY OF SUCH DAMAGE. */ /** * Makes sure all symbols in the public API actually resolves during linking. * This test needs to be updated manually when new symbols are added. */ #include "test.h" /* Typical include path would be , but this program * is built from within the librdkafka source tree and thus differs. */ #include "rdkafka.h" /* for Kafka driver */ int main (int argc, char **argv) { if (argc < 0 /* always false */) { rd_kafka_version(); rd_kafka_version_str(); rd_kafka_err2str(RD_KAFKA_RESP_ERR_NO_ERROR); rd_kafka_errno2err(EINVAL); rd_kafka_conf_new(); rd_kafka_conf_destroy(NULL); rd_kafka_conf_dup(NULL); rd_kafka_conf_set(NULL, NULL, NULL, NULL, 0); rd_kafka_conf_set_dr_cb(NULL, NULL); rd_kafka_conf_set_error_cb(NULL, NULL); rd_kafka_conf_set_stats_cb(NULL, NULL); rd_kafka_conf_set_opaque(NULL, NULL); rd_kafka_conf_dump(NULL, NULL); rd_kafka_topic_conf_dump(NULL, NULL); rd_kafka_conf_dump_free(NULL, 0); rd_kafka_conf_properties_show(NULL); rd_kafka_topic_conf_new(); rd_kafka_topic_conf_dup(NULL); rd_kafka_topic_conf_destroy(NULL); rd_kafka_topic_conf_set(NULL, NULL, NULL, NULL, 0); rd_kafka_topic_conf_set_opaque(NULL, NULL); rd_kafka_topic_conf_set_partitioner_cb(NULL, NULL); rd_kafka_topic_partition_available(NULL, 0); rd_kafka_msg_partitioner_random(NULL, NULL, 0, 0, NULL, NULL); rd_kafka_new(0, NULL, NULL, 0); rd_kafka_destroy(NULL); rd_kafka_name(NULL); rd_kafka_topic_new(NULL, NULL, NULL); rd_kafka_topic_destroy(NULL); rd_kafka_topic_name(NULL); rd_kafka_message_destroy(NULL); rd_kafka_message_errstr(NULL); rd_kafka_consume_start(NULL, 0, 0); rd_kafka_consume_stop(NULL, 0); rd_kafka_consume(NULL, 0, 0); rd_kafka_consume_batch(NULL, 0, 0, NULL, 0); rd_kafka_consume_callback(NULL, 0, 0, NULL, NULL); rd_kafka_offset_store(NULL, 0, 0); rd_kafka_produce(NULL, 0, 0, NULL, 0, NULL, 0, NULL); rd_kafka_poll(NULL, 0); rd_kafka_brokers_add(NULL, NULL); rd_kafka_set_logger(NULL, NULL); rd_kafka_set_log_level(NULL, 0); rd_kafka_log_print(NULL, 0, NULL, NULL); rd_kafka_log_syslog(NULL, 0, NULL, NULL); rd_kafka_outq_len(NULL); rd_kafka_dump(NULL, NULL); rd_kafka_thread_cnt(); rd_kafka_wait_destroyed(0); } return 0; } librdkafka-0.8.3/tests/0007-autotopic.c000066400000000000000000000106041227620010100175050ustar00rootroot00000000000000/* * librdkafka - Apache Kafka C library * * Copyright (c) 2012-2013, Magnus Edenhill * All rights reserved. * * Redistribution and use in source and binary forms, with or without * modification, are permitted provided that the following conditions are met: * * 1. Redistributions of source code must retain the above copyright notice, * this list of conditions and the following disclaimer. * 2. Redistributions in binary form must reproduce the above copyright notice, * this list of conditions and the following disclaimer in the documentation * and/or other materials provided with the distribution. * * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE * POSSIBILITY OF SUCH DAMAGE. */ /** * Auto create topics * * NOTE! This test requires auto.create.topics.enable=true to be * configured on the broker! */ #define _GNU_SOURCE #include #include #include "test.h" /* Typical include path would be , but this program * is built from within the librdkafka source tree and thus differs. */ #include "rdkafka.h" /* for Kafka driver */ static int msgs_wait = 0; /* bitmask */ /** * Delivery report callback. * Called for each message once to signal its delivery status. */ static void dr_cb (rd_kafka_t *rk, void *payload, size_t len, rd_kafka_resp_err_t err, void *opaque, void *msg_opaque) { int msgid = *(int *)msg_opaque; free(msg_opaque); if (!(msgs_wait & (1 << msgid))) TEST_FAIL("Unwanted delivery report for message #%i " "(waiting for 0x%x)\n", msgid, msgs_wait); TEST_SAY("Delivery report for message #%i: %s\n", msgid, rd_kafka_err2str(err)); msgs_wait &= ~(1 << msgid); if (err) TEST_FAIL("Message #%i failed with unexpected error %s\n", msgid, rd_kafka_err2str(err)); } int main (int argc, char **argv) { char topic[64]; int partition = 0; int r; rd_kafka_t *rk; rd_kafka_topic_t *rkt; rd_kafka_conf_t *conf; rd_kafka_topic_conf_t *topic_conf; char errstr[512]; char msg[128]; int msgcnt = 10; int i; /* Generate unique topic name */ test_conf_init(&conf, &topic_conf, 10); snprintf(topic, sizeof(topic), "rdkafkatest1_auto_%x%x", rand(), rand()); TEST_SAY("\033[33mNOTE! This test requires " "auto.create.topics.enable=true to be configured on " "the broker!\033[0m\n"); /* Set delivery report callback */ rd_kafka_conf_set_dr_cb(conf, dr_cb); /* Create kafka instance */ rk = rd_kafka_new(RD_KAFKA_PRODUCER, conf, errstr, sizeof(errstr)); if (!rk) TEST_FAIL("Failed to create rdkafka instance: %s\n", errstr); TEST_SAY("Created kafka instance %s\n", rd_kafka_name(rk)); rkt = rd_kafka_topic_new(rk, topic, topic_conf); if (!rkt) TEST_FAIL("Failed to create topic: %s\n", strerror(errno)); /* Produce a message */ for (i = 0 ; i < msgcnt ; i++) { int *msgidp = malloc(sizeof(*msgidp)); *msgidp = i; snprintf(msg, sizeof(msg), "%s test message #%i", argv[0], i); r = rd_kafka_produce(rkt, partition, RD_KAFKA_MSG_F_COPY, msg, strlen(msg), NULL, 0, msgidp); if (r == -1) TEST_FAIL("Failed to produce message #%i: %s\n", i, strerror(errno)); msgs_wait |= (1 << i); } /* Wait for messages to time out */ while (rd_kafka_outq_len(rk) > 0) rd_kafka_poll(rk, 50); if (msgs_wait != 0) TEST_FAIL("Still waiting for messages: 0x%x\n", msgs_wait); /* Destroy topic */ rd_kafka_topic_destroy(rkt); /* Destroy rdkafka instance */ TEST_SAY("Destroying kafka instance %s\n", rd_kafka_name(rk)); rd_kafka_destroy(rk); /* Wait for everything to be cleaned up since broker destroys are * handled in its own thread. */ test_wait_exit(10); /* If we havent failed at this point then * there were no threads leaked */ return 0; } librdkafka-0.8.3/tests/0008-reqacks.c000066400000000000000000000133021227620010100171260ustar00rootroot00000000000000/* * librdkafka - Apache Kafka C library * * Copyright (c) 2012-2013, Magnus Edenhill * All rights reserved. * * Redistribution and use in source and binary forms, with or without * modification, are permitted provided that the following conditions are met: * * 1. Redistributions of source code must retain the above copyright notice, * this list of conditions and the following disclaimer. * 2. Redistributions in binary form must reproduce the above copyright notice, * this list of conditions and the following disclaimer in the documentation * and/or other materials provided with the distribution. * * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE * POSSIBILITY OF SUCH DAMAGE. */ /** * Tests request.required.acks (issue #75) */ #define _GNU_SOURCE #include #include #include "test.h" /* Typical include path would be , but this program * is built from within the librdkafka source tree and thus differs. */ #include "rdkafka.h" /* for Kafka driver */ static int msgid_next = 0; static int fails = 0; /** * Delivery reported callback. * Called for each message once to signal its delivery status. */ static void dr_cb (rd_kafka_t *rk, void *payload, size_t len, rd_kafka_resp_err_t err, void *opaque, void *msg_opaque) { int msgid = *(int *)msg_opaque; free(msg_opaque); if (err != RD_KAFKA_RESP_ERR_NO_ERROR) TEST_FAIL("Message delivery failed: %s\n", rd_kafka_err2str(err)); if (msgid != msgid_next) { fails++; TEST_FAIL("Delivered msg %i, expected %i\n", msgid, msgid_next); return; } msgid_next = msgid+1; } int main (int argc, char **argv) { char *topic = "rdkafkatest1"; int partition = 0; int r; rd_kafka_t *rk; rd_kafka_topic_t *rkt; rd_kafka_conf_t *conf; rd_kafka_topic_conf_t *topic_conf; char errstr[512]; char msg[128]; int msgcnt = 100; int i; int reqacks; int idbase = 0; /* Try different request.required.acks settings (issue #75) */ for (reqacks = -1 ; reqacks <= 2 ; reqacks++) { char tmp[10]; test_conf_init(&conf, &topic_conf, 10); snprintf(tmp, sizeof(tmp), "%i", reqacks); if (rd_kafka_topic_conf_set(topic_conf, "request.required.acks", tmp, errstr, sizeof(errstr)) != RD_KAFKA_CONF_OK) TEST_FAIL("%s", errstr); /* Set delivery report callback */ rd_kafka_conf_set_dr_cb(conf, dr_cb); /* Create kafka instance */ rk = rd_kafka_new(RD_KAFKA_PRODUCER, conf, errstr, sizeof(errstr)); if (!rk) TEST_FAIL("Failed to create rdkafka instance: %s\n", errstr); TEST_SAY("Created kafka instance %s with required acks %i\n", rd_kafka_name(rk), reqacks); rkt = rd_kafka_topic_new(rk, topic, topic_conf); if (!rkt) TEST_FAIL("Failed to create topic: %s\n", strerror(errno)); /* Produce messages */ for (i = 0 ; i < msgcnt ; i++) { int *msgidp = malloc(sizeof(*msgidp)); *msgidp = idbase + i; snprintf(msg, sizeof(msg), "%s test message #%i (acks=%i)", argv[0], *msgidp, reqacks); r = rd_kafka_produce(rkt, partition, RD_KAFKA_MSG_F_COPY, msg, strlen(msg), NULL, 0, msgidp); if (r == -1) TEST_FAIL("Failed to produce message #%i: %s\n", *msgidp, strerror(errno)); } TEST_SAY("Produced %i messages, waiting for deliveries\n", msgcnt); /* Wait for messages to time out */ while (rd_kafka_outq_len(rk) > 0) rd_kafka_poll(rk, 50); if (fails) TEST_FAIL("%i failures, see previous errors", fails); if (msgid_next != idbase + msgcnt) TEST_FAIL("Still waiting for messages: " "next %i != end %i\n", msgid_next, msgcnt); idbase += i; /* Destroy topic */ rd_kafka_topic_destroy(rkt); /* Destroy rdkafka instance */ TEST_SAY("Destroying kafka instance %s\n", rd_kafka_name(rk)); rd_kafka_destroy(rk); } /* Wait for everything to be cleaned up since broker destroys are * handled in its own thread. */ test_wait_exit(10); /* If we havent failed at this point then * there were no threads leaked */ return 0; } librdkafka-0.8.3/tests/1000-unktopic.c000066400000000000000000000116221227620010100173250ustar00rootroot00000000000000/* * librdkafka - Apache Kafka C library * * Copyright (c) 2012-2013, Magnus Edenhill * All rights reserved. * * Redistribution and use in source and binary forms, with or without * modification, are permitted provided that the following conditions are met: * * 1. Redistributions of source code must retain the above copyright notice, * this list of conditions and the following disclaimer. * 2. Redistributions in binary form must reproduce the above copyright notice, * this list of conditions and the following disclaimer in the documentation * and/or other materials provided with the distribution. * * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE * POSSIBILITY OF SUCH DAMAGE. */ /** * Tests that producing to unknown topic fails. * Issue #39 * * NOTE! This test requires auto.create.topics.enable=false to be * configured on the broker! */ #define _GNU_SOURCE #include #include #include "test.h" /* Typical include path would be , but this program * is built from within the librdkafka source tree and thus differs. */ #include "rdkafka.h" /* for Kafka driver */ static int msgs_wait = 0; /* bitmask */ /** * Delivery report callback. * Called for each message once to signal its delivery status. */ static void dr_cb (rd_kafka_t *rk, void *payload, size_t len, rd_kafka_resp_err_t err, void *opaque, void *msg_opaque) { int msgid = *(int *)msg_opaque; free(msg_opaque); if (!(msgs_wait & (1 << msgid))) TEST_FAIL("Unwanted delivery report for message #%i " "(waiting for 0x%x)\n", msgid, msgs_wait); TEST_SAY("Delivery report for message #%i: %s\n", msgid, rd_kafka_err2str(err)); msgs_wait &= ~(1 << msgid); if (err != RD_KAFKA_RESP_ERR__UNKNOWN_TOPIC) TEST_FAIL("Message #%i failed with unexpected error %s\n", msgid, rd_kafka_err2str(err)); } int main (int argc, char **argv) { char topic[64]; int partition = 0; int r; rd_kafka_t *rk; rd_kafka_topic_t *rkt; rd_kafka_conf_t *conf; rd_kafka_topic_conf_t *topic_conf; char errstr[512]; char msg[128]; int msgcnt = 10; int i; /* Generate unique topic name */ test_conf_init(&conf, &topic_conf, 10); snprintf(topic, sizeof(topic), "rdkafkatest1_unk_%x%x", rand(), rand()); TEST_SAY("\033[33mNOTE! This test requires " "auto.create.topics.enable=false to be configured on " "the broker!\033[0m\n"); /* Set delivery report callback */ rd_kafka_conf_set_dr_cb(conf, dr_cb); /* Create kafka instance */ rk = rd_kafka_new(RD_KAFKA_PRODUCER, conf, errstr, sizeof(errstr)); if (!rk) TEST_FAIL("Failed to create rdkafka instance: %s\n", errstr); TEST_SAY("Created kafka instance %s\n", rd_kafka_name(rk)); rkt = rd_kafka_topic_new(rk, topic, topic_conf); if (!rkt) TEST_FAIL("Failed to create topic: %s\n", strerror(errno)); /* Produce a message */ for (i = 0 ; i < msgcnt ; i++) { int *msgidp = malloc(sizeof(*msgidp)); *msgidp = i; snprintf(msg, sizeof(msg), "%s test message #%i", argv[0], i); r = rd_kafka_produce(rkt, partition, RD_KAFKA_MSG_F_COPY, msg, strlen(msg), NULL, 0, msgidp); if (r == -1) { if (errno == ENOENT) TEST_SAY("Failed to produce message #%i: " "unknown topic: good!\n", i); else TEST_FAIL("Failed to produce message #%i: %s\n", i, strerror(errno)); } else { if (i > 5) TEST_FAIL("Message #%i produced: " "should've failed\n", i); msgs_wait |= (1 << i); } /* After half the messages: sleep to allow the metadata * to be fetched from broker and update the actual partition * count: this will make subsequent produce() calls fail * immediately. */ if (i == 5) sleep(2); } /* Wait for messages to time out */ while (rd_kafka_outq_len(rk) > 0) rd_kafka_poll(rk, 50); if (msgs_wait != 0) TEST_FAIL("Still waiting for messages: 0x%x\n", msgs_wait); /* Destroy topic */ rd_kafka_topic_destroy(rkt); /* Destroy rdkafka instance */ TEST_SAY("Destroying kafka instance %s\n", rd_kafka_name(rk)); rd_kafka_destroy(rk); /* Wait for everything to be cleaned up since broker destroys are * handled in its own thread. */ test_wait_exit(10); /* If we havent failed at this point then * there were no threads leaked */ return 0; } librdkafka-0.8.3/tests/Makefile000066400000000000000000000010641227620010100164060ustar00rootroot00000000000000TESTSRCS ?= $(wildcard 0*-*.c) TESTS ?= $(TESTSRCS:%.c=%.test) CC ?= cc CXX ?= g++ CFLAGS += -g -Wall -Werror -Wfloat-equal -Wpointer-arith -O2 -I../ CXXFLAGS += $(CFLAGS) LDFLAGS += -L../ -lrdkafka LDFLAGS += -lpthread -lz ifeq ($(shell uname -s), Linux) LDFLAGS += -lrt endif # Profiling #CFLAGS += -O0 -pg #LDFLAGS += -pg %.test: ../librdkafka.a test.o %.c $(CC) $(CFLAGS) $^ -o $@ $(LDFLAGS) all: $(TESTS) for i in $^ ; do \ LD_LIBRARY_PATH=../ ./run-test.sh $$i || exit 1 ; \ done test.o: test.c $(CC) $(CFLAGS) -c $< clean: rm -f *.test librdkafka-0.8.3/tests/README000066400000000000000000000014051227620010100156250ustar00rootroot00000000000000Automated regression tests for librdkafka ========================================= A local configuration file needs to be created to specify the broker address, and possibly other rdkafka configuration properties: cp test.conf.example test.conf $EDITOR test.conf To run tests: make To run specific test(s): TESTS=0004-conf.test make All tests in the 0000-0999 series are run automatically with 'make'. Tests 1000-1999 are subject to specific non-standard setups or broker configuration, these tests are run with "TESTS=1xxx-yyyyy.test make". See comments in the test's source file for specific requirements. The run-test.sh script can be modified to run tests in one or more of the following modes: * bare (default) * valgrind * helgrind librdkafka-0.8.3/tests/librdkafka.suppressions000066400000000000000000000022051227620010100215350ustar00rootroot00000000000000# Valgrind suppression file for librdkafka { allocate_tls_despite_detached_1 Memcheck:Leak fun:calloc fun:_dl_allocate_tls fun:pthread_create@@GLIBC_2.2.5 fun:rd_kafka_broker_add } { libc_gethostbyname_1 Helgrind:Race fun:_nss_files_gethostbyname2_r fun:gethostbyname2_r@@GLIBC_2.2.5 fun:gaih_inet fun:getaddrinfo fun:rd_getaddrinfo fun:rd_kafka_broker_resolve } { errno_tls_read_1 Helgrind:Race fun:rd_kafka_broker_fail fun:rd_kafka_broker_thread_main obj:/usr/lib/valgrind/vgpreload_helgrind-amd64-linux.so fun:start_thread fun:clone } { rkb_fields_cleared_in_caller_thread_1 drd:ConflictingAccess fun:connect fun:rd_kafka_broker_connect fun:rd_kafka_broker_thread_main obj:/usr/lib/valgrind/vgpreload_drd-amd64-linux.so fun:start_thread fun:clone } { rkb_fields_cleared_in_caller_thread_2 drd:ConflictingAccess fun:sendmsg fun:rd_kafka_broker_send fun:rd_kafka_send fun:rd_kafka_broker_io_serve fun:rd_kafka_broker_serve fun:rd_kafka_broker_thread_main obj:/usr/lib/valgrind/vgpreload_drd-amd64-linux.so fun:start_thread fun:clone } librdkafka-0.8.3/tests/run-test.sh000077500000000000000000000030371227620010100170700ustar00rootroot00000000000000#!/bin/bash # RED='\033[31m' GREEN='\033[32m' CYAN='\033[36m' CCLR='\033[0m' if [ -z "$1" ]; then echo "Usage: $0 <00xx-...test> [modes..]" echo "" echo " Modes: bare valgrind helgrind drd" exit 1 fi TEST=$1 if [ ! -z "$2" ]; then MODES=$2 else MODES="bare" # Enable valgrind: #MODES="bare valgrind" fi FAILED=0 # Enable valgrind suppressions for false positives SUPP="--suppressions=librdkafka.suppressions" # Uncomment to generate valgrind suppressions #GEN_SUPP="--gen-suppressions=yes" # Common valgrind arguments VALGRIND_ARGS="--error-exitcode=3" # Enable vgdb on valgrind errors. #VALGRIND_ARGS="$VALGRIND_ARGS --vgdb-error=1" echo -e "${CYAN}############## $TEST ################${CCLR}" for mode in $MODES; do echo -e "${CYAN}### Running test $TEST in $mode mode ###${CCLR}" case "$mode" in valgrind) valgrind $VALGRIND_ARGS --leak-check=full $SUPP $GEN_SUPP \ ./$TEST RET=$? ;; helgrind) valgrind $VALGRIND_ARGS --tool=helgrind $SUPP $GEN_SUPP \ ./$TEST RET=$? ;; drd) valgrind $VALGRIND_ARGS --tool=drd $SUPP $GEN_SUPP \ ./$TEST RET=$? ;; bare) ./$TEST RET=$? ;; *) echo -e "${RED}### Unknown mode $mode for $TEST ###${CCLR}" RET=1 ;; esac if [ $RET -gt 0 ]; then echo -e "${RED}###" echo -e "### Test $TEST in $mode mode FAILED! ###" echo -e "###${CCLR}" FAILED=1 else echo -e "${GREEN}###" echo -e "### $Test $TEST in $mode mode PASSED! ###" echo -e "###${CCLR}" fi done exit $FAILED librdkafka-0.8.3/tests/test.c000066400000000000000000000077741227620010100161070ustar00rootroot00000000000000/* * librdkafka - Apache Kafka C library * * Copyright (c) 2012-2013, Magnus Edenhill * All rights reserved. * * Redistribution and use in source and binary forms, with or without * modification, are permitted provided that the following conditions are met: * * 1. Redistributions of source code must retain the above copyright notice, * this list of conditions and the following disclaimer. * 2. Redistributions in binary form must reproduce the above copyright notice, * this list of conditions and the following disclaimer in the documentation * and/or other materials provided with the distribution. * * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE * POSSIBILITY OF SUCH DAMAGE. */ #include "test.h" #include /* Typical include path would be , but this program * is built from within the librdkafka source tree and thus differs. */ #include "rdkafka.h" int test_level = 2; int test_seed; static void sig_alarm (int sig) { TEST_FAIL("Test timed out"); } static void test_error_cb (rd_kafka_t *rk, int err, const char *reason, void *opaque) { TEST_FAIL("rdkafka error: %s: %s", rd_kafka_err2str(err), reason); } /** * Creates and sets up kafka configuration objects. * Will read "test.conf" file if it exists. */ void test_conf_init (rd_kafka_conf_t **conf, rd_kafka_topic_conf_t **topic_conf, int timeout) { FILE *fp; char buf[512]; int line = 0; const char *test_conf = getenv("RDKAFKA_TEST_CONF") ? : "test.conf"; char errstr[512]; char *tmp; int seed; /* Limit the test run time. */ alarm(timeout); signal(SIGALRM, sig_alarm); if ((tmp = getenv("TEST_LEVEL"))) test_level = atoi(tmp); if ((tmp = getenv("TEST_SEED"))) seed = atoi(tmp); else seed = test_clock() & 0xffffffff; srand(seed); test_seed = seed; *conf = rd_kafka_conf_new(); *topic_conf = rd_kafka_topic_conf_new(); rd_kafka_conf_set_error_cb(*conf, test_error_cb); /* Open and read optional local test configuration file, if any. */ if (!(fp = fopen(test_conf, "r"))) { if (errno == ENOENT) TEST_FAIL("%s not found\n", test_conf); else TEST_FAIL("Failed to read %s: %s", test_conf, strerror(errno)); } while (fgets(buf, sizeof(buf)-1, fp)) { char *t; char *b = buf; rd_kafka_conf_res_t res; char *name, *val; line++; if ((t = strchr(b, '\n'))) *t = '\0'; if (*b == '#' || !*b) continue; if (!(t = strchr(b, '='))) TEST_FAIL("%s:%i: expected name=value format\n", test_conf, line); name = b; *t = '\0'; val = t+1; if (!strncmp(name, "topic.", strlen("topic."))) { name += strlen("topic."); res = rd_kafka_topic_conf_set(*topic_conf, name, val, errstr, sizeof(errstr)); } else res = rd_kafka_conf_set(*conf, name, val, errstr, sizeof(errstr)); if (res != RD_KAFKA_CONF_OK) TEST_FAIL("%s:%i: %s\n", test_conf, line, errstr); } fclose(fp); } /** * Wait 'timeout' seconds for rdkafka to kill all its threads and clean up. */ void test_wait_exit (int timeout) { int r; while ((r = rd_kafka_thread_cnt()) && timeout-- >= 0) { TEST_SAY("%i thread(s) in use by librdkafka, waiting...\n", r); sleep(1); } TEST_SAY("%i thread(s) in use by librdkafka\n", r); if (r > 0) { assert(0); TEST_FAIL("%i thread(s) still active in librdkafka", r); } } librdkafka-0.8.3/tests/test.conf.example000066400000000000000000000003431227620010100202250ustar00rootroot00000000000000# Copy this file to test.conf and set up according to your configuration. # Bootstrap broker(s) metadata.broker.list=localhost:9092 # Debugging #debug=metadata,topic,msg,broker # Any other librdkafka configuration property. librdkafka-0.8.3/tests/test.h000066400000000000000000000031661227620010100161030ustar00rootroot00000000000000#pragma once #include #include #include #include #include #include #include #include #include "rdkafka.h" /** * Test output is controlled through "TEST_LEVEL=N" environemnt variable. * N < 2: TEST_SAY() is quiet. */ extern int test_level; extern int test_seed; #define TEST_FAIL(reason...) do { \ fprintf(stderr, "### Test failed at %s:%i:%s(): ###\n", \ __FILE__,__LINE__,__FUNCTION__); \ fprintf(stderr, reason); \ fprintf(stderr, "\n"); \ fprintf(stderr, "### Test random seed was %i ###\n", \ test_seed); \ exit(1); \ } while (0) #define TEST_PERROR(call) do { \ if (!(call)) \ TEST_FAIL(#call " failed: %s", strerror(errno)); \ } while (0) #define TEST_SAY(what...) do { \ if (test_level >= 2) \ fprintf(stderr, what); \ } while (0) #define TEST_REPORT(what...) do { \ fprintf(stdout, what); \ } while(0) void test_conf_init (rd_kafka_conf_t **conf, rd_kafka_topic_conf_t **topic_conf, int timeout); void test_wait_exit (int timeout); /** * A microsecond monotonic clock */ static inline int64_t test_clock (void) __attribute__((unused)); static inline int64_t test_clock (void) { #ifdef __APPLE__ /* No monotonic clock on Darwin */ struct timeval tv; gettimeofday(&tv, NULL); return ((int64_t)tv.tv_sec * 1000000LLU) + (int64_t)tv.tv_usec; #else struct timespec ts; clock_gettime(CLOCK_MONOTONIC, &ts); return ((int64_t)ts.tv_sec * 1000000LLU) + ((int64_t)ts.tv_nsec / 1000LLU); #endif }