-
Notifications
You must be signed in to change notification settings - Fork 460
/
ShardConsumer.java
361 lines (336 loc) · 15.2 KB
/
ShardConsumer.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
/*
* Copyright 2012-2014 Amazon.com, Inc. or its affiliates. All Rights Reserved.
*
* Licensed under the Amazon Software License (the "License").
* You may not use this file except in compliance with the License.
* A copy of the License is located at
*
* http://aws.amazon.com/asl/
*
* or in the "license" file accompanying this file. This file is distributed
* on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
* express or implied. See the License for the specific language governing
* permissions and limitations under the License.
*/
package com.amazonaws.services.kinesis.clientlibrary.lib.worker;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Future;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import com.amazonaws.services.kinesis.clientlibrary.exceptions.internal.BlockedOnParentShardException;
import com.amazonaws.services.kinesis.clientlibrary.interfaces.ICheckpoint;
import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessor;
import com.amazonaws.services.kinesis.clientlibrary.types.ShutdownReason;
import com.amazonaws.services.kinesis.leases.impl.KinesisClientLease;
import com.amazonaws.services.kinesis.leases.interfaces.ILeaseManager;
import com.amazonaws.services.kinesis.metrics.interfaces.IMetricsFactory;
/**
* Responsible for consuming data records of a (specified) shard.
* The instance should be shutdown when we lose the primary responsibility for a shard.
* A new instance should be created if the primary responsibility is reassigned back to this process.
*/
class ShardConsumer {
/**
* Enumerates processing states when working on a shard.
*/
enum ShardConsumerState {
WAITING_ON_PARENT_SHARDS, INITIALIZING, PROCESSING, SHUTTING_DOWN, SHUTDOWN_COMPLETE;
}
private static final Log LOG = LogFactory.getLog(ShardConsumer.class);
private final StreamConfig streamConfig;
private final IRecordProcessor recordProcessor;
private final RecordProcessorCheckpointer recordProcessorCheckpointer;
private final ExecutorService executorService;
private final ShardInfo shardInfo;
private final KinesisDataFetcher dataFetcher;
private final IMetricsFactory metricsFactory;
private final ILeaseManager<KinesisClientLease> leaseManager;
private ICheckpoint checkpoint;
// Backoff time when polling to check if application has finished processing parent shards
private final long parentShardPollIntervalMillis;
private final boolean cleanupLeasesOfCompletedShards;
private final long taskBackoffTimeMillis;
private ITask currentTask;
private long currentTaskSubmitTime;
private Future<TaskResult> future;
/*
* Tracks current state. It is only updated via the consumeStream/shutdown APIs. Therefore we don't do
* much coordination/synchronization to handle concurrent reads/updates.
*/
private ShardConsumerState currentState = ShardConsumerState.WAITING_ON_PARENT_SHARDS;
/*
* Used to track if we lost the primary responsibility. Once set to true, we will start shutting down.
* If we regain primary responsibility before shutdown is complete, Worker should create a new ShardConsumer object.
*/
private boolean beginShutdown;
private ShutdownReason shutdownReason;
/**
* @param shardInfo Shard information
* @param streamConfig Stream configuration to use
* @param checkpoint Checkpoint tracker
* @param recordProcessor Record processor used to process the data records for the shard
* @param leaseManager Used to create leases for new shards
* @param parentShardPollIntervalMillis Wait for this long if parent shards are not done (or we get an exception)
* @param executorService ExecutorService used to execute process tasks for this shard
* @param metricsFactory IMetricsFactory used to construct IMetricsScopes for this shard
* @param backoffTimeMillis backoff interval when we encounter exceptions
*/
// CHECKSTYLE:IGNORE ParameterNumber FOR NEXT 10 LINES
ShardConsumer(ShardInfo shardInfo,
StreamConfig streamConfig,
ICheckpoint checkpoint,
IRecordProcessor recordProcessor,
ILeaseManager<KinesisClientLease> leaseManager,
long parentShardPollIntervalMillis,
boolean cleanupLeasesOfCompletedShards,
ExecutorService executorService,
IMetricsFactory metricsFactory,
long backoffTimeMillis) {
this.streamConfig = streamConfig;
this.recordProcessor = recordProcessor;
this.executorService = executorService;
this.shardInfo = shardInfo;
this.checkpoint = checkpoint;
this.recordProcessorCheckpointer =
new RecordProcessorCheckpointer(shardInfo,
checkpoint,
new SequenceNumberValidator(streamConfig.getStreamProxy(),
shardInfo.getShardId(),
streamConfig.shouldValidateSequenceNumberBeforeCheckpointing()));
this.dataFetcher = new KinesisDataFetcher(streamConfig.getStreamProxy(), shardInfo);
this.leaseManager = leaseManager;
this.metricsFactory = metricsFactory;
this.parentShardPollIntervalMillis = parentShardPollIntervalMillis;
this.cleanupLeasesOfCompletedShards = cleanupLeasesOfCompletedShards;
this.taskBackoffTimeMillis = backoffTimeMillis;
}
/**
* No-op if current task is pending, otherwise submits next task for this shard.
* This method should NOT be called if the ShardConsumer is already in SHUTDOWN_COMPLETED state.
*
* @return true if a new process task was submitted, false otherwise
*/
synchronized boolean consumeShard() {
return checkAndSubmitNextTask();
}
// CHECKSTYLE:OFF CyclomaticComplexity
private synchronized boolean checkAndSubmitNextTask() {
// Task completed successfully (without exceptions)
boolean taskCompletedSuccessfully = false;
boolean submittedNewTask = false;
if ((future == null) || future.isCancelled() || future.isDone()) {
if ((future != null) && future.isDone()) {
try {
TaskResult result = future.get();
if (result.getException() == null) {
taskCompletedSuccessfully = true;
if (result.isShardEndReached()) {
markForShutdown(ShutdownReason.TERMINATE);
}
} else {
if (LOG.isDebugEnabled()) {
Exception taskException = result.getException();
if (taskException instanceof BlockedOnParentShardException) {
// No need to log the stack trace for this exception (it is very specific).
LOG.debug("Shard " + shardInfo.getShardId()
+ " is blocked on completion of parent shard.");
} else {
LOG.debug("Caught exception running " + currentTask.getTaskType() + " task: ",
result.getException());
}
}
}
} catch (InterruptedException e) {
if (LOG.isDebugEnabled()) {
LOG.debug(currentTask.getTaskType() + " task was interrupted: ", e);
}
} catch (ExecutionException e) {
if (LOG.isDebugEnabled()) {
LOG.debug(currentTask.getTaskType() + " task encountered execution exception: ", e);
}
}
}
updateState(taskCompletedSuccessfully);
ITask nextTask = getNextTask();
if (nextTask != null) {
currentTask = nextTask;
future = executorService.submit(currentTask);
currentTaskSubmitTime = System.currentTimeMillis();
submittedNewTask = true;
LOG.debug("Submitted new " + currentTask.getTaskType() + " task for shard " + shardInfo.getShardId());
} else {
if (LOG.isDebugEnabled()) {
LOG.debug(String.format("No new task to submit for shard %s, currentState %s",
shardInfo.getShardId(),
currentState.toString()));
}
}
} else {
if (LOG.isDebugEnabled()) {
LOG.debug("Previous " + currentTask.getTaskType() + " task still pending for shard "
+ shardInfo.getShardId() + " since " + (System.currentTimeMillis() - currentTaskSubmitTime)
+ " ms ago" + ". Not submitting new task.");
}
}
return submittedNewTask;
}
// CHECKSTYLE:ON CyclomaticComplexity
/**
* Shutdown this ShardConsumer (including invoking the RecordProcessor shutdown API).
* This is called by Worker when it loses responsibility for a shard.
*
* @return true if shutdown is complete (false if shutdown is still in progress)
*/
synchronized boolean beginShutdown() {
if (currentState != ShardConsumerState.SHUTDOWN_COMPLETE) {
markForShutdown(ShutdownReason.ZOMBIE);
checkAndSubmitNextTask();
}
return isShutdown();
}
synchronized void markForShutdown(ShutdownReason reason) {
beginShutdown = true;
// ShutdownReason.ZOMBIE takes precedence over TERMINATE (we won't be able to save checkpoint at end of shard)
if ((shutdownReason == null) || (shutdownReason == ShutdownReason.TERMINATE)) {
shutdownReason = reason;
}
}
/**
* Used (by Worker) to check if this ShardConsumer instance has been shutdown
* RecordProcessor shutdown() has been invoked, as appropriate.
*
* @return true if shutdown is complete
*/
boolean isShutdown() {
return currentState == ShardConsumerState.SHUTDOWN_COMPLETE;
}
/**
* @return the shutdownReason
*/
ShutdownReason getShutdownReason() {
return shutdownReason;
}
/**
* Figure out next task to run based on current state, task, and shutdown context.
*
* @return Return next task to run
*/
private ITask getNextTask() {
ITask nextTask = null;
switch (currentState) {
case WAITING_ON_PARENT_SHARDS:
nextTask = new BlockOnParentShardTask(shardInfo, leaseManager, parentShardPollIntervalMillis);
break;
case INITIALIZING:
nextTask =
new InitializeTask(shardInfo,
recordProcessor,
checkpoint,
recordProcessorCheckpointer,
dataFetcher,
taskBackoffTimeMillis);
break;
case PROCESSING:
nextTask =
new ProcessTask(shardInfo,
streamConfig,
recordProcessor,
recordProcessorCheckpointer,
dataFetcher,
taskBackoffTimeMillis);
break;
case SHUTTING_DOWN:
nextTask =
new ShutdownTask(shardInfo,
recordProcessor,
recordProcessorCheckpointer,
shutdownReason,
streamConfig.getStreamProxy(),
streamConfig.getInitialPositionInStream(),
cleanupLeasesOfCompletedShards,
leaseManager,
taskBackoffTimeMillis);
break;
case SHUTDOWN_COMPLETE:
break;
default:
break;
}
if (nextTask == null) {
return null;
} else {
return new MetricsCollectingTaskDecorator(nextTask, metricsFactory);
}
}
/**
* Note: This is a private/internal method with package level access solely for testing purposes.
* Update state based on information about: task success, current state, and shutdown info.
*
* @param taskCompletedSuccessfully Whether (current) task completed successfully.
*/
// CHECKSTYLE:OFF CyclomaticComplexity
void updateState(boolean taskCompletedSuccessfully) {
switch (currentState) {
case WAITING_ON_PARENT_SHARDS:
if (taskCompletedSuccessfully && TaskType.BLOCK_ON_PARENT_SHARDS.equals(currentTask.getTaskType())) {
if (beginShutdown) {
currentState = ShardConsumerState.SHUTTING_DOWN;
} else {
currentState = ShardConsumerState.INITIALIZING;
}
} else if ((currentTask == null) && beginShutdown) {
currentState = ShardConsumerState.SHUTDOWN_COMPLETE;
}
break;
case INITIALIZING:
if (taskCompletedSuccessfully && TaskType.INITIALIZE.equals(currentTask.getTaskType())) {
if (beginShutdown) {
currentState = ShardConsumerState.SHUTTING_DOWN;
} else {
currentState = ShardConsumerState.PROCESSING;
}
} else if ((currentTask == null) && beginShutdown) {
currentState = ShardConsumerState.SHUTDOWN_COMPLETE;
}
break;
case PROCESSING:
if (taskCompletedSuccessfully && TaskType.PROCESS.equals(currentTask.getTaskType())) {
if (beginShutdown) {
currentState = ShardConsumerState.SHUTTING_DOWN;
} else {
currentState = ShardConsumerState.PROCESSING;
}
}
break;
case SHUTTING_DOWN:
if (currentTask == null
|| (taskCompletedSuccessfully && TaskType.SHUTDOWN.equals(currentTask.getTaskType()))) {
currentState = ShardConsumerState.SHUTDOWN_COMPLETE;
}
break;
case SHUTDOWN_COMPLETE:
break;
default:
LOG.error("Unexpected state: " + currentState);
break;
}
}
// CHECKSTYLE:ON CyclomaticComplexity
/**
* Private/Internal method - has package level access solely for testing purposes.
*
* @return the currentState
*/
ShardConsumerState getCurrentState() {
return currentState;
}
/**
* Private/Internal method - has package level access solely for testing purposes.
*
* @return the beginShutdown
*/
boolean isBeginShutdown() {
return beginShutdown;
}
}