@@ -50,7 +50,7 @@ public class DefaultStreamPoller implements StreamPoller {
50
50
private ExecutorService processorThread ;
51
51
52
52
// start of the batch, inclusive
53
- private IngestionShardPointer batchStartPointer ;
53
+ private IngestionShardPointer initialBatchStartPointer ;
54
54
private boolean includeBatchStartPointer = false ;
55
55
56
56
private ResetState resetState ;
@@ -105,7 +105,7 @@ public DefaultStreamPoller(
105
105
this .consumer = Objects .requireNonNull (consumer );
106
106
this .resetState = resetState ;
107
107
this .resetValue = resetValue ;
108
- this .batchStartPointer = startPointer ;
108
+ this .initialBatchStartPointer = startPointer ;
109
109
this .state = initialState ;
110
110
this .persistedPointers = persistedPointers ;
111
111
if (!this .persistedPointers .isEmpty ()) {
@@ -170,23 +170,23 @@ protected void startPoll() {
170
170
if (resetState != ResetState .NONE ) {
171
171
switch (resetState ) {
172
172
case EARLIEST :
173
- batchStartPointer = consumer .earliestPointer ();
174
- logger .info ("Resetting offset by seeking to earliest offset {}" , batchStartPointer .asString ());
173
+ initialBatchStartPointer = consumer .earliestPointer ();
174
+ logger .info ("Resetting offset by seeking to earliest offset {}" , initialBatchStartPointer .asString ());
175
175
break ;
176
176
case LATEST :
177
- batchStartPointer = consumer .latestPointer ();
178
- logger .info ("Resetting offset by seeking to latest offset {}" , batchStartPointer .asString ());
177
+ initialBatchStartPointer = consumer .latestPointer ();
178
+ logger .info ("Resetting offset by seeking to latest offset {}" , initialBatchStartPointer .asString ());
179
179
break ;
180
180
case REWIND_BY_OFFSET :
181
- batchStartPointer = consumer .pointerFromOffset (resetValue );
182
- logger .info ("Resetting offset by seeking to offset {}" , batchStartPointer .asString ());
181
+ initialBatchStartPointer = consumer .pointerFromOffset (resetValue );
182
+ logger .info ("Resetting offset by seeking to offset {}" , initialBatchStartPointer .asString ());
183
183
break ;
184
184
case REWIND_BY_TIMESTAMP :
185
- batchStartPointer = consumer .pointerFromTimestampMillis (Long .parseLong (resetValue ));
185
+ initialBatchStartPointer = consumer .pointerFromTimestampMillis (Long .parseLong (resetValue ));
186
186
logger .info (
187
187
"Resetting offset by seeking to timestamp {}, corresponding offset {}" ,
188
188
resetValue ,
189
- batchStartPointer .asString ()
189
+ initialBatchStartPointer .asString ()
190
190
);
191
191
break ;
192
192
}
@@ -209,7 +209,8 @@ protected void startPoll() {
209
209
List <IngestionShardConsumer .ReadResult <? extends IngestionShardPointer , ? extends Message >> results ;
210
210
211
211
if (includeBatchStartPointer ) {
212
- results = consumer .readNext (batchStartPointer , true , MAX_POLL_SIZE , POLL_TIMEOUT );
212
+ results = consumer .readNext (initialBatchStartPointer , true , MAX_POLL_SIZE , POLL_TIMEOUT );
213
+ includeBatchStartPointer = false ;
213
214
} else {
214
215
results = consumer .readNext (MAX_POLL_SIZE , POLL_TIMEOUT );
215
216
}
@@ -220,38 +221,47 @@ protected void startPoll() {
220
221
}
221
222
222
223
state = State .PROCESSING ;
223
- // process the records
224
- boolean firstInBatch = true ;
225
- for (IngestionShardConsumer .ReadResult <? extends IngestionShardPointer , ? extends Message > result : results ) {
226
- if (firstInBatch ) {
227
- // update the batch start pointer to the next batch
228
- batchStartPointer = result .getPointer ();
229
- firstInBatch = false ;
230
- }
224
+ processRecords (results );
225
+ } catch (Exception e ) {
226
+ // Pause ingestion when an error is encountered while polling the streaming source.
227
+ // Currently we do not have a good way to skip past the failing messages.
228
+ // The user will have the option to manually update the offset and resume ingestion.
229
+ // todo: support retry?
230
+ logger .error ("Pausing ingestion. Fatal error occurred in polling the shard {}: {}" , consumer .getShardId (), e );
231
+ pause ();
232
+ }
233
+ }
234
+ }
231
235
232
- // check if the message is already processed
233
- if (isProcessed (result .getPointer ())) {
234
- logger .info ("Skipping message with pointer {} as it is already processed" , result .getPointer ().asString ());
235
- continue ;
236
- }
237
- totalPolledCount .inc ();
238
- blockingQueue .put (result );
239
-
240
- logger .debug (
241
- "Put message {} with pointer {} to the blocking queue" ,
242
- String .valueOf (result .getMessage ().getPayload ()),
243
- result .getPointer ().asString ()
244
- );
236
+ private void processRecords (List <IngestionShardConsumer .ReadResult <? extends IngestionShardPointer , ? extends Message >> results ) {
237
+ for (IngestionShardConsumer .ReadResult <? extends IngestionShardPointer , ? extends Message > result : results ) {
238
+ try {
239
+ // check if the message is already processed
240
+ if (isProcessed (result .getPointer ())) {
241
+ logger .debug ("Skipping message with pointer {} as it is already processed" , () -> result .getPointer ().asString ());
242
+ continue ;
245
243
}
246
- // for future reads, we do not need to include the batch start pointer, and read from the last successful pointer.
247
- includeBatchStartPointer = false ;
248
- } catch (Throwable e ) {
249
- logger .error ("Error in polling the shard {}: {}" , consumer .getShardId (), e );
244
+ totalPolledCount .inc ();
245
+ blockingQueue .put (result );
246
+
247
+ logger .debug (
248
+ "Put message {} with pointer {} to the blocking queue" ,
249
+ String .valueOf (result .getMessage ().getPayload ()),
250
+ result .getPointer ().asString ()
251
+ );
252
+ } catch (Exception e ) {
253
+ logger .error (
254
+ "Error in processing a record. Shard {}, pointer {}: {}" ,
255
+ consumer .getShardId (),
256
+ result .getPointer ().asString (),
257
+ e
258
+ );
250
259
errorStrategy .handleError (e , IngestionErrorStrategy .ErrorStage .POLLING );
251
260
252
261
if (!errorStrategy .shouldIgnoreError (e , IngestionErrorStrategy .ErrorStage .POLLING )) {
253
262
// Blocking error encountered. Pause poller to stop processing remaining updates.
254
263
pause ();
264
+ break ;
255
265
}
256
266
}
257
267
}
@@ -329,9 +339,16 @@ public boolean isClosed() {
329
339
return closed ;
330
340
}
331
341
342
+ /**
343
+ * Returns the batch start pointer from where the poller can resume in case of shard recovery. The poller and
344
+ * processor are decoupled in this implementation, and hence the latest pointer tracked by the processor acts as the
345
+ * recovery/start point. In case the processor has not started tracking, then the initial batchStartPointer used by
346
+ * the poller acts as the start point.
347
+ */
332
348
@ Override
333
349
public IngestionShardPointer getBatchStartPointer () {
334
- return batchStartPointer ;
350
+ IngestionShardPointer currentShardPointer = processorRunnable .getCurrentShardPointer ();
351
+ return currentShardPointer == null ? initialBatchStartPointer : currentShardPointer ;
335
352
}
336
353
337
354
@ Override
0 commit comments