static PyObject *Consumer_get_watermark_offsets (Handle *self, PyObject *args, PyObject *kwargs) { TopicPartition *tp; rd_kafka_resp_err_t err; double tmout = -1.0f; int cached = 0; int64_t low = RD_KAFKA_OFFSET_INVALID, high = RD_KAFKA_OFFSET_INVALID; static char *kws[] = { "partition", "timeout", "cached", NULL }; PyObject *rtup; if (!self->rk) { PyErr_SetString(PyExc_RuntimeError, "Consumer closed"); return NULL; } if (!PyArg_ParseTupleAndKeywords(args, kwargs, "O|db", kws, (PyObject **)&tp, &tmout, &cached)) return NULL; if (PyObject_Type((PyObject *)tp) != (PyObject *)&TopicPartitionType) { PyErr_Format(PyExc_TypeError, "expected %s", TopicPartitionType.tp_name); return NULL; } if (cached) { err = rd_kafka_get_watermark_offsets(self->rk, tp->topic, tp->partition, &low, &high); } else { Py_BEGIN_ALLOW_THREADS; err = rd_kafka_query_watermark_offsets(self->rk, tp->topic, tp->partition, &low, &high, tmout >= 0 ? (int)(tmout * 1000.0f) : -1); Py_END_ALLOW_THREADS; } if (err) { cfl_PyErr_Format(err, "Failed to get watermark offsets: %s", rd_kafka_err2str(err)); return NULL; } rtup = PyTuple_New(2); PyTuple_SetItem(rtup, 0, PyLong_FromLongLong(low)); PyTuple_SetItem(rtup, 1, PyLong_FromLongLong(high)); return rtup; }
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]; int64_t start_offset = 0; int report_offsets = 0; int do_conf_dump = 0; char tmp[16]; int64_t seek_offset = 0; int64_t tmp_offset = 0; int get_wmarks = 0; /* Kafka configuration */ conf = rd_kafka_conf_new(); /* Set logger */ rd_kafka_conf_set_log_cb(conf, logger); /* Quick termination */ snprintf(tmp, sizeof(tmp), "%i", SIGIO); rd_kafka_conf_set(conf, "internal.termination.signal", tmp, NULL, 0); /* Topic configuration */ topic_conf = rd_kafka_topic_conf_new(); while ((opt = getopt(argc, argv, "PCLt:p:b:z:qd:o:eX:As:")) != -1) { switch (opt) { case 'P': case 'C': case 'L': 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': case 's': if (!strcmp(optarg, "end")) tmp_offset = RD_KAFKA_OFFSET_END; else if (!strcmp(optarg, "beginning")) tmp_offset = RD_KAFKA_OFFSET_BEGINNING; else if (!strcmp(optarg, "stored")) tmp_offset = RD_KAFKA_OFFSET_STORED; else if (!strcmp(optarg, "report")) report_offsets = 1; else if (!strcmp(optarg, "wmark")) get_wmarks = 1; else { tmp_offset = strtoll(optarg, NULL, 10); if (tmp_offset < 0) tmp_offset = RD_KAFKA_OFFSET_TAIL(-tmp_offset); } if (opt == 'o') start_offset = tmp_offset; else if (opt == 's') seek_offset = tmp_offset; break; case 'e': exit_eof = 1; break; case 'd': if (rd_kafka_conf_set(conf, "debug", optarg, errstr, sizeof(errstr)) != RD_KAFKA_CONF_OK) { fprintf(stderr, "%% Debug configuration failed: " "%s: %s\n", errstr, optarg); exit(1); } 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, '='))) { char dest[512]; size_t dest_size = sizeof(dest); /* Return current value for property. */ res = RD_KAFKA_CONF_UNKNOWN; if (!strncmp(name, "topic.", strlen("topic."))) res = rd_kafka_topic_conf_get( topic_conf, name+strlen("topic."), dest, &dest_size); if (res == RD_KAFKA_CONF_UNKNOWN) res = rd_kafka_conf_get( conf, name, dest, &dest_size); if (res == RD_KAFKA_CONF_OK) { printf("%s = %s\n", name, dest); exit(0); } else { fprintf(stderr, "%% %s property\n", res == RD_KAFKA_CONF_UNKNOWN ? "Unknown" : "Invalid"); 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 < (int)cnt ; i += 2) printf("%s = %s\n", arr[i], arr[i+1]); printf("\n"); rd_kafka_conf_dump_free(arr, cnt); } exit(0); } if (optind != argc || (mode != 'L' && !topic)) { usage: fprintf(stderr, "Usage: %s -C|-P|-L -t <topic> " "[-p <partition>] [-b <host1:port1,host2:port2,..>]\n" "\n" "librdkafka version %s (0x%08x)\n" "\n" " Options:\n" " -C | -P Consumer or Producer mode\n" " -L Metadata list mode\n" " -t <topic> Topic to fetch / produce\n" " -p <num> Partition (random partitioner)\n" " -b <brokers> Broker address (localhost:9092)\n" " -z <codec> Enable compression:\n" " none|gzip|snappy\n" " -o <offset> Start offset (consumer):\n" " beginning, end, NNNNN or -NNNNN\n" " wmark returns the current hi&lo " "watermarks.\n" " -o report Report message offsets (producer)\n" " -e Exit consumer when last message\n" " in partition has been received.\n" " -d [facs..] Enable debugging contexts:\n" " %s\n" " -q Be quiet\n" " -A Raw payload output (consumer)\n" " -X <prop=name> Set arbitrary librdkafka " "configuration property\n" " Properties prefixed with \"topic.\" " "will be set on topic object.\n" " -X list Show full list of supported " "properties.\n" " -X <prop> Get single property value\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" " In List mode:\n" " queries broker for metadata information, " "topic is optional.\n" "\n" "\n" "\n", argv[0], rd_kafka_version_str(), rd_kafka_version(), RD_KAFKA_DEBUG_CONTEXTS); exit(1); } if ((mode == 'C' && !isatty(STDIN_FILENO)) || (mode != 'C' && !isatty(STDOUT_FILENO))) quiet = 1; signal(SIGINT, stop); signal(SIGUSR1, sig_usr1); 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. */ /* If offset reporting (-o report) is enabled, use the * richer dr_msg_cb instead. */ if (report_offsets) { rd_kafka_topic_conf_set(topic_conf, "produce.offset.report", "true", errstr, sizeof(errstr)); rd_kafka_conf_set_dr_msg_cb(conf, msg_delivered2); } else 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); } /* 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); topic_conf = NULL; /* Now owned by topic */ 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_last_error())); /* 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 topic */ rd_kafka_topic_destroy(rkt); /* 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); } /* Add brokers */ if (rd_kafka_brokers_add(rk, brokers) == 0) { fprintf(stderr, "%% No valid brokers specified\n"); exit(1); } if (get_wmarks) { int64_t lo, hi; rd_kafka_resp_err_t err; /* Only query for hi&lo partition watermarks */ if ((err = rd_kafka_query_watermark_offsets( rk, topic, partition, &lo, &hi, 5000))) { fprintf(stderr, "%% query_watermark_offsets() " "failed: %s\n", rd_kafka_err2str(err)); exit(1); } printf("%s [%d]: low - high offsets: " "%"PRId64" - %"PRId64"\n", topic, partition, lo, hi); rd_kafka_destroy(rk); exit(0); } /* Create topic */ rkt = rd_kafka_topic_new(rk, topic, topic_conf); topic_conf = NULL; /* Now owned by topic */ /* Start consuming */ if (rd_kafka_consume_start(rkt, partition, start_offset) == -1){ rd_kafka_resp_err_t err = rd_kafka_last_error(); fprintf(stderr, "%% Failed to start consuming: %s\n", rd_kafka_err2str(err)); if (err == RD_KAFKA_RESP_ERR__INVALID_ARG) fprintf(stderr, "%% Broker based offset storage " "requires a group.id, " "add: -X group.id=yourGroup\n"); exit(1); } while (run) { rd_kafka_message_t *rkmessage; rd_kafka_resp_err_t err; /* Poll for errors, etc. */ rd_kafka_poll(rk, 0); /* 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); if (seek_offset) { err = rd_kafka_seek(rkt, partition, seek_offset, 2000); if (err) printf("Seek failed: %s\n", rd_kafka_err2str(err)); else printf("Seeked to %"PRId64"\n", seek_offset); seek_offset = 0; } } /* Stop consuming */ rd_kafka_consume_stop(rkt, partition); while (rd_kafka_outq_len(rk) > 0) rd_kafka_poll(rk, 10); /* Destroy topic */ rd_kafka_topic_destroy(rkt); /* Destroy handle */ rd_kafka_destroy(rk); } else if (mode == 'L') { rd_kafka_resp_err_t err = RD_KAFKA_RESP_ERR_NO_ERROR; /* 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); } /* Add brokers */ if (rd_kafka_brokers_add(rk, brokers) == 0) { fprintf(stderr, "%% No valid brokers specified\n"); exit(1); } /* Create topic */ if (topic) { rkt = rd_kafka_topic_new(rk, topic, topic_conf); topic_conf = NULL; /* Now owned by topic */ } else rkt = NULL; while (run) { const struct rd_kafka_metadata *metadata; /* Fetch metadata */ err = rd_kafka_metadata(rk, rkt ? 0 : 1, rkt, &metadata, 5000); if (err != RD_KAFKA_RESP_ERR_NO_ERROR) { fprintf(stderr, "%% Failed to acquire metadata: %s\n", rd_kafka_err2str(err)); run = 0; break; } metadata_print(topic, metadata); rd_kafka_metadata_destroy(metadata); run = 0; } /* Destroy topic */ if (rkt) rd_kafka_topic_destroy(rkt); /* Destroy the handle */ rd_kafka_destroy(rk); if (topic_conf) rd_kafka_topic_conf_destroy(topic_conf); /* Exit right away, dont wait for background cleanup, we haven't * done anything important anyway. */ exit(err ? 2 : 0); } if (topic_conf) rd_kafka_topic_conf_destroy(topic_conf); /* Let background threads clean up and terminate cleanly. */ run = 5; while (run-- > 0 && rd_kafka_wait_destroyed(1000) == -1) printf("Waiting for librdkafka to decommission\n"); if (run <= 0) rd_kafka_dump(stdout, rk); return 0; }
int main_0006_symbols (int argc, char **argv) { if (argc < 0 /* always false */) { rd_kafka_version(); rd_kafka_version_str(); rd_kafka_get_debug_contexts(); rd_kafka_get_err_descs(NULL, NULL); rd_kafka_err2str(RD_KAFKA_RESP_ERR_NO_ERROR); rd_kafka_err2name(RD_KAFKA_RESP_ERR_NO_ERROR); rd_kafka_errno2err(EINVAL); rd_kafka_errno(); rd_kafka_last_error(); 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_dr_msg_cb(NULL, NULL); rd_kafka_conf_set_error_cb(NULL, NULL); rd_kafka_conf_set_stats_cb(NULL, NULL); rd_kafka_conf_set_log_cb(NULL, NULL); rd_kafka_conf_set_socket_cb(NULL, NULL); rd_kafka_conf_set_rebalance_cb(NULL, NULL); rd_kafka_conf_set_offset_commit_cb(NULL, NULL); rd_kafka_conf_set_throttle_cb(NULL, NULL); rd_kafka_conf_set_default_topic_conf(NULL, NULL); rd_kafka_conf_get(NULL, NULL, NULL, NULL); #ifndef _MSC_VER rd_kafka_conf_set_open_cb(NULL, NULL); #endif rd_kafka_conf_set_opaque(NULL, NULL); rd_kafka_opaque(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_get(NULL, NULL, NULL, NULL); rd_kafka_topic_conf_set_partitioner_cb(NULL, NULL); rd_kafka_topic_partition_available(NULL, 0); rd_kafka_topic_opaque(NULL); rd_kafka_msg_partitioner_random(NULL, NULL, 0, 0, NULL, NULL); rd_kafka_msg_partitioner_consistent(NULL, NULL, 0, 0, NULL, NULL); rd_kafka_msg_partitioner_consistent_random(NULL, NULL, 0, 0, NULL, NULL); rd_kafka_new(0, NULL, NULL, 0); rd_kafka_destroy(NULL); rd_kafka_name(NULL); rd_kafka_memberid(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_message_timestamp(NULL, 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_produce_batch(NULL, 0, 0, NULL, 0); rd_kafka_poll(NULL, 0); rd_kafka_brokers_add(NULL, NULL); /* DEPRECATED: rd_kafka_set_logger(NULL, NULL); */ rd_kafka_set_log_level(NULL, 0); rd_kafka_log_print(NULL, 0, NULL, NULL); #ifndef _MSC_VER rd_kafka_log_syslog(NULL, 0, NULL, NULL); #endif rd_kafka_outq_len(NULL); rd_kafka_dump(NULL, NULL); rd_kafka_thread_cnt(); rd_kafka_wait_destroyed(0); rd_kafka_metadata(NULL, 0, NULL, NULL, 0); rd_kafka_metadata_destroy(NULL); rd_kafka_queue_destroy(NULL); rd_kafka_consume_start_queue(NULL, 0, 0, NULL); rd_kafka_consume_queue(NULL, 0); rd_kafka_consume_batch_queue(NULL, 0, NULL, 0); rd_kafka_consume_callback_queue(NULL, 0, NULL, NULL); rd_kafka_seek(NULL, 0, 0, 0); rd_kafka_yield(NULL); rd_kafka_mem_free(NULL, NULL); rd_kafka_list_groups(NULL, NULL, NULL, 0); rd_kafka_group_list_destroy(NULL); /* KafkaConsumer API */ rd_kafka_subscribe(NULL, NULL); rd_kafka_unsubscribe(NULL); rd_kafka_subscription(NULL, NULL); rd_kafka_consumer_poll(NULL, 0); rd_kafka_consumer_close(NULL); rd_kafka_assign(NULL, NULL); rd_kafka_assignment(NULL, NULL); rd_kafka_commit(NULL, NULL, 0); rd_kafka_commit_message(NULL, NULL, 0); rd_kafka_committed(NULL, NULL, 0); rd_kafka_position(NULL, NULL); /* TopicPartition */ rd_kafka_topic_partition_list_new(0); rd_kafka_topic_partition_list_destroy(NULL); rd_kafka_topic_partition_list_add(NULL, NULL, 0); rd_kafka_topic_partition_list_add_range(NULL, NULL, 0, 0); rd_kafka_topic_partition_list_del(NULL, NULL, 0); rd_kafka_topic_partition_list_del_by_idx(NULL, 0); rd_kafka_topic_partition_list_copy(NULL); rd_kafka_topic_partition_list_set_offset(NULL, NULL, 0, 0); rd_kafka_topic_partition_list_find(NULL, NULL, 0); rd_kafka_query_watermark_offsets(NULL, NULL, 0, NULL, NULL, 0); rd_kafka_get_watermark_offsets(NULL, NULL, 0, NULL, NULL); } return 0; }
int main_0031_get_offsets (int argc, char **argv) { const char *topic = test_mk_topic_name(__FUNCTION__, 1); const int msgcnt = 100; rd_kafka_t *rk; rd_kafka_topic_t *rkt; int64_t qry_low = -1234, qry_high = -1235; int64_t get_low = -1234, get_high = -1235; rd_kafka_resp_err_t err; test_timing_t t_qry, t_get; uint64_t testid; /* Produce messages */ testid = test_produce_msgs_easy(topic, 0, 0, msgcnt); /* Get offsets */ rk = test_create_consumer(NULL, NULL, NULL, NULL ); TIMING_START(&t_qry, "query_watermark_offsets"); err = rd_kafka_query_watermark_offsets(rk, topic, 0, &qry_low, &qry_high, 10*1000); TIMING_STOP(&t_qry); if (err) TEST_FAIL("query_watermark_offsets failed: %s\n", rd_kafka_err2str(err)); if (qry_low != 0 && qry_high != msgcnt) TEST_FAIL("Expected low,high %d,%d, but got " "%"PRId64",%"PRId64, 0, msgcnt, qry_low, qry_high); TEST_SAY("query_watermark_offsets: " "offsets %"PRId64", %"PRId64"\n", qry_low, qry_high); /* Now start consuming to update the offset cache, then query it * with the get_ API. */ rkt = test_create_topic_object(rk, topic, NULL); test_consumer_start("get", rkt, 0, RD_KAFKA_OFFSET_BEGINNING); test_consume_msgs("get", rkt, testid, 0, TEST_NO_SEEK, 0, msgcnt, 0); /* After at least one message has been consumed the * watermarks are cached. */ TIMING_START(&t_get, "get_watermark_offsets"); err = rd_kafka_get_watermark_offsets(rk, topic, 0, &get_low, &get_high); TIMING_STOP(&t_get); if (err) TEST_FAIL("get_watermark_offsets failed: %s\n", rd_kafka_err2str(err)); TEST_SAY("get_watermark_offsets: " "offsets %"PRId64", %"PRId64"\n", get_low, get_high); if (get_high != qry_high) TEST_FAIL("query/get discrepancies: " "low: %"PRId64"/%"PRId64", high: %"PRId64"/%"PRId64, qry_low, get_low, qry_high, get_high); if (get_low >= get_high) TEST_FAIL("get_watermark_offsets: " "low %"PRId64" >= high %"PRId64, get_low, get_high); /* FIXME: We currently dont bother checking the get_low offset * since it requires stats to be enabled. */ test_consumer_stop("get", rkt, 0); rd_kafka_topic_destroy(rkt); rd_kafka_destroy(rk); return 0; }