@@ -36,19 +36,71 @@ const char *rd_kafka_offset2str(int64_t offset);
36
36
37
37
38
38
/**
39
- * Stores the offset for the toppar 'rktp'.
40
- * The actual commit of the offset to backing store is usually
41
- * performed at a later time (time or threshold based).
39
+ * @brief Stores the offset for the toppar 'rktp'.
40
+ * The actual commit of the offset to backing store is usually
41
+ * performed at a later time (time or threshold based).
42
+ *
43
+ * For the high-level consumer (assign()), this function will reject absolute
44
+ * offsets if the partition is not currently assigned, unless \p force is set.
45
+ * This check was added to avoid a race condition where an application
46
+ * would call offsets_store() after the partitions had been revoked, forcing
47
+ * a future auto-committer on the next assignment to commit this old offset and
48
+ * overwriting whatever newer offset was committed by another consumer.
49
+ *
50
+ * The \p force flag is useful for internal calls to offset_store0() which
51
+ * do not need the protection described above.
52
+ *
53
+ *
54
+ * There is one situation where the \p force flag is troublesome:
55
+ * If the application is using any of the consumer batching APIs,
56
+ * e.g., consume_batch() or the event-based consumption, then it's possible
57
+ * that while the batch is being accumulated or the application is picking off
58
+ * messages from the event a rebalance occurs (in the background) which revokes
59
+ * the current assignment. This revokal will remove all queued messages, but
60
+ * not the ones the application already has accumulated in the event object.
61
+ * Enforcing assignment for store in this state is tricky with a bunch of
62
+ * corner cases, so instead we let those places forcibly store the offset, but
63
+ * then in assign() we reset the stored offset to .._INVALID, just like we do
64
+ * on revoke.
65
+ * Illustrated (with fix):
66
+ * 1. ev = rd_kafka_queue_poll();
67
+ * 2. background rebalance revoke unassigns the partition and sets the
68
+ * stored offset to _INVALID.
69
+ * 3. application calls message_next(ev) which forcibly sets the
70
+ * stored offset.
71
+ * 4. background rebalance assigns the partition again, but forcibly sets
72
+ * the stored offset to .._INVALID to provide a clean state.
73
+ *
74
+ * @param offset Offset to set, may be an absolute offset or .._INVALID.
75
+ * @param force Forcibly set \p offset regardless of assignment state.
76
+ * @param do_lock Whether to lock the \p rktp or not (already locked by caller).
42
77
*
43
78
* See head of rdkafka_offset.c for more information.
79
+ *
80
+ * @returns RD_KAFKA_RESP_ERR__STATE if the partition is not currently assigned,
81
+ * unless \p force is set.
44
82
*/
45
- static RD_INLINE RD_UNUSED void
46
- rd_kafka_offset_store0 (rd_kafka_toppar_t * rktp , int64_t offset , int lock ) {
47
- if (lock )
83
+ static RD_INLINE RD_UNUSED rd_kafka_resp_err_t
84
+ rd_kafka_offset_store0 (rd_kafka_toppar_t * rktp ,
85
+ int64_t offset ,
86
+ rd_bool_t force ,
87
+ rd_dolock_t do_lock ) {
88
+ rd_kafka_resp_err_t err = RD_KAFKA_RESP_ERR_NO_ERROR ;
89
+
90
+ if (do_lock )
48
91
rd_kafka_toppar_lock (rktp );
49
- rktp -> rktp_stored_offset = offset ;
50
- if (lock )
92
+
93
+ if (unlikely (!force && !RD_KAFKA_OFFSET_IS_LOGICAL (offset ) &&
94
+ !(rktp -> rktp_flags & RD_KAFKA_TOPPAR_F_ASSIGNED ) &&
95
+ !rd_kafka_is_simple_consumer (rktp -> rktp_rkt -> rkt_rk )))
96
+ err = RD_KAFKA_RESP_ERR__STATE ;
97
+ else
98
+ rktp -> rktp_stored_offset = offset ;
99
+
100
+ if (do_lock )
51
101
rd_kafka_toppar_unlock (rktp );
102
+
103
+ return err ;
52
104
}
53
105
54
106
rd_kafka_resp_err_t
0 commit comments