@@ -742,7 +742,6 @@ static int rd_kafka_mock_handle_ListOffsets(rd_kafka_mock_connection_t *mconn,
742742 int32_t MaxNumOffsets ;
743743 rd_kafka_mock_partition_t * mpart = NULL ;
744744 rd_kafka_resp_err_t err = all_err ;
745- int32_t LeaderEpoch = -1 ;
746745
747746 rd_kafka_buf_read_i32 (rkbuf , & Partition );
748747
@@ -808,6 +807,7 @@ static int rd_kafka_mock_handle_ListOffsets(rd_kafka_mock_connection_t *mconn,
808807 if (rkbuf -> rkbuf_reqhdr .ApiVersion >= 4 ) {
809808 /* Response: LeaderEpoch */
810809 const rd_kafka_mock_msgset_t * mset = NULL ;
810+ int32_t leader_epoch = -1 ;
811811 rd_bool_t on_follower = rd_false ;
812812
813813 if (mpart ) {
@@ -818,12 +818,12 @@ static int rd_kafka_mock_handle_ListOffsets(rd_kafka_mock_connection_t *mconn,
818818 if (Offset >= 0 &&
819819 (mset = rd_kafka_mock_msgset_find (
820820 mpart , Offset , on_follower ))) {
821- LeaderEpoch =
821+ leader_epoch =
822822 mset -> leader_epoch ;
823823 }
824824 }
825825
826- rd_kafka_buf_write_i32 (resp , LeaderEpoch );
826+ rd_kafka_buf_write_i32 (resp , leader_epoch );
827827 }
828828
829829 /* Response: Partition tags */
@@ -835,7 +835,7 @@ static int rd_kafka_mock_handle_ListOffsets(rd_kafka_mock_connection_t *mconn,
835835 "offset %" PRId64 " (leader epoch %" PRId32
836836 ") for %s: %s" ,
837837 RD_KAFKAP_STR_PR (& Topic ), Partition ,
838- Offset , LeaderEpoch ,
838+ Offset , mpart ? mpart -> leader_epoch : -1 ,
839839 rd_kafka_offset2str (Timestamp ),
840840 rd_kafka_err2str (err ));
841841 }
@@ -930,13 +930,12 @@ static int rd_kafka_mock_handle_OffsetFetch(rd_kafka_mock_connection_t *mconn,
930930 mpart , & GroupId );
931931
932932 /* Response: CommittedOffset */
933- rd_kafka_buf_write_i64 (resp ,
934- coff ? coff -> pos .offset : -1 );
933+ rd_kafka_buf_write_i64 (resp , coff ? coff -> offset : -1 );
935934
936935 if (rkbuf -> rkbuf_reqhdr .ApiVersion >= 5 ) {
937936 /* Response: CommittedLeaderEpoch */
938937 rd_kafka_buf_write_i32 (
939- resp , coff ? coff -> pos . leader_epoch : -1 );
938+ resp , mpart ? mpart -> leader_epoch : -1 );
940939 }
941940
942941 /* Response: Metadata */
@@ -953,11 +952,10 @@ static int rd_kafka_mock_handle_OffsetFetch(rd_kafka_mock_connection_t *mconn,
953952 rd_kafka_dbg (mcluster -> rk , MOCK , "MOCK" ,
954953 "Topic %s [%" PRId32
955954 "] returning "
956- "committed offset %s"
955+ "committed offset %" PRId64
957956 " for group %s" ,
958957 mtopic -> name , mpart -> id ,
959- rd_kafka_fetch_pos2str (coff -> pos ),
960- coff -> group );
958+ coff -> offset , coff -> group );
961959 else
962960 rd_kafka_dbg (mcluster -> rk , MOCK , "MOCK" ,
963961 "Topic %.*s [%" PRId32
@@ -1111,7 +1109,6 @@ static int rd_kafka_mock_handle_OffsetCommit(rd_kafka_mock_connection_t *mconn,
11111109 rd_kafka_mock_partition_t * mpart = NULL ;
11121110 rd_kafka_resp_err_t err = all_err ;
11131111 int64_t CommittedOffset ;
1114- int32_t CommittedLeaderEpoch = -1 ;
11151112 rd_kafkap_str_t Metadata ;
11161113
11171114 rd_kafka_buf_read_i32 (rkbuf , & Partition );
@@ -1129,6 +1126,7 @@ static int rd_kafka_mock_handle_OffsetCommit(rd_kafka_mock_connection_t *mconn,
11291126 rd_kafka_buf_read_i64 (rkbuf , & CommittedOffset );
11301127
11311128 if (rkbuf -> rkbuf_reqhdr .ApiVersion >= 6 ) {
1129+ int32_t CommittedLeaderEpoch ;
11321130 rd_kafka_buf_read_i32 (rkbuf ,
11331131 & CommittedLeaderEpoch );
11341132
@@ -1147,11 +1145,9 @@ static int rd_kafka_mock_handle_OffsetCommit(rd_kafka_mock_connection_t *mconn,
11471145 rd_kafka_buf_skip_tags (rkbuf );
11481146
11491147 if (!err )
1150- rd_kafka_mock_commit_offset (
1151- mpart , & GroupId ,
1152- RD_KAFKA_FETCH_POS (CommittedOffset ,
1153- CommittedLeaderEpoch ),
1154- & Metadata );
1148+ rd_kafka_mock_commit_offset (mpart , & GroupId ,
1149+ CommittedOffset ,
1150+ & Metadata );
11551151
11561152 /* Response: ErrorCode */
11571153 rd_kafka_buf_write_i16 (resp , err );
0 commit comments