The Apache Kafka C/C++ library
Перейти к файлу
Magnus Edenhill 0028b345da Create INTRODUCTION.md - explaining the concepts of librdkafka 2013-09-10 23:56:55 +02:00
examples Added snappy and gzip compression support to the producer 2013-09-06 00:00:57 +02:00
.dir-locals.el Extracted rdkafka from librd to its own library. 2012-09-19 12:26:37 +02:00
.gitignore Extracted rdkafka from librd to its own library. 2012-09-19 12:26:37 +02:00
.travis.yml travis-ci.org integration 2013-05-19 22:07:22 +02:00
INTRODUCTION.md Create INTRODUCTION.md - explaining the concepts of librdkafka 2013-09-10 23:56:55 +02:00
LICENSE Extracted rdkafka from librd to its own library. 2012-09-19 12:26:37 +02:00
LICENSE.pycrc Extracted rdkafka from librd to its own library. 2012-09-19 12:26:37 +02:00
LICENSE.snappy Added Andi Kleen's Snappy compression C implementation 2013-09-05 23:44:23 +02:00
Makefile Added Andi Kleen's Snappy compression C implementation 2013-09-05 23:44:23 +02:00
README.md Added snappy and gzip compression support to the producer 2013-09-06 00:00:57 +02:00
librdkafka.lds Initial 0.8 producer support. 2013-06-05 22:22:20 +02:00
rd.c Extracted rdkafka from librd to its own library. 2012-09-19 12:26:37 +02:00
rd.h Extracted rdkafka from librd to its own library. 2012-09-19 12:26:37 +02:00
rdaddr.c Initial 0.8 producer support. 2013-06-05 22:22:20 +02:00
rdaddr.h Make somme fixes: 2013-05-17 22:09:25 +02:00
rdcrc32.c Extracted rdkafka from librd to its own library. 2012-09-19 12:26:37 +02:00
rdcrc32.h Extracted rdkafka from librd to its own library. 2012-09-19 12:26:37 +02:00
rdevent.h Initial 0.8 producer support. 2013-06-05 22:22:20 +02:00
rdgz.c Extracted rdkafka from librd to its own library. 2012-09-19 12:26:37 +02:00
rdgz.h Extracted rdkafka from librd to its own library. 2012-09-19 12:26:37 +02:00
rdkafka.c Some locking fixes 2013-08-26 22:34:23 +02:00
rdkafka.h Proper broker metadata refresh on leader failure. (issue #14) 2013-08-26 22:33:59 +02:00
rdkafka_broker.c Attempt to fill the batch queue before sending messagesets 2013-09-06 00:03:09 +02:00
rdkafka_broker.h Added broker state name printouts to rd_kafka_dump() 2013-08-21 00:20:32 +02:00
rdkafka_defaultconf.c Proper broker metadata refresh on leader failure. (issue #14) 2013-08-26 22:33:59 +02:00
rdkafka_int.h Attempt to fill the batch queue before sending messagesets 2013-09-06 00:03:09 +02:00
rdkafka_msg.c Debugging is now controlled per context 2013-08-26 21:58:19 +02:00
rdkafka_msg.h Initial 0.8 producer support. 2013-06-05 22:22:20 +02:00
rdkafka_proto.h Initial 0.8 producer support. 2013-06-05 22:22:20 +02:00
rdkafka_topic.c Metadata triggered debug messages should be in metadata context. 2013-08-26 22:34:51 +02:00
rdkafka_topic.h Re-delegate all affected topic+partitions when a broker fails (issue #14) 2013-08-21 00:19:51 +02:00
rdkafkacpp.h Add return code for Kafka::produce; Free payload when rkq is full 2013-06-05 10:22:52 +08:00
rdlog.c Initial 0.8 producer support. 2013-06-05 22:22:20 +02:00
rdlog.h Initial 0.8 producer support. 2013-06-05 22:22:20 +02:00
rdqueue.c Initial 0.8 producer support. 2013-06-05 22:22:20 +02:00
rdqueue.h Initial 0.8 producer support. 2013-06-05 22:22:20 +02:00
rdrand.c Extracted rdkafka from librd to its own library. 2012-09-19 12:26:37 +02:00
rdrand.h Extracted rdkafka from librd to its own library. 2012-09-19 12:26:37 +02:00
rdsignal.h Initial 0.8 producer support. 2013-06-05 22:22:20 +02:00
rdsysqueue.h Initial 0.8 producer support. 2013-06-05 22:22:20 +02:00
rdthread.c Initial 0.8 producer support. 2013-06-05 22:22:20 +02:00
rdthread.h Initial 0.8 producer support. 2013-06-05 22:22:20 +02:00
rdtime.h Extracted rdkafka from librd to its own library. 2012-09-19 12:26:37 +02:00
rdtypes.h Extracted rdkafka from librd to its own library. 2012-09-19 12:26:37 +02:00
snappy.c Added Andi Kleen's Snappy compression C implementation 2013-09-05 23:44:23 +02:00
snappy.h Added Andi Kleen's Snappy compression C implementation 2013-09-05 23:44:23 +02:00
snappy_compat.h Added Andi Kleen's Snappy compression C implementation 2013-09-05 23:44:23 +02:00

README.md

librdkafka - Apache Kafka C client library

Copyright (c) 2012-2013, Magnus Edenhill.

https://github.com/edenhill/librdkafka

librdkafka is a C library implementation of the Apache Kafka protocol, containing both Producer and Consumer support. It was designed with high performance (current figures around 150000 msgs/second) and message delivery reliability in mind.

librdkafka is licensed under the 2-clause BSD license.

Apache Kafka 0.8 support:

  • Branch: master
  • Producer: supported
  • Consumer: not yet implemented
  • Compression: snappy and gzip
  • Debian package: work in progress (separate "debian" branch)
  • ZooKeeper: not supported
  • API: not backwards compatible
  • Status: Testing, APIs subject to change.

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

Applications 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_performance.c for an example producer

  #include <librdkafka/rdkafka.h>

  ..

  rd_kafka_t *rk;
  rd_kafka_conf_t conf;
  rd_kafka_topic_t *rkt;
  rd_kafka_topic_conf_t topic_conf;
  char errstr[512];

  /* Base our Kafka configuration on the default configuration */
  rd_kafka_defaultconf_set(&conf);
  conf.error_cb       = error_cb;
  conf.producer.dr_cb = msg_delivered_cb;

  if (!(rk = rd_kafka_new(RD_KAFKA_PRODUCER, &conf,
                          errstr, sizeof(errstr)))) {
        printf("Kafka initialization failed: %s\n", errstr);
        exit(1);
  }

  /* Base our topic configuration on the default topic configuration */
  rd_kafka_topic_defaultconf_set(&topic_conf);
  topic_conf.required_acks      = 2;
  topic_conf.message_timeout_ms = 1000*5*60; /* 5 minutes */

  /* Create local handle for topic "testtopic" */
  if (!(rkt = rd_kafka_topic_new(rk, "testtopic", &topic_conf))) {
   	    printf("Failed to add topic: %s\n", strerror(errno));
    exit(1);
  }

  /* Add list of initial brokers. All brokers will eventually be
   * discovered by quering these brokers for the full list of brokers. */
  if (!rd_kafka_brokers_add(rk, "localhost,remotehost1,remotehost2:9099")) {
        printf("No valid brokers specified\n");
        exit(1);
  }

  ...

  /* Produce message */
  if (rd_kafka_produce(rkt, RD_KAFKA_PARTITION_UA /* random partition */,
                       RD_KAFKA_MSG_F_COPY,
                       mydata, mydata_len,
                       mykey, mykey_len,
                       per_message_opaque) == -1) {
        /* Try again in a short while if queue is full, or drop,
     * decision is left to the application. /
        if (errno == ENOBUFS)
           ... retry message later ...
  }


  ...

  /* Serve kafka events (error and delivery report callbacks) */
  rd_kafka_poll(rk, 1000/*ms*/);
  
  ...

  /* Decommission kafka handle */
  rd_kafka_destroy(rk);

Link your program with -lrdkafka -lz -lpthread -lrt.

Documentation

The API is documented in rdkafka.h

Examples

See the examples/sub-directory.