Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
26 changes: 26 additions & 0 deletions src/rdkafka_mock.h
Original file line number Diff line number Diff line change
Expand Up @@ -621,6 +621,32 @@ RD_EXPORT void rd_kafka_mock_sharegroup_set_heartbeat_interval(
rd_kafka_mock_cluster_t *mcluster,
int heartbeat_interval_ms);

/**
* @brief Set the maximum number of times a record can be acquired
* before it is automatically archived (dead-lettered).
*
* Default is 5. Set to 0 for unlimited deliveries.
*
* @param mcluster Mock cluster instance.
* @param max_attempts Maximum delivery attempts per record.
*/
RD_EXPORT void rd_kafka_mock_sharegroup_set_max_delivery_attempts(
rd_kafka_mock_cluster_t *mcluster,
int max_attempts);

/**
* @brief Set the per-record lock duration in milliseconds.
*
* When a record is acquired, its lock expires after this duration.
* Default is 0, which falls back to the session timeout value.
*
* @param mcluster Mock cluster instance.
* @param lock_duration_ms Lock duration in milliseconds.
*/
RD_EXPORT void rd_kafka_mock_sharegroup_set_record_lock_duration(
rd_kafka_mock_cluster_t *mcluster,
int lock_duration_ms);

/**
* @brief Set a manual target assignment for a sharegroup.
*
Expand Down
Loading