@@ -51,6 +51,9 @@ public class DefaultStreamPoller implements StreamPoller {
51
51
// start of the batch, inclusive
52
52
private IngestionShardPointer batchStartPointer ;
53
53
54
+ // indicates the last record successfully written to the blocking queue
55
+ private IngestionShardPointer lastSuccessfulPointer ;
56
+
54
57
private ResetState resetState ;
55
58
56
59
private Set <IngestionShardPointer > persistedPointers ;
@@ -63,19 +66,23 @@ public class DefaultStreamPoller implements StreamPoller {
63
66
@ Nullable
64
67
private IngestionShardPointer maxPersistedPointer ;
65
68
69
+ private IngestionErrorStrategy errorStrategy ;
70
+
66
71
public DefaultStreamPoller (
67
72
IngestionShardPointer startPointer ,
68
73
Set <IngestionShardPointer > persistedPointers ,
69
74
IngestionShardConsumer consumer ,
70
75
IngestionEngine ingestionEngine ,
71
- ResetState resetState
76
+ ResetState resetState ,
77
+ IngestionErrorStrategy errorStrategy
72
78
) {
73
79
this (
74
80
startPointer ,
75
81
persistedPointers ,
76
82
consumer ,
77
- new MessageProcessorRunnable (new ArrayBlockingQueue <>(100 ), ingestionEngine ),
78
- resetState
83
+ new MessageProcessorRunnable (new ArrayBlockingQueue <>(100 ), ingestionEngine , errorStrategy ),
84
+ resetState ,
85
+ errorStrategy
79
86
);
80
87
}
81
88
@@ -84,7 +91,8 @@ public DefaultStreamPoller(
84
91
Set <IngestionShardPointer > persistedPointers ,
85
92
IngestionShardConsumer consumer ,
86
93
MessageProcessorRunnable processorRunnable ,
87
- ResetState resetState
94
+ ResetState resetState ,
95
+ IngestionErrorStrategy errorStrategy
88
96
) {
89
97
this .consumer = Objects .requireNonNull (consumer );
90
98
this .resetState = resetState ;
@@ -109,6 +117,7 @@ public DefaultStreamPoller(
109
117
String .format (Locale .ROOT , "stream-poller-processor-%d-%d" , consumer .getShardId (), System .currentTimeMillis ())
110
118
)
111
119
);
120
+ this .errorStrategy = errorStrategy ;
112
121
}
113
122
114
123
@ Override
@@ -188,6 +197,7 @@ protected void startPoll() {
188
197
continue ;
189
198
}
190
199
blockingQueue .put (result );
200
+ lastSuccessfulPointer = result .getPointer ();
191
201
logger .debug (
192
202
"Put message {} with pointer {} to the blocking queue" ,
193
203
String .valueOf (result .getMessage ().getPayload ()),
@@ -197,8 +207,18 @@ protected void startPoll() {
197
207
// update the batch start pointer to the next batch
198
208
batchStartPointer = consumer .nextPointer ();
199
209
} catch (Throwable e ) {
200
- // TODO better error handling
201
210
logger .error ("Error in polling the shard {}: {}" , consumer .getShardId (), e );
211
+ errorStrategy .handleError (e , IngestionErrorStrategy .ErrorStage .POLLING );
212
+
213
+ if (errorStrategy .shouldPauseIngestion (e , IngestionErrorStrategy .ErrorStage .POLLING )) {
214
+ // Blocking error encountered. Pause poller to stop processing remaining updates.
215
+ pause ();
216
+ } else {
217
+ // Advance the batch start pointer to ignore the error and continue from next record
218
+ batchStartPointer = lastSuccessfulPointer == null
219
+ ? consumer .nextPointer (batchStartPointer )
220
+ : consumer .nextPointer (lastSuccessfulPointer );
221
+ }
202
222
}
203
223
}
204
224
}
0 commit comments