@@ -742,6 +742,7 @@ 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 ;
745746
746747                        rd_kafka_buf_read_i32 (rkbuf , & Partition );
747748
@@ -807,7 +808,6 @@ static int rd_kafka_mock_handle_ListOffsets(rd_kafka_mock_connection_t *mconn,
807808                        if  (rkbuf -> rkbuf_reqhdr .ApiVersion  >= 4 ) {
808809                                /* Response: LeaderEpoch */ 
809810                                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-                                                 leader_epoch  = 
821+                                                 LeaderEpoch  = 
822822                                                    mset -> leader_epoch ;
823823                                        }
824824                                }
825825
826-                                 rd_kafka_buf_write_i32 (resp , leader_epoch );
826+                                 rd_kafka_buf_write_i32 (resp , LeaderEpoch );
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 , mpart  ?  mpart -> leader_epoch  :  -1 ,
838+                                      Offset , LeaderEpoch ,
839839                                     rd_kafka_offset2str (Timestamp ),
840840                                     rd_kafka_err2str (err ));
841841                }
@@ -930,12 +930,13 @@ 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 , coff  ? coff -> offset  : -1 );
933+                         rd_kafka_buf_write_i64 (resp ,
934+                                                coff  ? coff -> pos .offset  : -1 );
934935
935936                        if  (rkbuf -> rkbuf_reqhdr .ApiVersion  >= 5 ) {
936937                                /* Response: CommittedLeaderEpoch */ 
937938                                rd_kafka_buf_write_i32 (
938-                                     resp , mpart  ? mpart -> leader_epoch  : -1 );
939+                                     resp , coff  ? coff -> pos . leader_epoch  : -1 );
939940                        }
940941
941942                        /* Response: Metadata */ 
@@ -952,10 +953,11 @@ static int rd_kafka_mock_handle_OffsetFetch(rd_kafka_mock_connection_t *mconn,
952953                                rd_kafka_dbg (mcluster -> rk , MOCK , "MOCK" ,
953954                                             "Topic %s [%"  PRId32 
954955                                             "] returning " 
955-                                              "committed offset %"    PRId64 
956+                                              "committed offset %s"  
956957                                             " for group %s" ,
957958                                             mtopic -> name , mpart -> id ,
958-                                              coff -> offset , coff -> group );
959+                                              rd_kafka_fetch_pos2str (coff -> pos ),
960+                                              coff -> group );
959961                        else 
960962                                rd_kafka_dbg (mcluster -> rk , MOCK , "MOCK" ,
961963                                             "Topic %.*s [%"  PRId32 
@@ -1109,6 +1111,7 @@ static int rd_kafka_mock_handle_OffsetCommit(rd_kafka_mock_connection_t *mconn,
11091111                        rd_kafka_mock_partition_t  * mpart  =  NULL ;
11101112                        rd_kafka_resp_err_t  err           =  all_err ;
11111113                        int64_t  CommittedOffset ;
1114+                         int32_t  CommittedLeaderEpoch  =  -1 ;
11121115                        rd_kafkap_str_t  Metadata ;
11131116
11141117                        rd_kafka_buf_read_i32 (rkbuf , & Partition );
@@ -1126,7 +1129,6 @@ static int rd_kafka_mock_handle_OffsetCommit(rd_kafka_mock_connection_t *mconn,
11261129                        rd_kafka_buf_read_i64 (rkbuf , & CommittedOffset );
11271130
11281131                        if  (rkbuf -> rkbuf_reqhdr .ApiVersion  >= 6 ) {
1129-                                 int32_t  CommittedLeaderEpoch ;
11301132                                rd_kafka_buf_read_i32 (rkbuf ,
11311133                                                      & CommittedLeaderEpoch );
11321134
@@ -1145,9 +1147,11 @@ static int rd_kafka_mock_handle_OffsetCommit(rd_kafka_mock_connection_t *mconn,
11451147                        rd_kafka_buf_skip_tags (rkbuf );
11461148
11471149                        if  (!err )
1148-                                 rd_kafka_mock_commit_offset (mpart , & GroupId ,
1149-                                                             CommittedOffset ,
1150-                                                             & Metadata );
1150+                                 rd_kafka_mock_commit_offset (
1151+                                     mpart , & GroupId ,
1152+                                     RD_KAFKA_FETCH_POS (CommittedOffset ,
1153+                                                        CommittedLeaderEpoch ),
1154+                                     & Metadata );
11511155
11521156                        /* Response: ErrorCode */ 
11531157                        rd_kafka_buf_write_i16 (resp , err );
0 commit comments