int cnd_timedwait_msp (cnd_t *cnd, mtx_t *mtx, int *timeout_msp) { rd_ts_t pre = rd_clock(); int r; r = cnd_timedwait_ms(cnd, mtx, *timeout_msp); if (r != thrd_timedout) { /* Subtract spent time */ (*timeout_msp) -= (int)(rd_clock()-pre) / 1000; } return r; }
/** * 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, int32_t version) { rd_kafka_op_t *rko; if (timeout_ms == RD_POLL_INFINITE) timeout_ms = INT_MAX; mtx_lock(&rkq->rkq_lock); if (!rkq->rkq_fwdq) { do { /* Filter out outdated ops */ while ((rko = TAILQ_FIRST(&rkq->rkq_q)) && !(rko = rd_kafka_op_filter(rkq, rko))) ; if (rko) { /* Proper versioned op */ rd_kafka_q_deq0(rkq, rko); break; } /* No op, wait for one */ rd_ts_t pre = rd_clock(); if (cnd_timedwait_ms(&rkq->rkq_cond, &rkq->rkq_lock, timeout_ms) == thrd_timedout) { mtx_unlock(&rkq->rkq_lock); return NULL; } /* Remove spent time */ timeout_ms -= (int) (rd_clock()-pre) / 1000; if (timeout_ms < 0) timeout_ms = RD_POLL_NOWAIT; } while (timeout_ms != RD_POLL_NOWAIT); mtx_unlock(&rkq->rkq_lock); } else { rd_kafka_q_t *fwdq = rkq->rkq_fwdq; rd_kafka_q_keep(fwdq); /* Since the q_pop may block we need to release the parent * queue's lock. */ mtx_unlock(&rkq->rkq_lock); rko = rd_kafka_q_pop(fwdq, timeout_ms, version); rd_kafka_q_destroy(fwdq); } return rko; }
int aescycles(){ __int64 start, end; int t; do{ rd_clock(&start); Encrypt(in, expkey, out); rd_clock(&end); t = (end - start); }while(t<=0 || t>=4000); return t; }
/** * Dispatch timers. * Will block up to 'timeout' microseconds before returning. */ void rd_kafka_timers_run (rd_kafka_timers_t *rkts, int timeout_us) { rd_ts_t now = rd_clock(); rd_ts_t end = now + timeout_us; rd_kafka_timers_lock(rkts); while (!rd_atomic32_get(&rkts->rkts_rk->rk_terminate) && now <= end) { int64_t sleeptime; rd_kafka_timer_t *rtmr; if (timeout_us != RD_POLL_NOWAIT) { sleeptime = rd_kafka_timers_next(rkts, timeout_us, 0/*no-lock*/); if (sleeptime > 0) { cnd_timedwait_ms(&rkts->rkts_cond, &rkts->rkts_lock, (int)(sleeptime / 1000)); } } now = rd_clock(); while ((rtmr = TAILQ_FIRST(&rkts->rkts_timers)) && rtmr->rtmr_next <= now) { rd_kafka_timer_unschedule(rkts, rtmr); rd_kafka_timers_unlock(rkts); rtmr->rtmr_callback(rkts, rtmr->rtmr_arg); rd_kafka_timers_lock(rkts); /* Restart timer, unless it has been stopped, or * already reschedueld (start()ed) from callback. */ if (rd_kafka_timer_started(rtmr) && !rd_kafka_timer_scheduled(rtmr)) rd_kafka_timer_schedule(rkts, rtmr, 0); } if (timeout_us == RD_POLL_NOWAIT) { /* Only iterate once, even if rd_clock doesn't change */ break; } } rd_kafka_timers_unlock(rkts); }
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 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 delivey 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); } }
int main (int argc, char **argv) { int fails = 0; srand(rd_clock()); fails += test_shuffle(); return fails ? 1 : 0; }
rd_avg_res_t rd_avg (rd_avg_t *ra, int period) { rd_avg_period_t *p; if (period == -1) period = ra->ra_curri; else if (period == -2) { if (ra->ra_curri == 0) period = ra->ra_periods-1; else period = (ra->ra_curri - 1) % ra->ra_periods; } p = &ra->ra_period[period]; if (p == ra->ra_curr) p->duration = (p->last ? : rd_clock()) - ra->ra_start; return *(rd_avg_calc(ra, p)); }
static void rd_kafka_timer_schedule (rd_kafka_timers_t *rkts, rd_kafka_timer_t *rtmr, int extra_us) { rd_kafka_timer_t *first; /* Timer has been stopped */ if (!rtmr->rtmr_interval) return; rtmr->rtmr_next = rd_clock() + rtmr->rtmr_interval + extra_us; if (!(first = TAILQ_FIRST(&rkts->rkts_timers)) || first->rtmr_next > rtmr->rtmr_next) { TAILQ_INSERT_HEAD(&rkts->rkts_timers, rtmr, rtmr_link); cnd_signal(&rkts->rkts_cond); } else TAILQ_INSERT_SORTED(&rkts->rkts_timers, rtmr, rd_kafka_timer_s, rtmr_link, rd_kafka_timer_cmp); }
/** * @returns the delta time to the next time (>=0) this timer fires, or -1 * if timer is stopped. */ rd_ts_t rd_kafka_timer_next (rd_kafka_timers_t *rkts, rd_kafka_timer_t *rtmr, int do_lock) { rd_ts_t now = rd_clock(); rd_ts_t delta = -1; if (do_lock) rd_kafka_timers_lock(rkts); if (rd_kafka_timer_scheduled(rtmr)) { delta = rtmr->rtmr_next - now; if (delta < 0) delta = 0; } if (do_lock) rd_kafka_timers_unlock(rkts); return delta; }
void rd_avg_put (rd_avg_t *ra, uint64_t val) { rd_ts_t now = rd_clock(); /* Check if current period has ended */ if (ra->ra_end <= now) rd_avg_roll(ra, now); ra->ra_curr->last = now; if (ra->ra_curr->res.high < val) ra->ra_curr->res.high = val; if (ra->ra_curr->res.low > val) ra->ra_curr->res.low = val; switch (ra->ra_type) { case RD_AVG_RATE: return rd_avg_put_rate(ra, val, now); case RD_AVG_HIST: return rd_avg_put_hist(ra, val, now); } }
/** * Returns the delta time to the next timer to fire, capped by 'timeout_ms'. */ rd_ts_t rd_kafka_timers_next (rd_kafka_timers_t *rkts, int timeout_us, int do_lock) { rd_ts_t now = rd_clock(); rd_ts_t sleeptime = 0; rd_kafka_timer_t *rtmr; if (do_lock) rd_kafka_timers_lock(rkts); if (likely((rtmr = TAILQ_FIRST(&rkts->rkts_timers)) != NULL)) { sleeptime = rtmr->rtmr_next - now; if (sleeptime < 0) sleeptime = 0; else if (sleeptime > (rd_ts_t)timeout_us) sleeptime = (rd_ts_t)timeout_us; } else sleeptime = (rd_ts_t)timeout_us; if (do_lock) rd_kafka_timers_unlock(rkts); return sleeptime; }
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; /* 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:q")) != -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': 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; default: goto usage; } } if (!topic || optind != argc) { usage: fprintf(stderr, "Usage: %s [-C|-P] -t <topic> " "[-p <partition>] [-b <broker,broker..>] [options..]\n" "\n" " Options:\n" " -C | -P Consumer or Producer mode\n" " -t <topic> Topic to fetch / produce\n" " -p <num> Partition (defaults to random)\n" " -b <brokers> Broker address list (host[:port],..)\n" " -s <size> Message size (producer)\n" " -k <key> Message key (producer)\n" " -c <cnt> Messages to transmit/receive\n" " -D Copy/Duplicate data buffer (producer)\n" " -i <ms> Display interval\n" " -m <msg> Message payload pattern\n" " -S <start> Send a sequence number starting at " "<start> as payload\n" " -R <seed> Random seed value (defaults to time)\n" " -a <acks> Required acks (producer): " "-1, 0, 1, >1\n" " -B <size> Consume batch size (# of msgs)\n" " -z <codec> Enable compression:\n" " none|gzip|snappy\n" " -o <offset> Start offset (consumer)\n" " -d [facs..] Enable debugging contexts:\n" " %s\n" " -X <prop=name> 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 <intvl> Enable statistics from librdkafka at " "specified interval (ms)\n" " -q Be more quiet\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_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); } /* Socket hangups are gracefully handled in librdkafka on socket error * without the use of signals, so SIGPIPE should be ignored by the * calling program. */ signal(SIGPIPE, SIG_IGN); 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 (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 (!quiet || errno != ENOBUFS) printf("produce error: %s%s\n", strerror(errno), errno == ENOBUFS ? " (backpressure)":""); 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); 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); 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 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); } /* 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", strerror(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", strerror(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; }
/** * @brief Build client-final-message * @returns -1 on error. */ static int rd_kafka_sasl_scram_build_client_final_message ( rd_kafka_transport_t *rktrans, const rd_chariov_t *salt, const char *server_nonce, const rd_chariov_t *server_first_msg, int itcnt, rd_chariov_t *out) { struct rd_kafka_sasl_scram_state *state = rktrans->rktrans_sasl.state; const rd_kafka_conf_t *conf = &rktrans->rktrans_rkb->rkb_rk->rk_conf; rd_chariov_t SaslPassword = { .ptr = conf->sasl.password, .size = strlen(conf->sasl.password) }; rd_chariov_t SaltedPassword = { .ptr = rd_alloca(EVP_MAX_MD_SIZE) }; rd_chariov_t ClientKey = { .ptr = rd_alloca(EVP_MAX_MD_SIZE) }; rd_chariov_t ServerKey = { .ptr = rd_alloca(EVP_MAX_MD_SIZE) }; rd_chariov_t StoredKey = { .ptr = rd_alloca(EVP_MAX_MD_SIZE) }; rd_chariov_t AuthMessage = RD_ZERO_INIT; rd_chariov_t ClientSignature = { .ptr = rd_alloca(EVP_MAX_MD_SIZE) }; rd_chariov_t ServerSignature = { .ptr = rd_alloca(EVP_MAX_MD_SIZE) }; const rd_chariov_t ClientKeyVerbatim = { .ptr = "Client Key", .size = 10 }; const rd_chariov_t ServerKeyVerbatim = { .ptr = "Server Key", .size = 10 }; rd_chariov_t ClientProof = { .ptr = rd_alloca(EVP_MAX_MD_SIZE) }; rd_chariov_t client_final_msg_wo_proof; char *ClientProofB64; int i; /* Constructing the ClientProof attribute (p): * * p = Base64-encoded ClientProof * SaltedPassword := Hi(Normalize(password), salt, i) * ClientKey := HMAC(SaltedPassword, "Client Key") * StoredKey := H(ClientKey) * AuthMessage := client-first-message-bare + "," + * server-first-message + "," + * client-final-message-without-proof * ClientSignature := HMAC(StoredKey, AuthMessage) * ClientProof := ClientKey XOR ClientSignature * ServerKey := HMAC(SaltedPassword, "Server Key") * ServerSignature := HMAC(ServerKey, AuthMessage) */ /* SaltedPassword := Hi(Normalize(password), salt, i) */ if (rd_kafka_sasl_scram_Hi( rktrans, &SaslPassword, salt, itcnt, &SaltedPassword) == -1) return -1; /* ClientKey := HMAC(SaltedPassword, "Client Key") */ if (rd_kafka_sasl_scram_HMAC( rktrans, &SaltedPassword, &ClientKeyVerbatim, &ClientKey) == -1) return -1; /* StoredKey := H(ClientKey) */ if (rd_kafka_sasl_scram_H(rktrans, &ClientKey, &StoredKey) == -1) return -1; /* client-final-message-without-proof */ rd_kafka_sasl_scram_build_client_final_message_wo_proof( state, server_nonce, &client_final_msg_wo_proof); /* AuthMessage := client-first-message-bare + "," + * server-first-message + "," + * client-final-message-without-proof */ AuthMessage.size = state->first_msg_bare.size + 1 + server_first_msg->size + 1 + client_final_msg_wo_proof.size; AuthMessage.ptr = rd_alloca(AuthMessage.size+1); rd_snprintf(AuthMessage.ptr, AuthMessage.size+1, "%.*s,%.*s,%.*s", (int)state->first_msg_bare.size, state->first_msg_bare.ptr, (int)server_first_msg->size, server_first_msg->ptr, (int)client_final_msg_wo_proof.size, client_final_msg_wo_proof.ptr); /* * Calculate ServerSignature for later verification when * server-final-message is received. */ /* ServerKey := HMAC(SaltedPassword, "Server Key") */ if (rd_kafka_sasl_scram_HMAC( rktrans, &SaltedPassword, &ServerKeyVerbatim, &ServerKey) == -1) { rd_free(client_final_msg_wo_proof.ptr); return -1; } /* ServerSignature := HMAC(ServerKey, AuthMessage) */ if (rd_kafka_sasl_scram_HMAC(rktrans, &ServerKey, &AuthMessage, &ServerSignature) == -1) { rd_free(client_final_msg_wo_proof.ptr); return -1; } /* Store the Base64 encoded ServerSignature for quick comparison */ state->ServerSignatureB64 = rd_base64_encode(&ServerSignature); /* * Continue with client-final-message */ /* ClientSignature := HMAC(StoredKey, AuthMessage) */ if (rd_kafka_sasl_scram_HMAC(rktrans, &StoredKey, &AuthMessage, &ClientSignature) == -1) { rd_free(client_final_msg_wo_proof.ptr); return -1; } /* ClientProof := ClientKey XOR ClientSignature */ assert(ClientKey.size == ClientSignature.size); for (i = 0 ; i < (int)ClientKey.size ; i++) ClientProof.ptr[i] = ClientKey.ptr[i] ^ ClientSignature.ptr[i]; ClientProof.size = ClientKey.size; /* Base64 encoded ClientProof */ ClientProofB64 = rd_base64_encode(&ClientProof); /* Construct client-final-message */ out->size = client_final_msg_wo_proof.size + strlen(",p=") + strlen(ClientProofB64); out->ptr = rd_malloc(out->size + 1); rd_snprintf(out->ptr, out->size+1, "%.*s,p=%s", (int)client_final_msg_wo_proof.size, client_final_msg_wo_proof.ptr, ClientProofB64); rd_free(ClientProofB64); rd_free(client_final_msg_wo_proof.ptr); return 0; } /** * @brief Handle first message from server * * Parse server response which looks something like: * "r=fyko+d2lbbFgONR....,s=QSXCR+Q6sek8bf92,i=4096" * * @returns -1 on error. */ static int rd_kafka_sasl_scram_handle_server_first_message (rd_kafka_transport_t *rktrans, const rd_chariov_t *in, rd_chariov_t *out, char *errstr, size_t errstr_size) { struct rd_kafka_sasl_scram_state *state = rktrans->rktrans_sasl.state; char *server_nonce; rd_chariov_t salt_b64, salt; char *itcntstr; const char *endptr; int itcnt; char *attr_m; /* Mandatory future extension check */ if ((attr_m = rd_kafka_sasl_scram_get_attr( in, 'm', NULL, NULL, 0))) { rd_snprintf(errstr, errstr_size, "Unsupported mandatory SCRAM extension"); rd_free(attr_m); return -1; } /* Server nonce */ if (!(server_nonce = rd_kafka_sasl_scram_get_attr( in, 'r', "Server nonce in server-first-message", errstr, errstr_size))) return -1; if (strlen(server_nonce) <= state->cnonce.size || strncmp(state->cnonce.ptr, server_nonce, state->cnonce.size)) { rd_snprintf(errstr, errstr_size, "Server/client nonce mismatch in " "server-first-message"); rd_free(server_nonce); return -1; } /* Salt (Base64) */ if (!(salt_b64.ptr = rd_kafka_sasl_scram_get_attr( in, 's', "Salt in server-first-message", errstr, errstr_size))) { rd_free(server_nonce); return -1; } salt_b64.size = strlen(salt_b64.ptr); /* Convert Salt to binary */ if (rd_base64_decode(&salt_b64, &salt) == -1) { rd_snprintf(errstr, errstr_size, "Invalid Base64 Salt in server-first-message"); rd_free(server_nonce); rd_free(salt_b64.ptr); } rd_free(salt_b64.ptr); /* Iteration count (as string) */ if (!(itcntstr = rd_kafka_sasl_scram_get_attr( in, 'i', "Iteration count in server-first-message", errstr, errstr_size))) { rd_free(server_nonce); rd_free(salt.ptr); return -1; } /* Iteration count (as int) */ errno = 0; itcnt = (int)strtoul(itcntstr, (char **)&endptr, 10); if (itcntstr == endptr || *endptr != '\0' || errno != 0 || itcnt > 1000000) { rd_snprintf(errstr, errstr_size, "Invalid value (not integer or too large) " "for Iteration count in server-first-message"); rd_free(server_nonce); rd_free(salt.ptr); rd_free(itcntstr); return -1; } rd_free(itcntstr); /* Build client-final-message */ if (rd_kafka_sasl_scram_build_client_final_message( rktrans, &salt, server_nonce, in, itcnt, out) == -1) { rd_snprintf(errstr, errstr_size, "Failed to build SCRAM client-final-message"); rd_free(salt.ptr); rd_free(server_nonce); return -1; } rd_free(server_nonce); rd_free(salt.ptr); return 0; } /** * @brief Handle server-final-message * * This is the end of authentication and the SCRAM state * will be freed at the end of this function regardless of * authentication outcome. * * @returns -1 on failure */ static int rd_kafka_sasl_scram_handle_server_final_message ( rd_kafka_transport_t *rktrans, const rd_chariov_t *in, char *errstr, size_t errstr_size) { struct rd_kafka_sasl_scram_state *state = rktrans->rktrans_sasl.state; char *attr_v, *attr_e; if ((attr_e = rd_kafka_sasl_scram_get_attr( in, 'e', "server-error in server-final-message", errstr, errstr_size))) { /* Authentication failed */ rd_snprintf(errstr, errstr_size, "SASL SCRAM authentication failed: " "broker responded with %s", attr_e); rd_free(attr_e); return -1; } else if ((attr_v = rd_kafka_sasl_scram_get_attr( in, 'v', "verifier in server-final-message", errstr, errstr_size))) { const rd_kafka_conf_t *conf; /* Authentication succesful on server, * but we need to verify the ServerSignature too. */ rd_rkb_dbg(rktrans->rktrans_rkb, SECURITY | RD_KAFKA_DBG_BROKER, "SCRAMAUTH", "SASL SCRAM authentication succesful on server: " "verifying ServerSignature"); if (strcmp(attr_v, state->ServerSignatureB64)) { rd_snprintf(errstr, errstr_size, "SASL SCRAM authentication failed: " "ServerSignature mismatch " "(server's %s != ours %s)", attr_v, state->ServerSignatureB64); rd_free(attr_v); return -1; } rd_free(attr_v); conf = &rktrans->rktrans_rkb->rkb_rk->rk_conf; rd_rkb_dbg(rktrans->rktrans_rkb, SECURITY | RD_KAFKA_DBG_BROKER, "SCRAMAUTH", "Authenticated as %s using %s", conf->sasl.username, conf->sasl.mechanisms); rd_kafka_sasl_auth_done(rktrans); return 0; } else { rd_snprintf(errstr, errstr_size, "SASL SCRAM authentication failed: " "no verifier or server-error returned from broker"); return -1; } } /** * @brief Build client-first-message */ static void rd_kafka_sasl_scram_build_client_first_message ( rd_kafka_transport_t *rktrans, rd_chariov_t *out) { char *sasl_username; struct rd_kafka_sasl_scram_state *state = rktrans->rktrans_sasl.state; const rd_kafka_conf_t *conf = &rktrans->rktrans_rkb->rkb_rk->rk_conf; rd_kafka_sasl_scram_generate_nonce(&state->cnonce); sasl_username = rd_kafka_sasl_safe_string(conf->sasl.username); out->size = strlen("n,,n=,r=") + strlen(sasl_username) + state->cnonce.size; out->ptr = rd_malloc(out->size+1); rd_snprintf(out->ptr, out->size+1, "n,,n=%s,r=%.*s", sasl_username, (int)state->cnonce.size, state->cnonce.ptr); rd_free(sasl_username); /* Save client-first-message-bare (skip gs2-header) */ state->first_msg_bare.size = out->size-3; state->first_msg_bare.ptr = rd_memdup(out->ptr+3, state->first_msg_bare.size); } /** * @brief SASL SCRAM client state machine * @returns -1 on failure (errstr set), else 0. */ static int rd_kafka_sasl_scram_fsm (rd_kafka_transport_t *rktrans, const rd_chariov_t *in, char *errstr, size_t errstr_size) { static const char *state_names[] = { "client-first-message", "server-first-message", "client-final-message", }; struct rd_kafka_sasl_scram_state *state = rktrans->rktrans_sasl.state; rd_chariov_t out = RD_ZERO_INIT; int r = -1; rd_ts_t ts_start = rd_clock(); int prev_state = state->state; rd_rkb_dbg(rktrans->rktrans_rkb, SECURITY, "SASLSCRAM", "SASL SCRAM client in state %s", state_names[state->state]); switch (state->state) { case RD_KAFKA_SASL_SCRAM_STATE_CLIENT_FIRST_MESSAGE: rd_dassert(!in); /* Not expecting any server-input */ rd_kafka_sasl_scram_build_client_first_message(rktrans, &out); state->state = RD_KAFKA_SASL_SCRAM_STATE_SERVER_FIRST_MESSAGE; break; case RD_KAFKA_SASL_SCRAM_STATE_SERVER_FIRST_MESSAGE: rd_dassert(in); /* Requires server-input */ if (rd_kafka_sasl_scram_handle_server_first_message( rktrans, in, &out, errstr, errstr_size) == -1) return -1; state->state = RD_KAFKA_SASL_SCRAM_STATE_CLIENT_FINAL_MESSAGE; break; case RD_KAFKA_SASL_SCRAM_STATE_CLIENT_FINAL_MESSAGE: rd_dassert(in); /* Requires server-input */ r = rd_kafka_sasl_scram_handle_server_final_message( rktrans, in, errstr, errstr_size); break; } if (out.ptr) { r = rd_kafka_sasl_send(rktrans, out.ptr, (int)out.size, errstr, errstr_size); rd_free(out.ptr); } ts_start = (rd_clock() - ts_start) / 1000; if (ts_start >= 100) rd_rkb_dbg(rktrans->rktrans_rkb, SECURITY, "SCRAM", "SASL SCRAM state %s handled in %"PRId64"ms", state_names[prev_state], ts_start); return r; } /** * @brief Handle received frame from broker. */ static int rd_kafka_sasl_scram_recv (rd_kafka_transport_t *rktrans, const void *buf, size_t size, char *errstr, size_t errstr_size) { const rd_chariov_t in = { .ptr = (char *)buf, .size = size }; return rd_kafka_sasl_scram_fsm(rktrans, &in, errstr, errstr_size); } /** * @brief Initialize and start SASL SCRAM (builtin) authentication. * * Returns 0 on successful init and -1 on error. * * @locality broker thread */ static int rd_kafka_sasl_scram_client_new (rd_kafka_transport_t *rktrans, const char *hostname, char *errstr, size_t errstr_size) { struct rd_kafka_sasl_scram_state *state; state = rd_calloc(1, sizeof(*state)); state->state = RD_KAFKA_SASL_SCRAM_STATE_CLIENT_FIRST_MESSAGE; rktrans->rktrans_sasl.state = state; /* Kick off the FSM */ return rd_kafka_sasl_scram_fsm(rktrans, NULL, errstr, errstr_size); } /** * @brief Validate SCRAM config and look up the hash function */ static int rd_kafka_sasl_scram_conf_validate (rd_kafka_t *rk, char *errstr, size_t errstr_size) { const char *mech = rk->rk_conf.sasl.mechanisms; if (!rk->rk_conf.sasl.username || !rk->rk_conf.sasl.password) { rd_snprintf(errstr, errstr_size, "sasl.username and sasl.password must be set"); return -1; } if (!strcmp(mech, "SCRAM-SHA-1")) { rk->rk_conf.sasl.scram_evp = EVP_sha1(); rk->rk_conf.sasl.scram_H = SHA1; rk->rk_conf.sasl.scram_H_size = SHA_DIGEST_LENGTH; } else if (!strcmp(mech, "SCRAM-SHA-256")) { rk->rk_conf.sasl.scram_evp = EVP_sha256(); rk->rk_conf.sasl.scram_H = SHA256; rk->rk_conf.sasl.scram_H_size = SHA256_DIGEST_LENGTH; } else if (!strcmp(mech, "SCRAM-SHA-512")) { rk->rk_conf.sasl.scram_evp = EVP_sha512(); rk->rk_conf.sasl.scram_H = SHA512; rk->rk_conf.sasl.scram_H_size = SHA512_DIGEST_LENGTH; } else { rd_snprintf(errstr, errstr_size, "Unsupported hash function: %s " "(try SCRAM-SHA-512)", mech); return -1; } return 0; } const struct rd_kafka_sasl_provider rd_kafka_sasl_scram_provider = { .name = "SCRAM (builtin)", .client_new = rd_kafka_sasl_scram_client_new, .recv = rd_kafka_sasl_scram_recv, .close = rd_kafka_sasl_scram_close, .conf_validate = rd_kafka_sasl_scram_conf_validate, };
void rd_avg_start (rd_avg_t *ra) { rd_avg_period_next(ra, rd_clock()); }
/** * Serve q like rd_kafka_q_serve() until an op is found that can be returned * as an event to the application. * * @returns the first event:able op, or NULL on timeout. * * Locality: any thread */ rd_kafka_op_t *rd_kafka_q_pop_serve (rd_kafka_q_t *rkq, int timeout_ms, int32_t version, int cb_type, int (*callback) (rd_kafka_t *rk, rd_kafka_op_t *rko, int cb_type, void *opaque), void *opaque) { rd_kafka_op_t *rko; if (timeout_ms == RD_POLL_INFINITE) timeout_ms = INT_MAX; mtx_lock(&rkq->rkq_lock); if (!rkq->rkq_fwdq) { do { /* Filter out outdated ops */ while ((rko = TAILQ_FIRST(&rkq->rkq_q)) && !(rko = rd_kafka_op_filter(rkq, rko, version))) ; if (rko) { int handled; /* Proper versioned op */ rd_kafka_q_deq0(rkq, rko); /* Ops with callbacks are considered handled * and we move on to the next op, if any. * Ops w/o callbacks are returned immediately */ if (callback) { handled = callback(rkq->rkq_rk, rko, cb_type, opaque); if (handled) { rd_kafka_op_destroy(rko); rko = NULL; } } else handled = 0; if (!handled) break; } /* No op, wait for one */ rd_ts_t pre = rd_clock(); if (cnd_timedwait_ms(&rkq->rkq_cond, &rkq->rkq_lock, timeout_ms) == thrd_timedout) { mtx_unlock(&rkq->rkq_lock); return NULL; } /* Remove spent time */ timeout_ms -= (int) (rd_clock()-pre) / 1000; if (timeout_ms < 0) timeout_ms = RD_POLL_NOWAIT; } while (timeout_ms != RD_POLL_NOWAIT); mtx_unlock(&rkq->rkq_lock); } else { rd_kafka_q_t *fwdq = rkq->rkq_fwdq; rd_kafka_q_keep(fwdq); /* Since the q_pop may block we need to release the parent * queue's lock. */ mtx_unlock(&rkq->rkq_lock); rko = rd_kafka_q_pop_serve(fwdq, timeout_ms, version, cb_type, callback, opaque); rd_kafka_q_destroy(fwdq); } return rko; }
(char *)(missing_topics->rl_elems[i])); RD_LIST_FOREACH(topic, missing_topics, i) { shptr_rd_kafka_itopic_t *s_rkt; s_rkt = rd_kafka_topic_find(rkb->rkb_rk, topic, 1/*lock*/); if (s_rkt) { rd_kafka_topic_metadata_none( rd_kafka_topic_s2i(s_rkt)); rd_kafka_topic_destroy0(s_rkt); } } } rd_kafka_wrlock(rkb->rkb_rk); rkb->rkb_rk->rk_ts_metadata = rd_clock(); /* Update cached cluster id. */ if (RD_KAFKAP_STR_LEN(&cluster_id) > 0 && (!rkb->rkb_rk->rk_clusterid || rd_kafkap_str_cmp_str(&cluster_id, rkb->rkb_rk->rk_clusterid))) { rd_rkb_dbg(rkb, BROKER|RD_KAFKA_DBG_GENERIC, "CLUSTERID", "ClusterId update \"%s\" -> \"%.*s\"", rkb->rkb_rk->rk_clusterid ? rkb->rkb_rk->rk_clusterid : "", RD_KAFKAP_STR_PR(&cluster_id)); if (rkb->rkb_rk->rk_clusterid) rd_free(rkb->rkb_rk->rk_clusterid); rkb->rkb_rk->rk_clusterid = RD_KAFKAP_STR_DUP(&cluster_id); }
int main (int argc, char **argv) { rd_kafka_t *rk; char *broker = NULL; char mode = 'C'; char *topic = NULL; int partition = 0; int opt; int msgsize = 1024; int msgcnt = -1; int sendflags = 0; int dispintvl = 1000; struct { rd_ts_t t_start; rd_ts_t t_end; rd_ts_t t_end_send; uint64_t msgs; uint64_t bytes; rd_ts_t t_latency; rd_ts_t t_last; rd_ts_t t_total; } cnt = {}; rd_ts_t now; char *dirstr = ""; while ((opt = getopt(argc, argv, "PCt:p:b:s:c:fi:D")) != -1) { switch (opt) { case 'P': case 'C': mode = opt; break; case 't': topic = optarg; break; case 'p': partition = atoi(optarg); break; case 'b': broker = optarg; break; case 's': msgsize = atoi(optarg); break; case 'c': msgcnt = atoi(optarg); break; case 'D': sendflags |= RD_KAFKA_OP_F_FREE; break; case 'i': dispintvl = atoi(optarg); break; default: goto usage; } } if (!topic || optind != argc) { usage: fprintf(stderr, "Usage: %s [-C|-P] -t <topic> " "[-p <partition>] [-b <broker>] [options..]\n" "\n" " Options:\n" " -C | -P Consumer or Producer mode\n" " -t <topic> Topic to fetch / produce\n" " -p <num> Partition (defaults to 0)\n" " -b <broker> Broker address (localhost:9092)\n" " -s <size> Message size (producer)\n" " -c <cnt> Messages to transmit/receive\n" " -D Copy/Duplicate data buffer (producer)\n" " -i <ms> Display interval\n" "\n" " In Consumer mode:\n" " consumes messages and prints thruput\n" " In Producer mode:\n" " writes messages of size -s <..> and prints thruput\n" "\n", argv[0]); exit(1); } dispintvl *= 1000; /* us */ signal(SIGINT, stop); /* Socket hangups are gracefully handled in librdkafka on socket error * without the use of signals, so SIGPIPE should be ignored by the * calling program. */ signal(SIGPIPE, SIG_IGN); if (mode == 'P') { /* * Producer */ char *sbuf = malloc(msgsize); int endwait; int outq; int i; memset(sbuf, 'R', msgsize); 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, broker, NULL))) { perror("kafka_new producer"); exit(1); } cnt.t_start = rd_clock(); while (run && (msgcnt == -1 || cnt.msgs < msgcnt)) { char *pbuf = sbuf; /* Send/Produce message. */ if (sendflags & RD_KAFKA_OP_F_FREE) { /* Duplicate memory */ pbuf = malloc(msgsize); memcpy(pbuf, sbuf, msgsize); } rd_kafka_produce(rk, topic, partition, sendflags, pbuf, msgsize); cnt.msgs++; cnt.bytes += msgsize; now = rd_clock(); if (cnt.t_last + dispintvl <= now) { printf("%% %"PRIu64" messages and %"PRIu64 "bytes: %"PRIu64" msgs/s and " "%.2f Mb/s\n", cnt.msgs, cnt.bytes, (cnt.msgs / (now - cnt.t_start)) * 1000000, (float)(cnt.bytes / (now - cnt.t_start))); cnt.t_last = now; } } /* Wait for messaging to finish. */ i = 0; while (run && rd_kafka_outq_len(rk) > 0) { if (!(i++ % (dispintvl/1000))) printf("%% Waiting for %i messages in outq " "to be sent. Abort with Ctrl-c\n", rd_kafka_outq_len(rk)); usleep(1000); } cnt.t_end_send = rd_clock(); outq = rd_kafka_outq_len(rk); cnt.msgs -= outq; cnt.bytes -= msgsize * outq; cnt.t_end = rd_clock(); /* Since there is no ack for produce messages in 0.7 * we wait some more for any packets in the socket buffers * to be sent. * This is fixed in protocol version 0.8 */ endwait = cnt.msgs * 10; printf("%% Test timers stopped, but waiting %ims more " "for the %"PRIu64 " messages to be transmitted from " "socket buffers.\n" "%% End with Ctrl-c\n", endwait / 1000, cnt.msgs); run = 1; while (run && endwait > 0) { usleep(10000); endwait -= 10000; } /* Destroy the handle */ rd_kafka_destroy(rk); dirstr = "sent"; } else if (mode == 'C') { /* * Consumer */ rd_kafka_op_t *rko; /* Base our configuration on the default config. */ rd_kafka_conf_t conf = rd_kafka_defaultconf; /* 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 * succesfully 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; /* Tell rdkafka to (try to) maintain 10000 messages * in its internal receive buffers. This is to avoid * application -> rdkafka -> broker per-message ping-pong * latency. */ conf.consumer.replyq_low_thres = 100000; /* Use the consumer convenience function * to create a Kafka handle. */ if (!(rk = rd_kafka_new_consumer(broker, topic, (uint32_t)partition, 0, &conf))) { perror("kafka_new_consumer"); exit(1); } cnt.t_start = rd_clock(); while (run && (msgcnt == -1 || msgcnt > cnt.msgs)) { /* Fetch an "op" which is one of: * - a kafka message (if rko_len>0 && rko_err==0) * - an error (if rko_err) */ uint64_t latency; latency = rd_clock(); if (!(rko = rd_kafka_consume(rk, 1000/*timeout ms*/))) continue; cnt.t_latency += rd_clock() - latency; if (rko->rko_err) fprintf(stderr, "%% Error: %.*s\n", rko->rko_len, rko->rko_payload); else if (rko->rko_len) { cnt.msgs++; cnt.bytes += rko->rko_len; } /* rko_offset contains the offset of the _next_ * message. We store it when we're done processing * the current message. */ if (rko->rko_offset) rd_kafka_offset_store(rk, rko->rko_offset); /* Destroy the op */ rd_kafka_op_destroy(rk, rko); now = rd_clock(); if (cnt.t_last + dispintvl <= now && cnt.t_start + 1000000 < now) { printf("%% %"PRIu64" messages and %"PRIu64 " bytes: %"PRIu64" msgs/s and " "%.2f Mb/s\n", cnt.msgs, cnt.bytes, (cnt.msgs / ((now - cnt.t_start)/1000)) * 1000, (float)(cnt.bytes / ((now - cnt.t_start) / 1000))); cnt.t_last = now; } } cnt.t_end = rd_clock(); /* Destroy the handle */ rd_kafka_destroy(rk); dirstr = "received"; } if (cnt.t_end_send) cnt.t_total = cnt.t_end_send - cnt.t_start; else cnt.t_total = cnt.t_end - cnt.t_start; printf("%% %"PRIu64" messages and %"PRIu64" bytes " "%s in %"PRIu64"ms: %"PRIu64" msgs/s and %.02f Mb/s\n", cnt.msgs, cnt.bytes, dirstr, cnt.t_total / 1000, (cnt.msgs / (cnt.t_total / 1000)) * 1000, (float)(cnt.bytes / (cnt.t_total / 1000))); if (cnt.t_latency) printf("%% Average application fetch latency: %"PRIu64"us\n", cnt.t_latency / cnt.msgs); return 0; }