static rd_kafka_resp_err_t rd_kafka_offset_broker_commit (rd_kafka_toppar_t *rktp) { rd_kafka_topic_partition_list_t *offsets; rd_kafka_topic_partition_t *rktpar; rd_kafka_assert(rktp->rktp_rkt->rkt_rk, rktp->rktp_cgrp != NULL); rd_kafka_assert(rktp->rktp_rkt->rkt_rk, rktp->rktp_flags & RD_KAFKA_TOPPAR_F_OFFSET_STORE); rktp->rktp_committing_offset = rktp->rktp_stored_offset; offsets = rd_kafka_topic_partition_list_new(1); rktpar = rd_kafka_topic_partition_list_add( offsets, rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition); rktpar->offset = rktp->rktp_committing_offset; rd_kafka_dbg(rktp->rktp_rkt->rkt_rk, TOPIC, "OFFSETCMT", "%.*s [%"PRId32"]: committing offset %"PRId64, RD_KAFKAP_STR_PR(rktp->rktp_rkt->rkt_topic), rktp->rktp_partition, rktp->rktp_committing_offset); rd_kafka_commit0(rktp->rktp_rkt->rkt_rk, offsets, rktp, RD_KAFKA_REPLYQ(rktp->rktp_ops, 0), rd_kafka_offset_broker_commit_cb, NULL); rd_kafka_topic_partition_list_destroy(offsets); return RD_KAFKA_RESP_ERR__IN_PROGRESS; }
void rd_kafka_timers_destroy (rd_kafka_timers_t *rkts) { rd_kafka_timer_t *rtmr; rd_kafka_assert(NULL, TAILQ_EMPTY(&rkts->rkts_timers)); while ((rtmr = TAILQ_FIRST(&rkts->rkts_timers))) rd_kafka_timer_stop(rkts, rtmr, 0); rd_kafka_assert(rkts->rkts_rk, TAILQ_EMPTY(&rkts->rkts_timers)); cnd_destroy(&rkts->rkts_cond); mtx_destroy(&rkts->rkts_lock); }
void rd_kafka_op_destroy (rd_kafka_op_t *rko) { /* Decrease refcount on rkbuf to eventually rd_free the shared buffer*/ if (rko->rko_rkbuf) rd_kafka_buf_handle_op(rko, RD_KAFKA_RESP_ERR__DESTROY); else if (rko->rko_payload && rko->rko_flags & RD_KAFKA_OP_F_FREE) { if (rko->rko_free_cb) rko->rko_free_cb(rko->rko_payload); else rd_free(rko->rko_payload); } if (rko->rko_rkt) rd_kafka_topic_destroy0(rd_kafka_topic_a2s(rko->rko_rkt)); if (rko->rko_rktp) rd_kafka_toppar_destroy(rko->rko_rktp); if (rko->rko_metadata) rd_kafka_metadata_destroy(rko->rko_metadata); if (rko->rko_replyq) rd_kafka_q_destroy(rko->rko_replyq); if (rd_atomic32_sub(&rd_kafka_op_cnt, 1) < 0) rd_kafka_assert(NULL, !*"rd_kafka_op_cnt < 0"); rd_free(rko); }
/** * @brief Convert relative to absolute offsets and also purge any messages * that are older than \p min_offset. * @remark Error ops with ERR__NOT_IMPLEMENTED will not be purged since * they are used to indicate unknnown compression codecs and compressed * messagesets may have a starting offset lower than what we requested. * @remark \p rkq locking is not performed (caller's responsibility) * @remark Must NOT be used on fwdq. */ void rd_kafka_q_fix_offsets (rd_kafka_q_t *rkq, int64_t min_offset, int64_t base_offset) { rd_kafka_op_t *rko, *next; int adj_len = 0; int64_t adj_size = 0; rd_kafka_assert(NULL, !rkq->rkq_fwdq); next = TAILQ_FIRST(&rkq->rkq_q); while ((rko = next)) { next = TAILQ_NEXT(next, rko_link); if (unlikely(rko->rko_type != RD_KAFKA_OP_FETCH)) continue; if (rko->rko_u.fetch.rkm.rkm_offset < min_offset && rko->rko_err != RD_KAFKA_RESP_ERR__NOT_IMPLEMENTED) { adj_len++; adj_size += rko->rko_len; TAILQ_REMOVE(&rkq->rkq_q, rko, rko_link); rd_kafka_op_destroy(rko); continue; } rko->rko_u.fetch.rkm.rkm_offset += base_offset; } rkq->rkq_qlen -= adj_len; rkq->rkq_qsize -= adj_size; }
rd_kafka_message_t *rd_kafka_message_get (rd_kafka_op_t *rko) { rd_kafka_message_t *rkmessage; if (!rko) return rd_kafka_message_new(); /* empty */ switch (rko->rko_type) { case RD_KAFKA_OP_FETCH: /* Use embedded rkmessage */ rkmessage = &rko->rko_u.fetch.rkm.rkm_rkmessage; break; case RD_KAFKA_OP_ERR: case RD_KAFKA_OP_CONSUMER_ERR: rkmessage = &rko->rko_u.err.rkm.rkm_rkmessage; rkmessage->payload = rko->rko_u.err.errstr; rkmessage->offset = rko->rko_u.err.offset; break; default: rd_kafka_assert(NULL, !*"unhandled optype"); RD_NOTREACHED(); return NULL; } return rd_kafka_message_setup(rko, rkmessage); }
/** * sprintf a message in rko->rko_payload (typically error string) */ void rd_kafka_op_sprintf (rd_kafka_op_t *rko, const char *fmt, ...) { va_list ap; char buf[2048]; va_start(ap, fmt); rd_vsnprintf(buf, sizeof(buf), fmt, ap); va_end(ap); rd_kafka_assert(NULL, !rko->rko_payload); rko->rko_payload = rd_strdup(buf); rko->rko_len = strlen(buf); rko->rko_flags |= RD_KAFKA_OP_F_FREE; rko->rko_free_cb = rd_free; }
/** * Trigger offset_commit_cb op, if configured. * */ void rd_kafka_offset_commit_cb_op (rd_kafka_t *rk, rd_kafka_resp_err_t err, const rd_kafka_topic_partition_list_t *offsets) { rd_kafka_op_t *rko; if (!rk->rk_conf.offset_commit_cb) return; rko = rd_kafka_op_new(RD_KAFKA_OP_OFFSET_COMMIT|RD_KAFKA_OP_REPLY); rko->rko_err = err; rd_kafka_assert(NULL, offsets->cnt > 0); rd_kafka_op_payload_set(rko, rd_kafka_topic_partition_list_copy(offsets), (void *)rd_kafka_topic_partition_list_destroy); rd_kafka_q_enq(&rk->rk_rep, rko); }
rd_kafka_resp_err_t rd_kafka_metadata (rd_kafka_t *rk, int all_topics, rd_kafka_topic_t *only_rkt, const struct rd_kafka_metadata **metadatap, int timeout_ms) { rd_kafka_q_t *rkq; rd_kafka_broker_t *rkb; rd_kafka_op_t *rko; rd_ts_t ts_end = rd_timeout_init(timeout_ms); rd_list_t topics; /* Query any broker that is up, and if none are up pick the first one, * if we're lucky it will be up before the timeout */ rkb = rd_kafka_broker_any_usable(rk, timeout_ms, 1, "application metadata request"); if (!rkb) return RD_KAFKA_RESP_ERR__TRANSPORT; rkq = rd_kafka_q_new(rk); rd_list_init(&topics, 0, rd_free); if (!all_topics) { if (only_rkt) rd_list_add(&topics, rd_strdup(rd_kafka_topic_a2i(only_rkt)-> rkt_topic->str)); else rd_kafka_local_topics_to_list(rkb->rkb_rk, &topics); } /* Async: request metadata */ rko = rd_kafka_op_new(RD_KAFKA_OP_METADATA); rd_kafka_op_set_replyq(rko, rkq, 0); rko->rko_u.metadata.force = 1; /* Force metadata request regardless * of outstanding metadata requests. */ rd_kafka_MetadataRequest(rkb, &topics, "application requested", rko); rd_list_destroy(&topics); rd_kafka_broker_destroy(rkb); /* Wait for reply (or timeout) */ rko = rd_kafka_q_pop(rkq, rd_timeout_remains(ts_end), 0); rd_kafka_q_destroy_owner(rkq); /* Timeout */ if (!rko) return RD_KAFKA_RESP_ERR__TIMED_OUT; /* Error */ if (rko->rko_err) { rd_kafka_resp_err_t err = rko->rko_err; rd_kafka_op_destroy(rko); return err; } /* Reply: pass metadata pointer to application who now owns it*/ rd_kafka_assert(rk, rko->rko_u.metadata.md); *metadatap = rko->rko_u.metadata.md; rko->rko_u.metadata.md = NULL; rd_kafka_op_destroy(rko); return RD_KAFKA_RESP_ERR_NO_ERROR; }
/** * @brief Handle a Metadata response message. * * @param topics are the requested topics (may be NULL) * * The metadata will be marshalled into 'struct rd_kafka_metadata*' structs. * * The marshalled metadata is returned in \p *mdp, (NULL on error). * @returns an error code on parse failure, else NO_ERRRO. * * @locality rdkafka main thread */ rd_kafka_resp_err_t rd_kafka_parse_Metadata (rd_kafka_broker_t *rkb, rd_kafka_buf_t *request, rd_kafka_buf_t *rkbuf, struct rd_kafka_metadata **mdp) { rd_kafka_t *rk = rkb->rkb_rk; int i, j, k; rd_tmpabuf_t tbuf; struct rd_kafka_metadata *md; size_t rkb_namelen; const int log_decode_errors = LOG_ERR; rd_list_t *missing_topics = NULL; const rd_list_t *requested_topics = request->rkbuf_u.Metadata.topics; int all_topics = request->rkbuf_u.Metadata.all_topics; const char *reason = request->rkbuf_u.Metadata.reason ? request->rkbuf_u.Metadata.reason : "(no reason)"; int ApiVersion = request->rkbuf_reqhdr.ApiVersion; rd_kafkap_str_t cluster_id = RD_ZERO_INIT; int32_t controller_id = -1; rd_kafka_resp_err_t err = RD_KAFKA_RESP_ERR_NO_ERROR; int broadcast_changes = 0; rd_kafka_assert(NULL, thrd_is_current(rk->rk_thread)); /* Remove topics from missing_topics as they are seen in Metadata. */ if (requested_topics) missing_topics = rd_list_copy(requested_topics, rd_list_string_copy, NULL); rd_kafka_broker_lock(rkb); rkb_namelen = strlen(rkb->rkb_name)+1; /* We assume that the marshalled representation is * no more than 4 times larger than the wire representation. */ rd_tmpabuf_new(&tbuf, sizeof(*md) + rkb_namelen + (rkbuf->rkbuf_totlen * 4), 0/*dont assert on fail*/); if (!(md = rd_tmpabuf_alloc(&tbuf, sizeof(*md)))) { err = RD_KAFKA_RESP_ERR__CRIT_SYS_RESOURCE; goto err; } md->orig_broker_id = rkb->rkb_nodeid; md->orig_broker_name = rd_tmpabuf_write(&tbuf, rkb->rkb_name, rkb_namelen); rd_kafka_broker_unlock(rkb); /* Read Brokers */ rd_kafka_buf_read_i32a(rkbuf, md->broker_cnt); if (md->broker_cnt > RD_KAFKAP_BROKERS_MAX) rd_kafka_buf_parse_fail(rkbuf, "Broker_cnt %i > BROKERS_MAX %i", md->broker_cnt, RD_KAFKAP_BROKERS_MAX); if (!(md->brokers = rd_tmpabuf_alloc(&tbuf, md->broker_cnt * sizeof(*md->brokers)))) rd_kafka_buf_parse_fail(rkbuf, "%d brokers: tmpabuf memory shortage", md->broker_cnt); for (i = 0 ; i < md->broker_cnt ; i++) { rd_kafka_buf_read_i32a(rkbuf, md->brokers[i].id); rd_kafka_buf_read_str_tmpabuf(rkbuf, &tbuf, md->brokers[i].host); rd_kafka_buf_read_i32a(rkbuf, md->brokers[i].port); if (ApiVersion >= 1) { rd_kafkap_str_t rack; rd_kafka_buf_read_str(rkbuf, &rack); } } if (ApiVersion >= 2) rd_kafka_buf_read_str(rkbuf, &cluster_id); if (ApiVersion >= 1) { rd_kafka_buf_read_i32(rkbuf, &controller_id); rd_rkb_dbg(rkb, METADATA, "METADATA", "ClusterId: %.*s, ControllerId: %"PRId32, RD_KAFKAP_STR_PR(&cluster_id), controller_id); } /* Read TopicMetadata */ rd_kafka_buf_read_i32a(rkbuf, md->topic_cnt); rd_rkb_dbg(rkb, METADATA, "METADATA", "%i brokers, %i topics", md->broker_cnt, md->topic_cnt); if (md->topic_cnt > RD_KAFKAP_TOPICS_MAX) rd_kafka_buf_parse_fail(rkbuf, "TopicMetadata_cnt %"PRId32 " > TOPICS_MAX %i", md->topic_cnt, RD_KAFKAP_TOPICS_MAX); if (!(md->topics = rd_tmpabuf_alloc(&tbuf, md->topic_cnt * sizeof(*md->topics)))) rd_kafka_buf_parse_fail(rkbuf, "%d topics: tmpabuf memory shortage", md->topic_cnt); for (i = 0 ; i < md->topic_cnt ; i++) { rd_kafka_buf_read_i16a(rkbuf, md->topics[i].err); rd_kafka_buf_read_str_tmpabuf(rkbuf, &tbuf, md->topics[i].topic); if (ApiVersion >= 1) { int8_t is_internal; rd_kafka_buf_read_i8(rkbuf, &is_internal); } /* PartitionMetadata */ rd_kafka_buf_read_i32a(rkbuf, md->topics[i].partition_cnt); if (md->topics[i].partition_cnt > RD_KAFKAP_PARTITIONS_MAX) rd_kafka_buf_parse_fail(rkbuf, "TopicMetadata[%i]." "PartitionMetadata_cnt %i " "> PARTITIONS_MAX %i", i, md->topics[i].partition_cnt, RD_KAFKAP_PARTITIONS_MAX); if (!(md->topics[i].partitions = rd_tmpabuf_alloc(&tbuf, md->topics[i].partition_cnt * sizeof(*md->topics[i].partitions)))) rd_kafka_buf_parse_fail(rkbuf, "%s: %d partitions: " "tmpabuf memory shortage", md->topics[i].topic, md->topics[i].partition_cnt); for (j = 0 ; j < md->topics[i].partition_cnt ; j++) { rd_kafka_buf_read_i16a(rkbuf, md->topics[i].partitions[j].err); rd_kafka_buf_read_i32a(rkbuf, md->topics[i].partitions[j].id); rd_kafka_buf_read_i32a(rkbuf, md->topics[i].partitions[j].leader); /* Replicas */ rd_kafka_buf_read_i32a(rkbuf, md->topics[i].partitions[j].replica_cnt); if (md->topics[i].partitions[j].replica_cnt > RD_KAFKAP_BROKERS_MAX) rd_kafka_buf_parse_fail(rkbuf, "TopicMetadata[%i]." "PartitionMetadata[%i]." "Replica_cnt " "%i > BROKERS_MAX %i", i, j, md->topics[i]. partitions[j]. replica_cnt, RD_KAFKAP_BROKERS_MAX); if (!(md->topics[i].partitions[j].replicas = rd_tmpabuf_alloc(&tbuf, md->topics[i]. partitions[j].replica_cnt * sizeof(*md->topics[i]. partitions[j].replicas)))) rd_kafka_buf_parse_fail( rkbuf, "%s [%"PRId32"]: %d replicas: " "tmpabuf memory shortage", md->topics[i].topic, md->topics[i].partitions[j].id, md->topics[i].partitions[j].replica_cnt); for (k = 0 ; k < md->topics[i].partitions[j].replica_cnt; k++) rd_kafka_buf_read_i32a(rkbuf, md->topics[i].partitions[j]. replicas[k]); /* Isrs */ rd_kafka_buf_read_i32a(rkbuf, md->topics[i].partitions[j].isr_cnt); if (md->topics[i].partitions[j].isr_cnt > RD_KAFKAP_BROKERS_MAX) rd_kafka_buf_parse_fail(rkbuf, "TopicMetadata[%i]." "PartitionMetadata[%i]." "Isr_cnt " "%i > BROKERS_MAX %i", i, j, md->topics[i]. partitions[j].isr_cnt, RD_KAFKAP_BROKERS_MAX); if (!(md->topics[i].partitions[j].isrs = rd_tmpabuf_alloc(&tbuf, md->topics[i]. partitions[j].isr_cnt * sizeof(*md->topics[i]. partitions[j].isrs)))) rd_kafka_buf_parse_fail( rkbuf, "%s [%"PRId32"]: %d isrs: " "tmpabuf memory shortage", md->topics[i].topic, md->topics[i].partitions[j].id, md->topics[i].partitions[j].isr_cnt); for (k = 0 ; k < md->topics[i].partitions[j].isr_cnt; k++) rd_kafka_buf_read_i32a(rkbuf, md->topics[i]. partitions[j].isrs[k]); } /* Sort partitions by partition id */ qsort(md->topics[i].partitions, md->topics[i].partition_cnt, sizeof(*md->topics[i].partitions), rd_kafka_metadata_partition_id_cmp); } /* Entire Metadata response now parsed without errors: * update our internal state according to the response. */ /* Avoid metadata updates when we're terminating. */ if (rd_kafka_terminating(rkb->rkb_rk)) { err = RD_KAFKA_RESP_ERR__DESTROY; goto done; } if (md->broker_cnt == 0 && md->topic_cnt == 0) { rd_rkb_dbg(rkb, METADATA, "METADATA", "No brokers or topics in metadata: should retry"); err = RD_KAFKA_RESP_ERR__PARTIAL; goto err; } /* Update our list of brokers. */ for (i = 0 ; i < md->broker_cnt ; i++) { rd_rkb_dbg(rkb, METADATA, "METADATA", " Broker #%i/%i: %s:%i NodeId %"PRId32, i, md->broker_cnt, md->brokers[i].host, md->brokers[i].port, md->brokers[i].id); rd_kafka_broker_update(rkb->rkb_rk, rkb->rkb_proto, &md->brokers[i]); } /* Update partition count and leader for each topic we know about */ for (i = 0 ; i < md->topic_cnt ; i++) { rd_kafka_metadata_topic_t *mdt = &md->topics[i]; rd_rkb_dbg(rkb, METADATA, "METADATA", " Topic #%i/%i: %s with %i partitions%s%s", i, md->topic_cnt, mdt->topic, mdt->partition_cnt, mdt->err ? ": " : "", mdt->err ? rd_kafka_err2str(mdt->err) : ""); /* Ignore topics in blacklist */ if (rkb->rkb_rk->rk_conf.topic_blacklist && rd_kafka_pattern_match(rkb->rkb_rk->rk_conf.topic_blacklist, mdt->topic)) { rd_rkb_dbg(rkb, TOPIC, "BLACKLIST", "Ignoring blacklisted topic \"%s\" " "in metadata", mdt->topic); continue; } /* Ignore metadata completely for temporary errors. (issue #513) * LEADER_NOT_AVAILABLE: Broker is rebalancing */ if (mdt->err == RD_KAFKA_RESP_ERR_LEADER_NOT_AVAILABLE && mdt->partition_cnt == 0) { rd_rkb_dbg(rkb, TOPIC, "METADATA", "Temporary error in metadata reply for " "topic %s (PartCnt %i): %s: ignoring", mdt->topic, mdt->partition_cnt, rd_kafka_err2str(mdt->err)); if (missing_topics) rd_list_free_cb( missing_topics, rd_list_remove_cmp(missing_topics, mdt->topic, (void *)strcmp)); continue; } /* Update local topic & partition state based on metadata */ rd_kafka_topic_metadata_update2(rkb, mdt); if (requested_topics) { rd_list_free_cb(missing_topics, rd_list_remove_cmp(missing_topics, mdt->topic, (void*)strcmp)); if (!all_topics) { rd_kafka_wrlock(rk); rd_kafka_metadata_cache_topic_update(rk, mdt); rd_kafka_wrunlock(rk); } } } /* Requested topics not seen in metadata? Propogate to topic code. */ if (missing_topics) { char *topic; rd_rkb_dbg(rkb, TOPIC, "METADATA", "%d/%d requested topic(s) seen in metadata", rd_list_cnt(requested_topics) - rd_list_cnt(missing_topics), rd_list_cnt(requested_topics)); for (i = 0 ; i < rd_list_cnt(missing_topics) ; i++) rd_rkb_dbg(rkb, TOPIC, "METADATA", "wanted %s", (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); } } }
/** * @returns a newly allocated copy of metadata \p src of size \p size */ struct rd_kafka_metadata * rd_kafka_metadata_copy (const struct rd_kafka_metadata *src, size_t size) { struct rd_kafka_metadata *md; rd_tmpabuf_t tbuf; int i; /* metadata is stored in one contigious buffer where structs and * and pointed-to fields are layed out in a memory aligned fashion. * rd_tmpabuf_t provides the infrastructure to do this. * Because of this we copy all the structs verbatim but * any pointer fields needs to be copied explicitly to update * the pointer address. */ rd_tmpabuf_new(&tbuf, size, 1/*assert on fail*/); md = rd_tmpabuf_write(&tbuf, src, sizeof(*md)); rd_tmpabuf_write_str(&tbuf, src->orig_broker_name); /* Copy Brokers */ md->brokers = rd_tmpabuf_write(&tbuf, src->brokers, md->broker_cnt * sizeof(*md->brokers)); for (i = 0 ; i < md->broker_cnt ; i++) md->brokers[i].host = rd_tmpabuf_write_str(&tbuf, src->brokers[i].host); /* Copy TopicMetadata */ md->topics = rd_tmpabuf_write(&tbuf, src->topics, md->topic_cnt * sizeof(*md->topics)); for (i = 0 ; i < md->topic_cnt ; i++) { int j; md->topics[i].topic = rd_tmpabuf_write_str(&tbuf, src->topics[i].topic); /* Copy partitions */ md->topics[i].partitions = rd_tmpabuf_write(&tbuf, src->topics[i].partitions, md->topics[i].partition_cnt * sizeof(*md->topics[i].partitions)); for (j = 0 ; j < md->topics[i].partition_cnt ; j++) { /* Copy replicas and ISRs */ md->topics[i].partitions[j].replicas = rd_tmpabuf_write(&tbuf, src->topics[i].partitions[j]. replicas, md->topics[i].partitions[j]. replica_cnt * sizeof(*md->topics[i]. partitions[j]. replicas)); md->topics[i].partitions[j].isrs = rd_tmpabuf_write(&tbuf, src->topics[i].partitions[j]. isrs, md->topics[i].partitions[j]. isr_cnt * sizeof(*md->topics[i]. partitions[j]. isrs)); } } /* Check for tmpabuf errors */ if (rd_tmpabuf_failed(&tbuf)) rd_kafka_assert(NULL, !*"metadata copy failed"); /* Delibarely not destroying the tmpabuf since we return * its allocated memory. */ return md; }
void rd_kafka_op_destroy (rd_kafka_op_t *rko) { switch (rko->rko_type & ~RD_KAFKA_OP_FLAGMASK) { case RD_KAFKA_OP_FETCH: rd_kafka_msg_destroy(NULL, &rko->rko_u.fetch.rkm); /* Decrease refcount on rkbuf to eventually rd_free shared buf*/ if (rko->rko_u.fetch.rkbuf) rd_kafka_buf_handle_op(rko, RD_KAFKA_RESP_ERR__DESTROY); break; case RD_KAFKA_OP_OFFSET_FETCH: if (rko->rko_u.offset_fetch.partitions && rko->rko_u.offset_fetch.do_free) rd_kafka_topic_partition_list_destroy( rko->rko_u.offset_fetch.partitions); break; case RD_KAFKA_OP_OFFSET_COMMIT: RD_IF_FREE(rko->rko_u.offset_commit.partitions, rd_kafka_topic_partition_list_destroy); RD_IF_FREE(rko->rko_u.offset_commit.reason, rd_free); break; case RD_KAFKA_OP_SUBSCRIBE: case RD_KAFKA_OP_GET_SUBSCRIPTION: RD_IF_FREE(rko->rko_u.subscribe.topics, rd_kafka_topic_partition_list_destroy); break; case RD_KAFKA_OP_ASSIGN: case RD_KAFKA_OP_GET_ASSIGNMENT: RD_IF_FREE(rko->rko_u.assign.partitions, rd_kafka_topic_partition_list_destroy); break; case RD_KAFKA_OP_REBALANCE: RD_IF_FREE(rko->rko_u.rebalance.partitions, rd_kafka_topic_partition_list_destroy); break; case RD_KAFKA_OP_NAME: RD_IF_FREE(rko->rko_u.name.str, rd_free); break; case RD_KAFKA_OP_ERR: case RD_KAFKA_OP_CONSUMER_ERR: RD_IF_FREE(rko->rko_u.err.errstr, rd_free); rd_kafka_msg_destroy(NULL, &rko->rko_u.err.rkm); break; break; case RD_KAFKA_OP_THROTTLE: RD_IF_FREE(rko->rko_u.throttle.nodename, rd_free); break; case RD_KAFKA_OP_STATS: RD_IF_FREE(rko->rko_u.stats.json, rd_free); break; case RD_KAFKA_OP_XMIT_RETRY: case RD_KAFKA_OP_XMIT_BUF: case RD_KAFKA_OP_RECV_BUF: if (rko->rko_u.xbuf.rkbuf) rd_kafka_buf_handle_op(rko, RD_KAFKA_RESP_ERR__DESTROY); RD_IF_FREE(rko->rko_u.xbuf.rkbuf, rd_kafka_buf_destroy); break; case RD_KAFKA_OP_DR: rd_kafka_msgq_purge(rko->rko_rk, &rko->rko_u.dr.msgq); if (rko->rko_u.dr.do_purge2) rd_kafka_msgq_purge(rko->rko_rk, &rko->rko_u.dr.msgq2); if (rko->rko_u.dr.s_rkt) rd_kafka_topic_destroy0(rko->rko_u.dr.s_rkt); break; case RD_KAFKA_OP_OFFSET_RESET: RD_IF_FREE(rko->rko_u.offset_reset.reason, rd_free); break; case RD_KAFKA_OP_METADATA: RD_IF_FREE(rko->rko_u.metadata.md, rd_kafka_metadata_destroy); break; case RD_KAFKA_OP_LOG: rd_free(rko->rko_u.log.str); break; default: break; } if (rko->rko_type & RD_KAFKA_OP_CB && rko->rko_op_cb) { rd_kafka_op_res_t res; /* Let callback clean up */ rko->rko_err = RD_KAFKA_RESP_ERR__DESTROY; res = rko->rko_op_cb(rko->rko_rk, NULL, rko); assert(res != RD_KAFKA_OP_RES_YIELD); } RD_IF_FREE(rko->rko_rktp, rd_kafka_toppar_destroy); rd_kafka_replyq_destroy(&rko->rko_replyq); #if ENABLE_DEVEL if (rd_atomic32_sub(&rd_kafka_op_cnt, 1) < 0) rd_kafka_assert(NULL, !*"rd_kafka_op_cnt < 0"); #endif rd_free(rko); }
/** * Pop all available ops from a queue and call the provided * callback for each op. * `max_cnt` limits the number of ops served, 0 = no limit. * * Returns the number of ops served. * * Locality: any thread. */ int rd_kafka_q_serve (rd_kafka_q_t *rkq, int timeout_ms, int max_cnt, int cb_type, int (*callback) (rd_kafka_t *rk, rd_kafka_op_t *rko, int cb_type, void *opaque), void *opaque) { rd_kafka_t *rk = rkq->rkq_rk; rd_kafka_op_t *rko; rd_kafka_q_t localq; int cnt = 0; int handled = 0; mtx_lock(&rkq->rkq_lock); rd_dassert(TAILQ_EMPTY(&rkq->rkq_q) || rkq->rkq_qlen > 0); if (rkq->rkq_fwdq) { rd_kafka_q_t *fwdq = rkq->rkq_fwdq; int ret; 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); ret = rd_kafka_q_serve(fwdq, timeout_ms, max_cnt, cb_type, callback, opaque); rd_kafka_q_destroy(fwdq); return ret; } if (timeout_ms == RD_POLL_INFINITE) timeout_ms = INT_MAX; /* Wait for op */ while (!(rko = TAILQ_FIRST(&rkq->rkq_q)) && timeout_ms != 0) { if (cnd_timedwait_ms(&rkq->rkq_cond, &rkq->rkq_lock, timeout_ms) != thrd_success) break; timeout_ms = 0; } if (!rko) { mtx_unlock(&rkq->rkq_lock); return 0; } /* Move the first `max_cnt` ops. */ rd_kafka_q_init(&localq, rkq->rkq_rk); rd_kafka_q_move_cnt(&localq, rkq, max_cnt == 0 ? -1/*all*/ : max_cnt, 0/*no-locks*/); mtx_unlock(&rkq->rkq_lock); rd_kafka_yield_thread = 0; /* Call callback for each op */ while ((rko = TAILQ_FIRST(&localq.rkq_q))) { handled += callback(rk, rko, cb_type, opaque); rd_kafka_q_deq0(&localq, rko); rd_kafka_op_destroy(rko); cnt++; if (unlikely(rd_kafka_yield_thread)) { /* Callback called rd_kafka_yield(), we must * stop our callback dispatching and put the * ops in localq back on the original queue head. */ if (!TAILQ_EMPTY(&localq.rkq_q)) rd_kafka_q_prepend(rkq, &localq); break; } } /* Make sure no op was left unhandled. i.e., * a consumer op ended up on the global queue. */ rd_kafka_assert(NULL, handled == cnt); rd_kafka_q_destroy(&localq); return cnt; }
/** * 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"); } }