static int rd_kafka_offset_file_open (rd_kafka_toppar_t *rktp) { rd_kafka_t *rk = rktp->rktp_rkt->rkt_rk; int fd; #ifndef _MSC_VER mode_t mode = 0644; #else mode_t mode = _S_IREAD|_S_IWRITE; #endif if ((fd = rk->rk_conf.open_cb(rktp->rktp_offset_path, O_CREAT|O_RDWR, mode, rk->rk_conf.opaque)) == -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, rd_strerror(errno)); return -1; } rktp->rktp_offset_fp = #ifndef _MSC_VER fdopen(fd, "r+"); #else _fdopen(fd, "r+"); #endif return 0; }
static const char *socket_strerror(int err) { #ifdef _MSC_VER static RD_TLS char buf[256]; FormatMessageA(FORMAT_MESSAGE_FROM_SYSTEM | FORMAT_MESSAGE_IGNORE_INSERTS, NULL, err, MAKELANGID(LANG_NEUTRAL, SUBLANG_DEFAULT), (LPSTR)buf, sizeof(buf)-1, NULL); return buf; #else return rd_strerror(err); #endif }
static int64_t rd_kafka_offset_file_read (rd_kafka_toppar_t *rktp) { char buf[22]; char *end; int64_t offset; size_t r; if (fseek(rktp->rktp_offset_fp, 0, SEEK_SET) == -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, rd_strerror(errno)); rd_kafka_offset_file_close(rktp); return RD_KAFKA_OFFSET_INVALID; } r = fread(buf, 1, sizeof(buf) - 1, rktp->rktp_offset_fp); 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 RD_KAFKA_OFFSET_INVALID; } 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 RD_KAFKA_OFFSET_INVALID; } 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; }
/** * Set transport IO event polling based on SSL error. * * Returns -1 on permanent errors. * * Locality: broker thread */ static RD_INLINE int rd_kafka_transport_ssl_io_update (rd_kafka_transport_t *rktrans, int ret, char *errstr, size_t errstr_size) { int serr = SSL_get_error(rktrans->rktrans_ssl, ret); int serr2; switch (serr) { case SSL_ERROR_WANT_READ: rd_kafka_transport_poll_set(rktrans, POLLIN); break; case SSL_ERROR_WANT_WRITE: case SSL_ERROR_WANT_CONNECT: rd_kafka_transport_poll_set(rktrans, POLLOUT); break; case SSL_ERROR_SYSCALL: if (!(serr2 = SSL_get_error(rktrans->rktrans_ssl, ret))) { if (ret == 0) errno = ECONNRESET; rd_snprintf(errstr, errstr_size, "SSL syscall error: %s", rd_strerror(errno)); } else rd_snprintf(errstr, errstr_size, "SSL syscall error number: %d: %s", serr2, rd_strerror(errno)); return -1; default: rd_kafka_ssl_error(NULL, rktrans->rktrans_rkb, errstr, errstr_size); return -1; } return 0; }
static ssize_t rd_kafka_transport_socket_sendmsg (rd_kafka_transport_t *rktrans, const struct msghdr *msg, char *errstr, size_t errstr_size) { #ifndef _MSC_VER ssize_t r; #ifdef sun /* See recvmsg() comment. Setting it here to be safe. */ socket_errno = EAGAIN; #endif r = sendmsg(rktrans->rktrans_s, msg, MSG_DONTWAIT #ifdef MSG_NOSIGNAL | MSG_NOSIGNAL #endif ); if (r == -1) { if (socket_errno == EAGAIN) return 0; rd_snprintf(errstr, errstr_size, "%s", rd_strerror(errno)); } return r; #else int i; ssize_t sum = 0; for (i = 0; i < msg->msg_iovlen; i++) { ssize_t r; r = send(rktrans->rktrans_s, msg->msg_iov[i].iov_base, (int) msg->msg_iov[i].iov_len, 0); if (r == SOCKET_ERROR) { if (sum > 0 || WSAGetLastError() == WSAEWOULDBLOCK) return sum; else { rd_snprintf(errstr, errstr_size, "%s", socket_strerror(WSAGetLastError())); return -1; } } sum += r; if ((size_t)r < msg->msg_iov[i].iov_len) break; } return sum; #endif }
static ssize_t rd_kafka_transport_socket_recvmsg (rd_kafka_transport_t *rktrans, struct msghdr *msg, char *errstr, size_t errstr_size) { #ifndef _MSC_VER ssize_t r; #ifdef sun /* SunOS doesn't seem to set errno when recvmsg() fails * due to no data and MSG_DONTWAIT is set. */ socket_errno = EAGAIN; #endif r = recvmsg(rktrans->rktrans_s, msg, MSG_DONTWAIT); if (r == -1 && socket_errno == EAGAIN) return 0; else if (r == 0) { /* Receive 0 after POLLIN event means connection closed. */ rd_snprintf(errstr, errstr_size, "Disconnected"); return -1; } else if (r == -1) rd_snprintf(errstr, errstr_size, "%s", rd_strerror(errno)); return r; #else ssize_t sum = 0; int i; for (i = 0; i < msg->msg_iovlen; i++) { ssize_t r; r = recv(rktrans->rktrans_s, msg->msg_iov[i].iov_base, (int) msg->msg_iov[i].iov_len, 0); if (r == SOCKET_ERROR) { if (WSAGetLastError() == WSAEWOULDBLOCK) break; rd_snprintf(errstr, errstr_size, "%s", socket_strerror(WSAGetLastError())); return -1; } sum += r; if ((size_t)r < msg->msg_iov[i].iov_len) break; } return sum; #endif }
/** * Produces 'msgcnt' messages split over 'partition_cnt' partitions. */ static void produce_messages (uint64_t testid, const char *topic, int partition_cnt, int msgcnt) { 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 failcnt = 0; int i; rd_kafka_message_t *rkmessages; int32_t partition; int msgid = 0; test_conf_init(&conf, &topic_conf, 20); rd_kafka_conf_set_dr_cb(conf, dr_cb); /* Make sure all replicas are in-sync after producing * so that consume test wont fail. */ rd_kafka_topic_conf_set(topic_conf, "request.required.acks", "-1", errstr, sizeof(errstr)); /* Create kafka instance */ rk = test_create_handle(RD_KAFKA_PRODUCER, conf); rkt = rd_kafka_topic_new(rk, topic, topic_conf); if (!rkt) TEST_FAIL("Failed to create topic: %s\n", rd_strerror(errno)); /* Create messages. */ prod_msg_remains = msgcnt; rkmessages = calloc(sizeof(*rkmessages), msgcnt / partition_cnt); for (partition = 0 ; partition < partition_cnt ; partition++) { int batch_cnt = msgcnt / partition_cnt; for (i = 0 ; i < batch_cnt ; i++) { rd_snprintf(msg, sizeof(msg), "testid=%"PRIu64", partition=%i, msg=%i", testid, (int)partition, msgid); rkmessages[i].payload = rd_strdup(msg); rkmessages[i].len = strlen(msg); msgid++; } TEST_SAY("Start produce to partition %i: msgs #%d..%d\n", (int)partition, msgid-batch_cnt, msgid); /* Produce batch for this partition */ r = rd_kafka_produce_batch(rkt, partition, RD_KAFKA_MSG_F_FREE, rkmessages, batch_cnt); if (r == -1) TEST_FAIL("Failed to produce " "batch for partition %i: %s", (int)partition, rd_kafka_err2str(rd_kafka_errno2err(errno))); /* Scan through messages to check for errors. */ for (i = 0 ; i < batch_cnt ; i++) { if (rkmessages[i].err) { failcnt++; if (failcnt < 100) TEST_SAY("Message #%i failed: %s\n", i, rd_kafka_err2str(rkmessages[i]. err)); } } /* All messages should've been produced. */ if (r < batch_cnt) { TEST_SAY("Not all messages were accepted " "by produce_batch(): %i < %i\n", r, batch_cnt); if (batch_cnt - r != failcnt) TEST_SAY("Discrepency between failed " "messages (%i) " "and return value %i (%i - %i)\n", failcnt, batch_cnt - r, batch_cnt, r); TEST_FAIL("%i/%i messages failed\n", batch_cnt - r, batch_cnt); } TEST_SAY("Produced %i messages to partition %i, " "waiting for deliveries\n", r, partition); } free(rkmessages); /* Wait for messages to be delivered */ while (rd_kafka_outq_len(rk) > 0) rd_kafka_poll(rk, 100); if (fails) TEST_FAIL("%i failures, see previous errors", fails); if (prod_msg_remains != 0) TEST_FAIL("Still waiting for %i messages to be produced", prod_msg_remains); /* 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); }
static void consume_messages_with_queues (uint64_t testid, const char *topic, int partition_cnt, int msgcnt) { rd_kafka_t *rk; rd_kafka_topic_t *rkt; rd_kafka_conf_t *conf; rd_kafka_topic_conf_t *topic_conf; rd_kafka_queue_t *rkqu; int i; int32_t partition; int batch_cnt = msgcnt / partition_cnt; test_conf_init(&conf, &topic_conf, 20); /* Create kafka instance */ rk = test_create_handle(RD_KAFKA_CONSUMER, conf); /* Create queue */ rkqu = rd_kafka_queue_new(rk); rkt = rd_kafka_topic_new(rk, topic, topic_conf); if (!rkt) TEST_FAIL("Failed to create topic: %s\n", rd_strerror(errno)); TEST_SAY("Consuming %i messages from one queue serving %i partitions\n", msgcnt, partition_cnt); /* Start consuming each partition */ for (partition = 0 ; partition < partition_cnt ; partition++) { /* Consume messages */ TEST_SAY("Start consuming partition %i at offset -%i\n", partition, batch_cnt); if (rd_kafka_consume_start_queue(rkt, partition, RD_KAFKA_OFFSET_TAIL(batch_cnt), rkqu) == -1) TEST_FAIL("consume_start_queue(%i) failed: %s", (int)partition, rd_kafka_err2str(rd_kafka_errno2err(errno))); } /* Consume messages from queue */ for (i = 0 ; i < msgcnt ; ) { rd_kafka_message_t *rkmessage; rkmessage = rd_kafka_consume_queue(rkqu, tmout_multip(5000)); if (!rkmessage) TEST_FAIL("Failed to consume message %i/%i from " "queue: %s", i, msgcnt, rd_kafka_err2str(rd_kafka_errno2err(errno))); if (rkmessage->err) { if (rkmessage->err == RD_KAFKA_RESP_ERR__PARTITION_EOF){ TEST_SAY("Topic %s [%"PRId32"] reached " "EOF at offset %"PRId64"\n", rd_kafka_topic_name(rkmessage->rkt), rkmessage->partition, rkmessage->offset); rd_kafka_message_destroy(rkmessage); continue; } TEST_FAIL("Consume message %i/%i from queue " "has error (offset %"PRId64 ", partition %"PRId32"): %s", i, msgcnt, rkmessage->offset, rkmessage->partition, rd_kafka_err2str(rkmessage->err)); } verify_consumed_msg(testid, -1, -1, rkmessage); rd_kafka_message_destroy(rkmessage); i++; } /* Stop consuming each partition */ for (partition = 0 ; partition < partition_cnt ; partition++) rd_kafka_consume_stop(rkt, partition); /* Destroy queue */ rd_kafka_queue_destroy(rkqu); /* 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); }
static void consume_messages (uint64_t testid, const char *topic, int32_t partition, int msg_base, int batch_cnt, int msgcnt) { rd_kafka_t *rk; rd_kafka_topic_t *rkt; rd_kafka_conf_t *conf; rd_kafka_topic_conf_t *topic_conf; int i; test_conf_init(&conf, &topic_conf, 20); /* Create kafka instance */ rk = test_create_handle(RD_KAFKA_CONSUMER, conf); 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", rd_strerror(errno)); TEST_SAY("Consuming %i messages from partition %i\n", batch_cnt, partition); /* Consume messages */ if (rd_kafka_consume_start(rkt, partition, RD_KAFKA_OFFSET_TAIL(batch_cnt)) == -1) TEST_FAIL("consume_start(%i, -%i) failed: %s", (int)partition, batch_cnt, rd_kafka_err2str(rd_kafka_errno2err(errno))); for (i = 0 ; i < batch_cnt ; ) { rd_kafka_message_t *rkmessage; rkmessage = rd_kafka_consume(rkt, partition, tmout_multip(5000)); if (!rkmessage) TEST_FAIL("Failed to consume message %i/%i from " "partition %i: %s", i, batch_cnt, (int)partition, rd_kafka_err2str(rd_kafka_errno2err(errno))); if (rkmessage->err) { if (rkmessage->err == RD_KAFKA_RESP_ERR__PARTITION_EOF){ rd_kafka_message_destroy(rkmessage); continue; } TEST_FAIL("Consume message %i/%i from partition %i " "has error: %s", i, batch_cnt, (int)partition, rd_kafka_err2str(rkmessage->err)); } verify_consumed_msg(testid, partition, msg_base+i, rkmessage); rd_kafka_message_destroy(rkmessage); i++; } rd_kafka_consume_stop(rkt, partition); /* 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); }
int main_0008_reqacks (int argc, char **argv) { 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; const char *topic = NULL; TEST_SAY("\033[33mNOTE! This test requires at " "least 3 brokers!\033[0m\n"); TEST_SAY("\033[33mNOTE! This test requires " "default.replication.factor=3 to be configured on " "all brokers!\033[0m\n"); /* Try different request.required.acks settings (issue #75) */ for (reqacks = -1 ; reqacks <= 1 ; reqacks++) { char tmp[10]; test_conf_init(&conf, &topic_conf, 10); if (!topic) topic = test_mk_topic_name("0008", 0); rd_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", rd_strerror(errno)); /* Produce messages */ for (i = 0 ; i < msgcnt ; i++) { int *msgidp = malloc(sizeof(*msgidp)); *msgidp = idbase + i; rd_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, rd_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); } return 0; }
int main_0003_msgmaxsize (int argc, char **argv) { 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; static const int msgsize = 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 = test_create_handle(RD_KAFKA_PRODUCER, conf); rkt = rd_kafka_topic_new(rk, test_mk_topic_name("0003", 0), topic_conf); if (!rkt) TEST_FAIL("Failed to create topic: %s\n", rd_strerror(errno)); msg = calloc(1, msgsize); /* 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); } rd_snprintf(msg, msgsize, "%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, rd_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); free(msg); /* 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); return 0; }
int main_0002_unkpart (int argc, char **argv) { 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 msg[128]; int msgcnt = 10; int i; int fails = 0; const struct rd_kafka_metadata *metadata; test_conf_init(&conf, &topic_conf, 10); /* Set delivery report callback */ rd_kafka_conf_set_dr_cb(conf, dr_cb); /* Create kafka instance */ rk = test_create_handle(RD_KAFKA_PRODUCER, conf); rkt = rd_kafka_topic_new(rk, test_mk_topic_name("0002", 0), topic_conf); if (!rkt) TEST_FAIL("Failed to create topic: %s\n", rd_strerror(errno)); /* Request metadata so that we know the cluster is up before producing * messages, otherwise erroneous partitions will not fail immediately.*/ if ((r = rd_kafka_metadata(rk, 0, rkt, &metadata, tmout_multip(15000))) != RD_KAFKA_RESP_ERR_NO_ERROR) TEST_FAIL("Failed to acquire metadata: %s\n", rd_kafka_err2str(r)); rd_kafka_metadata_destroy(metadata); /* Produce a message */ for (i = 0 ; i < msgcnt ; i++) { int *msgidp = malloc(sizeof(*msgidp)); *msgidp = i; rd_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, rd_kafka_err2str( rd_kafka_errno2err(errno))); free(msgidp); } else { if (i > 5) { fails++; TEST_SAY("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) rd_sleep(2); } /* Wait for messages to time out */ rd_kafka_flush(rk, -1); if (msgs_wait != 0) TEST_FAIL("Still waiting for messages: 0x%x\n", msgs_wait); if (fails > 0) TEST_FAIL("See previous error(s)\n"); /* 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); return 0; }
/** * IO event handler. * * Locality: broker thread */ static void rd_kafka_transport_io_event (rd_kafka_transport_t *rktrans, int events) { char errstr[512]; int r; rd_kafka_broker_t *rkb = rktrans->rktrans_rkb; switch (rkb->rkb_state) { case RD_KAFKA_BROKER_STATE_CONNECT: #if WITH_SSL if (rktrans->rktrans_ssl) { /* Currently setting up SSL connection: * perform handshake. */ rd_kafka_transport_ssl_handhsake(rktrans); return; } #endif /* Asynchronous connect finished, read status. */ if (!(events & (POLLOUT|POLLERR|POLLHUP))) return; if (rd_kafka_transport_get_socket_error(rktrans, &r) == -1) { rd_kafka_broker_fail( rkb, LOG_ERR, RD_KAFKA_RESP_ERR__TRANSPORT, "Connect to %s failed: " "unable to get status from " "socket %d: %s", rd_sockaddr2str(rkb->rkb_addr_last, RD_SOCKADDR2STR_F_PORT | RD_SOCKADDR2STR_F_FAMILY), rktrans->rktrans_s, rd_strerror(socket_errno)); } else if (r != 0) { /* Connect failed */ errno = r; rd_snprintf(errstr, sizeof(errstr), "Connect to %s failed: %s", rd_sockaddr2str(rkb->rkb_addr_last, RD_SOCKADDR2STR_F_PORT | RD_SOCKADDR2STR_F_FAMILY), rd_strerror(r)); rd_kafka_transport_connect_done(rktrans, errstr); } else { /* Connect succeeded */ rd_kafka_transport_connected(rktrans); } break; case RD_KAFKA_BROKER_STATE_AUTH: #if WITH_SASL rd_kafka_assert(NULL, rktrans->rktrans_sasl.conn != NULL); /* SASL handshake */ if (rd_kafka_sasl_io_event(rktrans, events, errstr, sizeof(errstr)) == -1) { errno = EINVAL; rd_kafka_broker_fail(rkb, LOG_ERR, RD_KAFKA_RESP_ERR__AUTHENTICATION, "SASL authentication failure: %s", errstr); return; } #endif break; case RD_KAFKA_BROKER_STATE_APIVERSION_QUERY: case RD_KAFKA_BROKER_STATE_AUTH_HANDSHAKE: case RD_KAFKA_BROKER_STATE_UP: case RD_KAFKA_BROKER_STATE_UPDATE: if (events & POLLIN) { while (rkb->rkb_state >= RD_KAFKA_BROKER_STATE_UP && rd_kafka_recv(rkb) > 0) ; } if (events & POLLHUP) { rd_kafka_broker_fail(rkb, rkb->rkb_rk->rk_conf. log_connection_close ? LOG_NOTICE : LOG_DEBUG, RD_KAFKA_RESP_ERR__TRANSPORT, "Connection closed"); return; } if (events & POLLOUT) { while (rd_kafka_send(rkb) > 0) ; } break; case RD_KAFKA_BROKER_STATE_INIT: case RD_KAFKA_BROKER_STATE_DOWN: rd_kafka_assert(rkb->rkb_rk, !*"bad state"); } }
/* Produce a batch of messages to a single partition. */ static void test_single_partition (void) { 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 msg[128]; int msgcnt = 100000; int failcnt = 0; int i; rd_kafka_message_t *rkmessages; int msgcounter = 0; msgid_next = 0; test_conf_init(&conf, &topic_conf, 20); /* Set delivery report callback */ rd_kafka_conf_set_dr_cb(conf, dr_single_partition_cb); rd_kafka_conf_set_opaque(conf, &msgcounter); /* Create kafka instance */ rk = test_create_handle(RD_KAFKA_PRODUCER, conf); TEST_SAY("test_single_partition: Created kafka instance %s\n", rd_kafka_name(rk)); rkt = rd_kafka_topic_new(rk, test_mk_topic_name("0011", 0), topic_conf); if (!rkt) TEST_FAIL("Failed to create topic: %s\n", rd_strerror(errno)); /* Create messages */ rkmessages = calloc(sizeof(*rkmessages), msgcnt); for (i = 0 ; i < msgcnt ; i++) { int *msgidp = malloc(sizeof(*msgidp)); *msgidp = i; rd_snprintf(msg, sizeof(msg), "%s:%s test message #%i", __FILE__, __FUNCTION__, i); rkmessages[i].payload = rd_strdup(msg); rkmessages[i].len = strlen(msg); rkmessages[i]._private = msgidp; } r = rd_kafka_produce_batch(rkt, partition, RD_KAFKA_MSG_F_FREE, rkmessages, msgcnt); /* Scan through messages to check for errors. */ for (i = 0 ; i < msgcnt ; i++) { if (rkmessages[i].err) { failcnt++; if (failcnt < 100) TEST_SAY("Message #%i failed: %s\n", i, rd_kafka_err2str(rkmessages[i].err)); } } /* All messages should've been produced. */ if (r < msgcnt) { TEST_SAY("Not all messages were accepted " "by produce_batch(): %i < %i\n", r, msgcnt); if (msgcnt - r != failcnt) TEST_SAY("Discrepency between failed messages (%i) " "and return value %i (%i - %i)\n", failcnt, msgcnt - r, msgcnt, r); TEST_FAIL("%i/%i messages failed\n", msgcnt - r, msgcnt); } free(rkmessages); TEST_SAY("Single partition: " "Produced %i messages, waiting for deliveries\n", r); msgcounter = msgcnt; /* Wait for messages to be delivered */ test_wait_delivery(rk, &msgcounter); 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); return; }
/** * Write offset to offset file. * * Locality: toppar's broker thread */ static rd_kafka_resp_err_t rd_kafka_offset_file_commit (rd_kafka_toppar_t *rktp) { rd_kafka_itopic_t *rkt = rktp->rktp_rkt; int attempt; rd_kafka_resp_err_t err = RD_KAFKA_RESP_ERR_NO_ERROR; int64_t offset = rktp->rktp_stored_offset; for (attempt = 0 ; attempt < 2 ; attempt++) { char buf[22]; int len; if (!rktp->rktp_offset_fp) if (rd_kafka_offset_file_open(rktp) == -1) continue; if (fseek(rktp->rktp_offset_fp, 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, rd_strerror(errno)); err = RD_KAFKA_RESP_ERR__FS; rd_kafka_offset_file_close(rktp); continue; } len = rd_snprintf(buf, sizeof(buf), "%"PRId64"\n", offset); if (fwrite(buf, 1, len, rktp->rktp_offset_fp) < 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: %s", rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition, offset, rktp->rktp_offset_path, rd_strerror(errno)); err = RD_KAFKA_RESP_ERR__FS; rd_kafka_offset_file_close(rktp); continue; } /* Need to flush before truncate to preserve write ordering */ (void)fflush(rktp->rktp_offset_fp); /* Truncate file */ #ifdef _MSC_VER if (_chsize_s(_fileno(rktp->rktp_offset_fp), len) == -1) ; /* Ignore truncate failures */ #else if (ftruncate(fileno(rktp->rktp_offset_fp), len) == -1) ; /* Ignore truncate failures */ #endif 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_committed_offset = offset; /* If sync interval is set to immediate we sync right away. */ if (rkt->rkt_conf.offset_store_sync_interval_ms == 0) rd_kafka_offset_file_sync(rktp); return RD_KAFKA_RESP_ERR_NO_ERROR; } return err; }
/** * @brief Decompress MessageSet, pass the uncompressed MessageSet to * the MessageSet reader. */ static rd_kafka_resp_err_t rd_kafka_msgset_reader_decompress (rd_kafka_msgset_reader_t *msetr, int MsgVersion, int Attributes, int64_t Timestamp, int64_t Offset, const void *compressed, size_t compressed_size) { struct iovec iov = { .iov_base = NULL, .iov_len = 0 }; rd_kafka_toppar_t *rktp = msetr->msetr_rktp; int codec = Attributes & RD_KAFKA_MSG_ATTR_COMPRESSION_MASK; rd_kafka_resp_err_t err = RD_KAFKA_RESP_ERR_NO_ERROR; rd_kafka_buf_t *rkbufz; switch (codec) { #if WITH_ZLIB case RD_KAFKA_COMPRESSION_GZIP: { uint64_t outlenx = 0; /* Decompress Message payload */ iov.iov_base = rd_gz_decompress(compressed, (int)compressed_size, &outlenx); if (unlikely(!iov.iov_base)) { rd_rkb_dbg(msetr->msetr_rkb, MSG, "GZIP", "Failed to decompress Gzip " "message at offset %"PRId64 " of %"PRIusz" bytes: " "ignoring message", Offset, compressed_size); err = RD_KAFKA_RESP_ERR__BAD_COMPRESSION; goto err; } iov.iov_len = (size_t)outlenx; } break; #endif #if WITH_SNAPPY case RD_KAFKA_COMPRESSION_SNAPPY: { const char *inbuf = compressed; size_t inlen = compressed_size; int r; static const unsigned char snappy_java_magic[] = { 0x82, 'S','N','A','P','P','Y', 0 }; static const size_t 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(inlen > snappy_java_hdrlen + 4 && !memcmp(inbuf, snappy_java_magic, 8))) { /* snappy-java framing */ char errstr[128]; inbuf = inbuf + snappy_java_hdrlen; inlen -= snappy_java_hdrlen; iov.iov_base = rd_kafka_snappy_java_uncompress( inbuf, inlen, &iov.iov_len, errstr, sizeof(errstr)); if (unlikely(!iov.iov_base)) { rd_rkb_dbg(msetr->msetr_rkb, MSG, "SNAPPY", "%s [%"PRId32"]: " "Snappy decompression for message " "at offset %"PRId64" failed: %s: " "ignoring message", rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition, Offset, errstr); err = RD_KAFKA_RESP_ERR__BAD_COMPRESSION; goto err; } } else { /* No framing */ /* Acquire uncompressed length */ if (unlikely(!rd_kafka_snappy_uncompressed_length( inbuf, inlen, &iov.iov_len))) { rd_rkb_dbg(msetr->msetr_rkb, MSG, "SNAPPY", "Failed to get length of Snappy " "compressed payload " "for message at offset %"PRId64 " (%"PRIusz" bytes): " "ignoring message", Offset, inlen); err = RD_KAFKA_RESP_ERR__BAD_COMPRESSION; goto err; } /* Allocate output buffer for uncompressed data */ iov.iov_base = rd_malloc(iov.iov_len); if (unlikely(!iov.iov_base)) { rd_rkb_dbg(msetr->msetr_rkb, MSG, "SNAPPY", "Failed to allocate Snappy " "decompress buffer of size %"PRIusz "for message at offset %"PRId64 " (%"PRIusz" bytes): %s: " "ignoring message", iov.iov_len, Offset, inlen, rd_strerror(errno)); err = RD_KAFKA_RESP_ERR__CRIT_SYS_RESOURCE; goto err; } /* Uncompress to outbuf */ if (unlikely((r = rd_kafka_snappy_uncompress( inbuf, inlen, iov.iov_base)))) { rd_rkb_dbg(msetr->msetr_rkb, MSG, "SNAPPY", "Failed to decompress Snappy " "payload for message at offset " "%"PRId64" (%"PRIusz" bytes): %s: " "ignoring message", Offset, inlen, rd_strerror(-r/*negative errno*/)); rd_free(iov.iov_base); err = RD_KAFKA_RESP_ERR__BAD_COMPRESSION; goto err; } } } break; #endif case RD_KAFKA_COMPRESSION_LZ4: { err = rd_kafka_lz4_decompress(msetr->msetr_rkb, /* Proper HC? */ MsgVersion >= 1 ? 1 : 0, Offset, /* @warning Will modify compressed * if no proper HC */ (char *)compressed, compressed_size, &iov.iov_base, &iov.iov_len); if (err) goto err; } break; default: rd_rkb_dbg(msetr->msetr_rkb, MSG, "CODEC", "%s [%"PRId32"]: Message at offset %"PRId64 " with unsupported " "compression codec 0x%x: message ignored", rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition, Offset, (int)codec); err = RD_KAFKA_RESP_ERR__NOT_IMPLEMENTED; goto err; } rd_assert(iov.iov_base); /* * Decompression successful */ /* Create a new buffer pointing to the uncompressed * allocated buffer (outbuf) and let messages keep a reference to * this new buffer. */ rkbufz = rd_kafka_buf_new_shadow(iov.iov_base, iov.iov_len, rd_free); rkbufz->rkbuf_rkb = msetr->msetr_rkbuf->rkbuf_rkb; rd_kafka_broker_keep(rkbufz->rkbuf_rkb); /* In MsgVersion v0..1 the decompressed data contains * an inner MessageSet, pass it to a new MessageSet reader. * * For MsgVersion v2 the decompressed data are the list of messages. */ if (MsgVersion <= 1) { /* Pass decompressed data (inner Messageset) * to new instance of the MessageSet parser. */ rd_kafka_msgset_reader_t inner_msetr; rd_kafka_msgset_reader_init(&inner_msetr, rkbufz, msetr->msetr_rktp, msetr->msetr_tver, &msetr->msetr_rkq); if (MsgVersion == 1) { /* postproc() will convert relative to * absolute offsets */ inner_msetr.msetr_relative_offsets = 1; inner_msetr.msetr_outer.offset = Offset; /* Apply single LogAppendTime timestamp for * all messages. */ if (Attributes & RD_KAFKA_MSG_ATTR_LOG_APPEND_TIME) { inner_msetr.msetr_outer.tstype = RD_KAFKA_TIMESTAMP_LOG_APPEND_TIME; inner_msetr.msetr_outer.timestamp = Timestamp; } } /* Parse the inner MessageSet */ err = rd_kafka_msgset_reader_run(&inner_msetr); } else { /* MsgVersion 2 */ rd_kafka_buf_t *orig_rkbuf = msetr->msetr_rkbuf; /* Temporarily replace read buffer with uncompressed buffer */ msetr->msetr_rkbuf = rkbufz; /* Read messages */ err = rd_kafka_msgset_reader_msgs_v2(msetr); /* Restore original buffer */ msetr->msetr_rkbuf = orig_rkbuf; } /* Loose our refcnt of the uncompressed rkbuf. * Individual messages/rko's will have their own reference. */ rd_kafka_buf_destroy(rkbufz); return err; err: /* Enqueue error messsage: * Create op and push on temporary queue. */ rd_kafka_q_op_err(&msetr->msetr_rkq, RD_KAFKA_OP_CONSUMER_ERR, err, msetr->msetr_tver->version, rktp, Offset, "Decompression (codec 0x%x) of message at %"PRIu64 " of %"PRIu64" bytes failed: %s", codec, Offset, compressed_size, rd_kafka_err2str(err)); return err; } /** * @brief Message parser for MsgVersion v0..1 * * @returns RD_KAFKA_RESP_ERR_NO_ERROR on success or on single-message errors, * or any other error code when the MessageSet parser should stop * parsing (such as for partial Messages). */ static rd_kafka_resp_err_t rd_kafka_msgset_reader_msg_v0_1 (rd_kafka_msgset_reader_t *msetr) { rd_kafka_buf_t *rkbuf = msetr->msetr_rkbuf; rd_kafka_toppar_t *rktp = msetr->msetr_rktp; rd_kafka_broker_t *rkb = msetr->msetr_rkb; struct { int64_t Offset; /* MessageSet header */ int32_t MessageSize; /* MessageSet header */ uint32_t Crc; int8_t MagicByte; /* MsgVersion */ int8_t Attributes; int64_t Timestamp; /* v1 */ } hdr; /* Message header */ rd_kafkap_bytes_t Key; rd_kafkap_bytes_t Value; int32_t Value_len; rd_kafka_op_t *rko; size_t hdrsize = 6; /* Header size following MessageSize */ rd_slice_t crc_slice; rd_kafka_msg_t *rkm; int relative_offsets = 0; const char *reloff_str = ""; /* Only log decoding errors if protocol debugging enabled. */ int log_decode_errors = (rkbuf->rkbuf_rkb->rkb_rk->rk_conf.debug & RD_KAFKA_DBG_PROTOCOL) ? LOG_DEBUG : 0; size_t message_end; rd_kafka_buf_read_i64(rkbuf, &hdr.Offset); rd_kafka_buf_read_i32(rkbuf, &hdr.MessageSize); message_end = rd_slice_offset(&rkbuf->rkbuf_reader) + hdr.MessageSize; rd_kafka_buf_read_i32(rkbuf, &hdr.Crc); if (!rd_slice_narrow_copy_relative(&rkbuf->rkbuf_reader, &crc_slice, hdr.MessageSize - 4)) rd_kafka_buf_check_len(rkbuf, hdr.MessageSize - 4); rd_kafka_buf_read_i8(rkbuf, &hdr.MagicByte); rd_kafka_buf_read_i8(rkbuf, &hdr.Attributes); if (hdr.MagicByte == 1) { /* MsgVersion */ rd_kafka_buf_read_i64(rkbuf, &hdr.Timestamp); hdrsize += 8; /* MsgVersion 1 has relative offsets for compressed MessageSets*/ if (!(hdr.Attributes & RD_KAFKA_MSG_ATTR_COMPRESSION_MASK) && msetr->msetr_relative_offsets) { relative_offsets = 1; reloff_str = "relative "; } } else hdr.Timestamp = 0; /* Verify MessageSize */ if (unlikely(hdr.MessageSize < (ssize_t)hdrsize)) rd_kafka_buf_parse_fail(rkbuf, "Message at %soffset %"PRId64 " MessageSize %"PRId32 " < hdrsize %"PRIusz, reloff_str, hdr.Offset, hdr.MessageSize, hdrsize); /* Early check for partial messages */ rd_kafka_buf_check_len(rkbuf, hdr.MessageSize - hdrsize); if (rkb->rkb_rk->rk_conf.check_crcs) { /* Verify CRC32 if desired. */ uint32_t calc_crc; calc_crc = rd_slice_crc32(&crc_slice); rd_dassert(rd_slice_remains(&crc_slice) == 0); if (unlikely(hdr.Crc != calc_crc)) { /* Propagate CRC error to application and * continue with next message. */ rd_kafka_q_op_err(&msetr->msetr_rkq, RD_KAFKA_OP_CONSUMER_ERR, RD_KAFKA_RESP_ERR__BAD_MSG, msetr->msetr_tver->version, rktp, hdr.Offset, "Message at %soffset %"PRId64 " (%"PRId32" bytes) " "failed CRC32 check " "(original 0x%"PRIx32" != " "calculated 0x%"PRIx32")", reloff_str, hdr.Offset, hdr.MessageSize, hdr.Crc, calc_crc); rd_kafka_buf_skip_to(rkbuf, message_end); rd_atomic64_add(&rkb->rkb_c.rx_err, 1); /* Continue with next message */ return RD_KAFKA_RESP_ERR_NO_ERROR; } } /* Extract key */ rd_kafka_buf_read_bytes(rkbuf, &Key); /* Extract Value */ rd_kafka_buf_read_bytes(rkbuf, &Value); Value_len = RD_KAFKAP_BYTES_LEN(&Value); /* MessageSets may contain offsets earlier than we * requested (compressed MessageSets in particular), * drop the earlier messages. * Note: the inner offset may only be trusted for * absolute offsets. KIP-31 introduced * ApiVersion 2 that maintains relative offsets * of compressed messages and the base offset * in the outer message is the offset of * the *LAST* message in the MessageSet. * This requires us to assign offsets * after all messages have been read from * the messageset, and it also means * we cant perform this offset check here * in that case. */ if (!relative_offsets && hdr.Offset < rktp->rktp_offsets.fetch_offset) return RD_KAFKA_RESP_ERR_NO_ERROR; /* Continue with next msg */ /* Handle compressed MessageSet */ if (unlikely(hdr.Attributes & RD_KAFKA_MSG_ATTR_COMPRESSION_MASK)) return rd_kafka_msgset_reader_decompress( msetr, hdr.MagicByte, hdr.Attributes, hdr.Timestamp, hdr.Offset, Value.data, Value_len); /* Pure uncompressed message, this is the innermost * handler after all compression and cascaded * MessageSets have been peeled off. */ /* Create op/message container for message. */ rko = rd_kafka_op_new_fetch_msg(&rkm, rktp, msetr->msetr_tver->version, rkbuf, hdr.Offset, (size_t)RD_KAFKAP_BYTES_LEN(&Key), RD_KAFKAP_BYTES_IS_NULL(&Key) ? NULL : Key.data, (size_t)RD_KAFKAP_BYTES_LEN(&Value), RD_KAFKAP_BYTES_IS_NULL(&Value) ? NULL : Value.data); /* Assign message timestamp. * If message was in a compressed MessageSet and the outer/wrapper * Message.Attribute had a LOG_APPEND_TIME set, use the * outer timestamp */ if (msetr->msetr_outer.tstype == RD_KAFKA_TIMESTAMP_LOG_APPEND_TIME) { rkm->rkm_timestamp = msetr->msetr_outer.timestamp; rkm->rkm_tstype = msetr->msetr_outer.tstype; } else if (hdr.MagicByte >= 1 && hdr.Timestamp) { rkm->rkm_timestamp = hdr.Timestamp; if (hdr.Attributes & RD_KAFKA_MSG_ATTR_LOG_APPEND_TIME) rkm->rkm_tstype = RD_KAFKA_TIMESTAMP_LOG_APPEND_TIME; else rkm->rkm_tstype = RD_KAFKA_TIMESTAMP_CREATE_TIME; } /* Enqueue message on temporary queue */ rd_kafka_q_enq(&msetr->msetr_rkq, rko); msetr->msetr_msgcnt++; return RD_KAFKA_RESP_ERR_NO_ERROR; /* Continue */ err_parse: /* Count all parse errors as partial message errors. */ rd_atomic64_add(&msetr->msetr_rkb->rkb_c.rx_partial, 1); return rkbuf->rkbuf_err; }
int main_0004_conf (int argc, char **argv) { 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; const char *topic; static const char *gconfs[] = { "message.max.bytes", "12345", /* int property */ "client.id", "my id", /* string property */ "debug", "topic,metadata", /* S2F property */ "topic.blacklist", "__.*", /* #778 */ "auto.offset.reset", "earliest", /* Global->Topic fallthru */ #if WITH_ZLIB "compression.codec", "gzip", /* S2I property */ #endif 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, 10); rd_kafka_conf_destroy(ignore_conf); rd_kafka_topic_conf_destroy(ignore_topic_conf); topic = test_mk_topic_name("0004", 0); /* 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 = test_create_handle(RD_KAFKA_PRODUCER, conf); rkt = rd_kafka_topic_new(rk, topic, tconf); if (!rkt) TEST_FAIL("Failed to create topic: %s\n", rd_strerror(errno)); rd_kafka_topic_destroy(rkt); rd_kafka_destroy(rk); /* copied */ rk = test_create_handle(RD_KAFKA_PRODUCER, conf2); rkt = rd_kafka_topic_new(rk, topic, tconf2); if (!rkt) TEST_FAIL("Failed to create topic: %s\n", rd_strerror(errno)); rd_kafka_topic_destroy(rkt); rd_kafka_destroy(rk); /* Incremental S2F property. * NOTE: The order of fields returned in get() is hardcoded here. */ { static const char *s2fs[] = { "generic,broker,queue,cgrp", "generic,broker,queue,cgrp", "-broker,+queue,topic", "generic,topic,queue,cgrp", "-all,security,-fetch,+metadata", "metadata,security", NULL }; TEST_SAY("Incremental S2F tests\n"); conf = rd_kafka_conf_new(); for (i = 0 ; s2fs[i] ; i += 2) { const char *val; TEST_SAY(" Set: %s\n", s2fs[i]); test_conf_set(conf, "debug", s2fs[i]); val = test_conf_get(conf, "debug"); TEST_SAY(" Now: %s\n", val); if (strcmp(val, s2fs[i+1])) TEST_FAIL_LATER("\n" "Expected: %s\n" " Got: %s", s2fs[i+1], val); } rd_kafka_conf_destroy(conf); } /* Canonical int values, aliases, s2i-verified strings */ { static const struct { const char *prop; const char *val; const char *exp; int is_global; } props[] = { { "request.required.acks", "0", "0" }, { "request.required.acks", "-1", "-1" }, { "request.required.acks", "1", "1" }, { "acks", "3", "3" }, /* alias test */ { "request.required.acks", "393", "393" }, { "request.required.acks", "bad", NULL }, { "request.required.acks", "all", "-1" }, { "request.required.acks", "all", "-1", 1/*fallthru*/ }, { "acks", "0", "0" }, /* alias test */ #if WITH_SASL { "sasl.mechanisms", "GSSAPI", "GSSAPI", 1 }, { "sasl.mechanisms", "PLAIN", "PLAIN", 1 }, { "sasl.mechanisms", "GSSAPI,PLAIN", NULL, 1 }, { "sasl.mechanisms", "", NULL, 1 }, #endif { NULL } }; TEST_SAY("Canonical tests\n"); tconf = rd_kafka_topic_conf_new(); conf = rd_kafka_conf_new(); for (i = 0 ; props[i].prop ; i++) { char dest[64]; size_t destsz; rd_kafka_conf_res_t res; TEST_SAY(" Set: %s=%s expect %s (%s)\n", props[i].prop, props[i].val, props[i].exp, props[i].is_global ? "global":"topic"); /* Set value */ if (props[i].is_global) res = rd_kafka_conf_set(conf, props[i].prop, props[i].val, errstr, sizeof(errstr)); else res = rd_kafka_topic_conf_set(tconf, props[i].prop, props[i].val, errstr, sizeof(errstr)); if ((res == RD_KAFKA_CONF_OK ? 1:0) != (props[i].exp ? 1:0)) TEST_FAIL("Expected %s, got %s", props[i].exp ? "success" : "failure", (res == RD_KAFKA_CONF_OK ? "OK" : (res == RD_KAFKA_CONF_INVALID ? "INVALID" : "UNKNOWN"))); if (!props[i].exp) continue; /* Get value and compare to expected result */ destsz = sizeof(dest); if (props[i].is_global) res = rd_kafka_conf_get(conf, props[i].prop, dest, &destsz); else res = rd_kafka_topic_conf_get(tconf, props[i].prop, dest, &destsz); TEST_ASSERT(res == RD_KAFKA_CONF_OK, ".._conf_get(%s) returned %d", props[i].prop, res); TEST_ASSERT(!strcmp(props[i].exp, dest), "Expected \"%s\", got \"%s\"", props[i].exp, dest); } rd_kafka_topic_conf_destroy(tconf); rd_kafka_conf_destroy(conf); } return 0; }
/** * @brief Test delivery report events */ int main_0039_event_dr (int argc, char **argv) { 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 msg[128]; int msgcnt = test_on_ci ? 5000 : 50000; int i; test_timing_t t_produce, t_delivery; rd_kafka_queue_t *eventq; test_conf_init(&conf, &topic_conf, 10); /* Set delivery report callback */ rd_kafka_conf_set_dr_msg_cb(conf, test_dr_msg_cb); rd_kafka_conf_set_events(conf, RD_KAFKA_EVENT_DR); /* Create kafka instance */ rk = test_create_handle(RD_KAFKA_PRODUCER, conf); eventq = rd_kafka_queue_get_main(rk); rkt = rd_kafka_topic_new(rk, test_mk_topic_name("0005", 0), topic_conf); if (!rkt) TEST_FAIL("Failed to create topic: %s\n", rd_strerror(errno)); /* Produce messages */ TIMING_START(&t_produce, "PRODUCE"); for (i = 0 ; i < msgcnt ; i++) { int *msgidp = malloc(sizeof(*msgidp)); *msgidp = i; rd_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, rd_strerror(errno)); } TIMING_STOP(&t_produce); TEST_SAY("Produced %i messages, waiting for deliveries\n", msgcnt); /* Wait for messages to be delivered */ TIMING_START(&t_delivery, "DELIVERY"); while (rd_kafka_outq_len(rk) > 0) { rd_kafka_event_t *rkev; rkev = rd_kafka_queue_poll(eventq, 1000); switch (rd_kafka_event_type(rkev)) { case RD_KAFKA_EVENT_DR: TEST_SAYL(3, "%s event with %zd messages\n", rd_kafka_event_name(rkev), rd_kafka_event_message_count(rkev)); handle_drs(rkev); break; default: TEST_SAY("Unhandled event: %s\n", rd_kafka_event_name(rkev)); break; } rd_kafka_event_destroy(rkev); } TIMING_STOP(&t_delivery); 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); rd_kafka_queue_destroy(eventq); /* 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); return 0; }