|
25 | 25 | import org.apache.spark.sql.execution.streaming.MicroBatchExecution; |
26 | 26 | import org.apache.spark.sql.execution.streaming.StreamExecution; |
27 | 27 | import org.apache.spark.unsafe.types.UTF8String; |
| 28 | +import org.apache.spark.util.TaskFailureListener; |
28 | 29 |
|
29 | 30 | import java.io.BufferedWriter; |
30 | 31 | import java.io.File; |
@@ -119,6 +120,13 @@ public boolean next() { |
119 | 120 | this.solaceBroker.close(); |
120 | 121 | throw new SolaceSessionException(this.solaceBroker.getException()); |
121 | 122 | } |
| 123 | + |
| 124 | + if (TaskContext.get() != null && TaskContext.get().isInterrupted()) { |
| 125 | + log.info("SolaceSparkConnector - Interrupted while waiting for next message"); |
| 126 | + SolaceConnectionManager.close(this.solaceInputPartition.getId()); |
| 127 | + SolaceMessageTracker.resetId(uniqueId); |
| 128 | + throw new RuntimeException("Task was interrupted."); |
| 129 | + } |
122 | 130 | solaceMessage = getNextMessage(); |
123 | 131 | return solaceMessage != null; |
124 | 132 | } |
@@ -225,16 +233,24 @@ public void close() { |
225 | 233 | } |
226 | 234 | } |
227 | 235 |
|
| 236 | + private void logShutdownMessage(TaskContext context) { |
| 237 | + log.info("SolaceSparkConnector - Closing connections to Solace as task {} is interrupted or failed", String.join(",", context.getLocalProperty(StreamExecution.QUERY_ID_KEY()), |
| 238 | + context.getLocalProperty(MicroBatchExecution.BATCH_ID_KEY()), |
| 239 | + Integer.toString(context.stageId()), |
| 240 | + Integer.toString(context.partitionId()))); |
| 241 | + SolaceConnectionManager.close(this.solaceInputPartition.getId()); |
| 242 | + SolaceMessageTracker.resetId(uniqueId); |
| 243 | + } |
| 244 | + |
228 | 245 | private void registerTaskListener() { |
| 246 | + this.taskContext.addTaskFailureListener((context, error) -> { |
| 247 | + log.error("SolaceSparkConnector - Input Partition {} failed with error", this.solaceInputPartition.getId(), error); |
| 248 | + logShutdownMessage(context); |
| 249 | + }); |
229 | 250 | this.taskContext.addTaskCompletionListener(context -> { |
230 | 251 | log.info("SolaceSparkConnector - Task {} state is completed :: {}, failed :: {}, interrupted :: {}", uniqueId, context.isCompleted(), context.isFailed(), context.isInterrupted()); |
231 | 252 | if(context.isInterrupted() || context.isFailed()) { |
232 | | - log.info("SolaceSparkConnector - Closing connections to Solace as task {} is interrupted or failed", String.join(",", context.getLocalProperty(StreamExecution.QUERY_ID_KEY()), |
233 | | - context.getLocalProperty(MicroBatchExecution.BATCH_ID_KEY()), |
234 | | - Integer.toString(context.stageId()), |
235 | | - Integer.toString(context.partitionId()))); |
236 | | - SolaceConnectionManager.close(this.solaceInputPartition.getId()); |
237 | | - SolaceMessageTracker.resetId(uniqueId); |
| 253 | + logShutdownMessage(context); |
238 | 254 | } else if (context.isCompleted()) { |
239 | 255 | List<String> ids = SolaceMessageTracker.getIds(); |
240 | 256 | try { |
|
0 commit comments