17
17
package org .apache .kafka .streams .processor .internals ;
18
18
19
19
import org .apache .kafka .clients .consumer .ConsumerRecord ;
20
- import org .apache .kafka .clients .consumer .ConsumerRecords ;
21
20
import org .apache .kafka .common .TopicPartition ;
22
21
import org .apache .kafka .common .metrics .Sensor ;
23
- import org .apache .kafka .common .utils .LogContext ;
24
- import org .apache .kafka .streams .StreamsConfig ;
25
- import org .slf4j .Logger ;
26
22
27
23
import java .util .Collections ;
28
24
import java .util .Comparator ;
29
- import java .util .HashMap ;
30
- import java .util .HashSet ;
31
- import java .util .Iterator ;
32
25
import java .util .Map ;
33
26
import java .util .PriorityQueue ;
34
27
import java .util .Set ;
28
+ import java .util .Iterator ;
29
+ import java .util .HashSet ;
35
30
import java .util .function .Function ;
36
31
37
32
/**
58
53
*/
59
54
public class PartitionGroup {
60
55
61
- private final Logger logger ;
62
56
private final Map <TopicPartition , RecordQueue > partitionQueues ;
63
- private final Sensor enforcedProcessingSensor ;
64
- private final long maxTaskIdleMs ;
65
57
private final Sensor recordLatenessSensor ;
66
58
private final PriorityQueue <RecordQueue > nonEmptyQueuesByTime ;
67
59
68
60
private long streamTime ;
69
61
private int totalBuffered ;
70
62
private boolean allBuffered ;
71
- private final Map <TopicPartition , Long > fetchedLags = new HashMap <>();
72
- private final Map <TopicPartition , Long > idlePartitionDeadlines = new HashMap <>();
63
+
73
64
74
65
static class RecordInfo {
75
66
RecordQueue queue ;
@@ -87,144 +78,15 @@ RecordQueue queue() {
87
78
}
88
79
}
89
80
90
- PartitionGroup (final LogContext logContext ,
91
- final Map <TopicPartition , RecordQueue > partitionQueues ,
92
- final Sensor recordLatenessSensor ,
93
- final Sensor enforcedProcessingSensor ,
94
- final long maxTaskIdleMs ) {
95
- this .logger = logContext .logger (PartitionGroup .class );
81
+ PartitionGroup (final Map <TopicPartition , RecordQueue > partitionQueues , final Sensor recordLatenessSensor ) {
96
82
nonEmptyQueuesByTime = new PriorityQueue <>(partitionQueues .size (), Comparator .comparingLong (RecordQueue ::headRecordTimestamp ));
97
83
this .partitionQueues = partitionQueues ;
98
- this .enforcedProcessingSensor = enforcedProcessingSensor ;
99
- this .maxTaskIdleMs = maxTaskIdleMs ;
100
84
this .recordLatenessSensor = recordLatenessSensor ;
101
85
totalBuffered = 0 ;
102
86
allBuffered = false ;
103
87
streamTime = RecordQueue .UNKNOWN ;
104
88
}
105
89
106
- public void addFetchedMetadata (final TopicPartition partition , final ConsumerRecords .Metadata metadata ) {
107
- final Long lag = metadata .lag ();
108
- if (lag != null ) {
109
- logger .trace ("added fetched lag {}: {}" , partition , lag );
110
- fetchedLags .put (partition , lag );
111
- }
112
- }
113
-
114
- public boolean readyToProcess (final long wallClockTime ) {
115
- if (logger .isTraceEnabled ()) {
116
- for (final Map .Entry <TopicPartition , RecordQueue > entry : partitionQueues .entrySet ()) {
117
- logger .trace (
118
- "buffered/lag {}: {}/{}" ,
119
- entry .getKey (),
120
- entry .getValue ().size (),
121
- fetchedLags .get (entry .getKey ())
122
- );
123
- }
124
- }
125
- // Log-level strategy:
126
- // TRACE for messages that don't wait for fetches
127
- // TRACE when we waited for a fetch and decided to wait some more, as configured
128
- // TRACE when we are ready for processing and didn't have to enforce processing
129
- // INFO when we enforce processing, since this has to wait for fetches AND may result in disorder
130
-
131
- if (maxTaskIdleMs == StreamsConfig .MAX_TASK_IDLE_MS_DISABLED ) {
132
- if (logger .isTraceEnabled () && !allBuffered && totalBuffered > 0 ) {
133
- final Set <TopicPartition > bufferedPartitions = new HashSet <>();
134
- final Set <TopicPartition > emptyPartitions = new HashSet <>();
135
- for (final Map .Entry <TopicPartition , RecordQueue > entry : partitionQueues .entrySet ()) {
136
- if (entry .getValue ().isEmpty ()) {
137
- emptyPartitions .add (entry .getKey ());
138
- } else {
139
- bufferedPartitions .add (entry .getKey ());
140
- }
141
- }
142
- logger .trace ("Ready for processing because max.task.idle.ms is disabled." +
143
- "\n \t There may be out-of-order processing for this task as a result." +
144
- "\n \t Buffered partitions: {}" +
145
- "\n \t Non-buffered partitions: {}" ,
146
- bufferedPartitions ,
147
- emptyPartitions );
148
- }
149
- return true ;
150
- }
151
-
152
- final Set <TopicPartition > queued = new HashSet <>();
153
- Map <TopicPartition , Long > enforced = null ;
154
-
155
- for (final Map .Entry <TopicPartition , RecordQueue > entry : partitionQueues .entrySet ()) {
156
- final TopicPartition partition = entry .getKey ();
157
- final RecordQueue queue = entry .getValue ();
158
-
159
- final Long nullableFetchedLag = fetchedLags .get (partition );
160
-
161
- if (!queue .isEmpty ()) {
162
- // this partition is ready for processing
163
- idlePartitionDeadlines .remove (partition );
164
- queued .add (partition );
165
- } else if (nullableFetchedLag == null ) {
166
- // must wait to fetch metadata for the partition
167
- idlePartitionDeadlines .remove (partition );
168
- logger .trace ("Waiting to fetch data for {}" , partition );
169
- return false ;
170
- } else if (nullableFetchedLag > 0L ) {
171
- // must wait to poll the data we know to be on the broker
172
- idlePartitionDeadlines .remove (partition );
173
- logger .trace (
174
- "Lag for {} is currently {}, but no data is buffered locally. Waiting to buffer some records." ,
175
- partition ,
176
- nullableFetchedLag
177
- );
178
- return false ;
179
- } else {
180
- // p is known to have zero lag. wait for maxTaskIdleMs to see if more data shows up.
181
- // One alternative would be to set the deadline to nullableMetadata.receivedTimestamp + maxTaskIdleMs
182
- // instead. That way, we would start the idling timer as of the freshness of our knowledge about the zero
183
- // lag instead of when we happen to run this method, but realistically it's probably a small difference
184
- // and using wall clock time seems more intuitive for users,
185
- // since the log message will be as of wallClockTime.
186
- idlePartitionDeadlines .putIfAbsent (partition , wallClockTime + maxTaskIdleMs );
187
- final long deadline = idlePartitionDeadlines .get (partition );
188
- if (wallClockTime < deadline ) {
189
- logger .trace (
190
- "Lag for {} is currently 0 and current time is {}. Waiting for new data to be produced for configured idle time {} (deadline is {})." ,
191
- partition ,
192
- wallClockTime ,
193
- maxTaskIdleMs ,
194
- deadline
195
- );
196
- return false ;
197
- } else {
198
- // this partition is ready for processing due to the task idling deadline passing
199
- if (enforced == null ) {
200
- enforced = new HashMap <>();
201
- }
202
- enforced .put (partition , deadline );
203
- }
204
- }
205
- }
206
- if (enforced == null ) {
207
- logger .trace ("All partitions were buffered locally, so this task is ready for processing." );
208
- return true ;
209
- } else if (queued .isEmpty ()) {
210
- logger .trace ("No partitions were buffered locally, so this task is not ready for processing." );
211
- return false ;
212
- } else {
213
- enforcedProcessingSensor .record (1.0d , wallClockTime );
214
- logger .info ("Continuing to process although some partition timestamps were not buffered locally." +
215
- "\n \t There may be out-of-order processing for this task as a result." +
216
- "\n \t Partitions with local data: {}." +
217
- "\n \t Partitions we gave up waiting for, with their corresponding deadlines: {}." +
218
- "\n \t Configured max.task.idle.ms: {}." +
219
- "\n \t Current wall-clock time: {}." ,
220
- queued ,
221
- enforced ,
222
- maxTaskIdleMs ,
223
- wallClockTime );
224
- return true ;
225
- }
226
- }
227
-
228
90
// visible for testing
229
91
long partitionTimestamp (final TopicPartition partition ) {
230
92
final RecordQueue queue = partitionQueues .get (partition );
@@ -377,7 +239,7 @@ int numBuffered() {
377
239
return totalBuffered ;
378
240
}
379
241
380
- boolean allPartitionsBufferedLocally () {
242
+ boolean allPartitionsBuffered () {
381
243
return allBuffered ;
382
244
}
383
245
0 commit comments