/** * @brief Message parser for MsgVersion v2 */ static rd_kafka_resp_err_t rd_kafka_msgset_reader_msg_v2 (rd_kafka_msgset_reader_t *msetr) { rd_kafka_buf_t *rkbuf = msetr->msetr_rkbuf; rd_kafka_toppar_t *rktp = msetr->msetr_rktp; struct { int64_t Length; int64_t MsgAttributes; /* int8_t, but int64 req. for varint */ int64_t TimestampDelta; int64_t OffsetDelta; int64_t Offset; /* Absolute offset */ rd_kafkap_bytes_t Key; rd_kafkap_bytes_t Value; int64_t HeaderCnt; } hdr; rd_kafka_op_t *rko; rd_kafka_msg_t *rkm; /* 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_varint(rkbuf, &hdr.Length); message_end = rd_slice_offset(&rkbuf->rkbuf_reader)+(size_t)hdr.Length; rd_kafka_buf_read_varint(rkbuf, &hdr.MsgAttributes); rd_kafka_buf_read_varint(rkbuf, &hdr.TimestampDelta); rd_kafka_buf_read_varint(rkbuf, &hdr.OffsetDelta); hdr.Offset = msetr->msetr_v2_hdr->BaseOffset + hdr.OffsetDelta; /* Skip message if outdated */ if (hdr.Offset < rktp->rktp_offsets.fetch_offset) { rd_rkb_dbg(msetr->msetr_rkb, MSG, "MSG", "Skip offset %"PRId64" < fetch_offset %"PRId64, hdr.Offset, rktp->rktp_offsets.fetch_offset); rd_kafka_buf_skip_to(rkbuf, message_end); return RD_KAFKA_RESP_ERR_NO_ERROR; /* Continue with next msg */ } rd_kafka_buf_read_bytes_varint(rkbuf, &hdr.Key); rd_kafka_buf_read_bytes_varint(rkbuf, &hdr.Value); /* Ignore headers for now */ rd_kafka_buf_skip_to(rkbuf, message_end); /* 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(&hdr.Key), RD_KAFKAP_BYTES_IS_NULL(&hdr.Key) ? NULL : hdr.Key.data, (size_t)RD_KAFKAP_BYTES_LEN(&hdr.Value), RD_KAFKAP_BYTES_IS_NULL(&hdr.Value) ? NULL : hdr.Value.data); /* Set timestamp. * * When broker assigns the timestamps (LOG_APPEND_TIME) it will * assign the same timestamp for all messages in a MessageSet * using MaxTimestamp. */ if ((msetr->msetr_v2_hdr->Attributes & RD_KAFKA_MSG_ATTR_LOG_APPEND_TIME) || (hdr.MsgAttributes & RD_KAFKA_MSG_ATTR_LOG_APPEND_TIME)) { rkm->rkm_tstype = RD_KAFKA_TIMESTAMP_LOG_APPEND_TIME; rkm->rkm_timestamp = msetr->msetr_v2_hdr->MaxTimestamp; } else { rkm->rkm_tstype = RD_KAFKA_TIMESTAMP_CREATE_TIME; rkm->rkm_timestamp = msetr->msetr_v2_hdr->BaseTimestamp + hdr.TimestampDelta; } /* Enqueue message on temporary queue */ rd_kafka_q_enq(&msetr->msetr_rkq, rko); msetr->msetr_msgcnt++; return RD_KAFKA_RESP_ERR_NO_ERROR; 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; }
/** * @brief Message parser for MsgVersion v2 */ static rd_kafka_resp_err_t rd_kafka_msgset_reader_msg_v2 (rd_kafka_msgset_reader_t *msetr) { rd_kafka_buf_t *rkbuf = msetr->msetr_rkbuf; rd_kafka_toppar_t *rktp = msetr->msetr_rktp; struct { int64_t Length; int8_t MsgAttributes; int64_t TimestampDelta; int64_t OffsetDelta; int64_t Offset; /* Absolute offset */ rd_kafkap_bytes_t Key; rd_kafkap_bytes_t Value; rd_kafkap_bytes_t Headers; } hdr; rd_kafka_op_t *rko; rd_kafka_msg_t *rkm; /* 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_varint(rkbuf, &hdr.Length); message_end = rd_slice_offset(&rkbuf->rkbuf_reader)+(size_t)hdr.Length; rd_kafka_buf_read_i8(rkbuf, &hdr.MsgAttributes); rd_kafka_buf_read_varint(rkbuf, &hdr.TimestampDelta); rd_kafka_buf_read_varint(rkbuf, &hdr.OffsetDelta); hdr.Offset = msetr->msetr_v2_hdr->BaseOffset + hdr.OffsetDelta; /* Skip message if outdated */ if (hdr.Offset < rktp->rktp_offsets.fetch_offset) { rd_rkb_dbg(msetr->msetr_rkb, MSG, "MSG", "%s [%"PRId32"]: " "Skip offset %"PRId64" < fetch_offset %"PRId64, rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition, hdr.Offset, rktp->rktp_offsets.fetch_offset); rd_kafka_buf_skip_to(rkbuf, message_end); return RD_KAFKA_RESP_ERR_NO_ERROR; /* Continue with next msg */ } rd_kafka_buf_read_bytes_varint(rkbuf, &hdr.Key); rd_kafka_buf_read_bytes_varint(rkbuf, &hdr.Value); /* We parse the Headers later, just store the size (possibly truncated) * and pointer to the headers. */ hdr.Headers.len = (int32_t)(message_end - rd_slice_offset(&rkbuf->rkbuf_reader)); rd_kafka_buf_read_ptr(rkbuf, &hdr.Headers.data, hdr.Headers.len); /* 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(&hdr.Key), RD_KAFKAP_BYTES_IS_NULL(&hdr.Key) ? NULL : hdr.Key.data, (size_t)RD_KAFKAP_BYTES_LEN(&hdr.Value), RD_KAFKAP_BYTES_IS_NULL(&hdr.Value) ? NULL : hdr.Value.data); /* Store pointer to unparsed message headers, they will * be parsed on the first access. * This pointer points to the rkbuf payload. * Note: can't perform struct copy here due to const fields (MSVC) */ rkm->rkm_u.consumer.binhdrs.len = hdr.Headers.len; rkm->rkm_u.consumer.binhdrs.data = hdr.Headers.data; /* Set timestamp. * * When broker assigns the timestamps (LOG_APPEND_TIME) it will * assign the same timestamp for all messages in a MessageSet * using MaxTimestamp. */ if ((msetr->msetr_v2_hdr->Attributes & RD_KAFKA_MSG_ATTR_LOG_APPEND_TIME) || (hdr.MsgAttributes & RD_KAFKA_MSG_ATTR_LOG_APPEND_TIME)) { rkm->rkm_tstype = RD_KAFKA_TIMESTAMP_LOG_APPEND_TIME; rkm->rkm_timestamp = msetr->msetr_v2_hdr->MaxTimestamp; } else { rkm->rkm_tstype = RD_KAFKA_TIMESTAMP_CREATE_TIME; rkm->rkm_timestamp = msetr->msetr_v2_hdr->BaseTimestamp + hdr.TimestampDelta; } /* Enqueue message on temporary queue */ rd_kafka_q_enq(&msetr->msetr_rkq, rko); msetr->msetr_msgcnt++; msetr->msetr_msg_bytes += rkm->rkm_key_len + rkm->rkm_len; return RD_KAFKA_RESP_ERR_NO_ERROR; 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; }