Skip to content

Commit 4a0d367

Browse files
limitiumlimitium
authored andcommitted
Moved from SourceTask#commit() to SourceTask#commitRecord() to eliminate over acknowledgments and prevent message loss
1 parent 46b02db commit 4a0d367

File tree

1 file changed

+105
-27
lines changed

1 file changed

+105
-27
lines changed

src/main/java/com/solace/connector/kafka/connect/source/SolaceSourceTask.java

Lines changed: 105 additions & 27 deletions
Original file line numberDiff line numberDiff line change
@@ -19,22 +19,16 @@
1919

2020
package com.solace.connector.kafka.connect.source;
2121

22-
import com.solacesystems.jcsmp.BytesXMLMessage;
23-
import com.solacesystems.jcsmp.DeliveryMode;
24-
import com.solacesystems.jcsmp.JCSMPException;
25-
import com.solacesystems.jcsmp.JCSMPSession;
22+
import com.solacesystems.jcsmp.*;
23+
import org.apache.kafka.clients.producer.RecordMetadata;
2624
import org.apache.kafka.connect.errors.ConnectException;
2725
import org.apache.kafka.connect.source.SourceRecord;
2826
import org.apache.kafka.connect.source.SourceTask;
2927
import org.slf4j.Logger;
3028
import org.slf4j.LoggerFactory;
3129

32-
import java.util.ArrayList;
33-
import java.util.Collections;
34-
import java.util.List;
35-
import java.util.Map;
36-
import java.util.concurrent.BlockingQueue;
37-
import java.util.concurrent.LinkedBlockingQueue;
30+
import java.util.*;
31+
import java.util.concurrent.*;
3832

3933

4034
public class SolaceSourceTask extends SourceTask { // implements XMLMessageListener{
@@ -45,8 +39,16 @@ public class SolaceSourceTask extends SourceTask { // implements XMLMessageListe
4539
private SolSessionHandler solSessionHandler = null;
4640
BlockingQueue<BytesXMLMessage> ingressMessages
4741
= new LinkedBlockingQueue<>(); // LinkedBlockingQueue for any incoming message from PS+ topics and queue
48-
BlockingQueue<BytesXMLMessage> outstandingAckList
49-
= new LinkedBlockingQueue<>(); // LinkedBlockingQueue for Solace Flow messages
42+
Map<BytesXMLMessage, Integer> pendingAcks = new HashMap<>(); // Pending acks for solace message with number of created record
43+
Map<SourceRecord, BytesXMLMessage> recordToMessage = new HashMap<>(); // Map record to solace message
44+
45+
//Scheduled buffer for acks,
46+
private final List<BytesXMLMessage> ackBuffer = new ArrayList<>(); // Buffer for acknowledgments
47+
private int ackBufferSize; // Maximum buffer size before flush
48+
49+
private long lastMessageAddedTime = System.currentTimeMillis(); // Track time of last addition to the buffer
50+
private final ScheduledExecutorService flushScheduler = Executors.newScheduledThreadPool(1);
51+
5052
String skafkaTopic;
5153
SolaceSourceTopicListener topicListener = null;
5254
SolaceSourceQueueConsumer queueConsumer = null;
@@ -56,6 +58,7 @@ public class SolaceSourceTask extends SourceTask { // implements XMLMessageListe
5658

5759
// private Class<?> cProcessor;
5860
private SolMessageProcessorIF processor;
61+
private int ackTimeout;
5962

6063
@Override
6164
public String version() {
@@ -98,6 +101,26 @@ public void start(Map<String, String> props) {
98101
throw new ConnectException("Failed to start queue consumer", e);
99102
}
100103
}
104+
105+
setupScheduler();
106+
}
107+
108+
private void setupScheduler() {
109+
//Align scheduler and buffer with underlying flowHandle.startAckTimer() and UnackedMessageList2.thresholdCount
110+
int subWinSz = (Integer)solSessionHandler.getSession().getProperty(JCSMPProperties.SUB_ACK_WINDOW_SIZE);
111+
int subThreshold = (Integer)solSessionHandler.getSession().getProperty(JCSMPProperties.SUB_ACK_WINDOW_THRESHOLD);
112+
ackBufferSize = subWinSz * subThreshold / 100;
113+
114+
ackTimeout = (Integer)solSessionHandler.getSession().getProperty(JCSMPProperties.SUB_ACK_TIME);
115+
116+
// Start the scheduled task to periodically flush the buffer
117+
flushScheduler.scheduleAtFixedRate(() -> {
118+
try {
119+
flushAckBufferIfNeeded();
120+
} catch (Exception e) {
121+
log.error("Error during scheduled ack buffer flush", e);
122+
}
123+
}, ackTimeout, ackTimeout, TimeUnit.MILLISECONDS);
101124
}
102125

103126
@Override
@@ -132,38 +155,72 @@ public synchronized List<SourceRecord> poll() throws InterruptedException {
132155
} catch (Exception e) {
133156
if (connectorConfig.getBoolean(SolaceSourceConstants.SOL_MESSAGE_PROCESSOR_IGNORE_ERROR)) {
134157
log.warn("================ Encountered exception in message processing....discarded.", e);
135-
scheduleForAck(msg);
158+
msg.ackMessage(); // Effective discard solace message
136159
discarded++;
137160
continue;
138161
} else {
139162
throw new ConnectException("Encountered exception in message processing", e);
140163
}
141164
}
142-
Collections.addAll(records, processor.getRecords(skafkaTopic));
143-
scheduleForAck(msg);
165+
SourceRecord[] processorRecords = processor.getRecords(skafkaTopic);
166+
Collections.addAll(records, processorRecords);
167+
scheduleForAck(msg, processorRecords);
144168
}
145169
log.debug("Processed {} records in this batch. Discarded {}", processedInThisBatch - discarded, discarded);
146170
return records;
147171
}
148172

149-
private synchronized void scheduleForAck(BytesXMLMessage msg) {
173+
private synchronized void scheduleForAck(BytesXMLMessage msg, SourceRecord[] processorRecords) {
150174
if (msg.getDeliveryMode() == DeliveryMode.NON_PERSISTENT
151175
|| msg.getDeliveryMode() == DeliveryMode.PERSISTENT) {
152-
outstandingAckList.add(msg); // enqueue messages received from guaranteed messaging endpoint for later ack
176+
for (SourceRecord processorRecord : processorRecords) {
177+
recordToMessage.put(processorRecord, msg); // Map each record to solace message id
178+
}
179+
pendingAcks.put(msg, processorRecords.length); // enqueue messages received from guaranteed messaging endpoint for later ack
153180
}
154181
}
155182

156-
/**
157-
* Kafka Connect method that write records to disk.
158-
*/
159-
public synchronized void commit() throws InterruptedException {
160-
log.trace("Committing records");
161-
int currentLoad = outstandingAckList.size();
162-
int count = 0;
163-
while (count != currentLoad) {
164-
outstandingAckList.take().ackMessage();
165-
count++;
183+
@Override
184+
public synchronized void commitRecord(SourceRecord record, RecordMetadata metadata) {
185+
BytesXMLMessage msg = recordToMessage.remove(record);
186+
if (msg == null) {
187+
log.error("Unable to find message for record {}", record); // Shouldn't happens
188+
return;
166189
}
190+
191+
if (!pendingAcks.containsKey(msg)) {
192+
log.error("Unable to find message counter for message {}", msg); // Shouldn't happens
193+
}
194+
195+
pendingAcks.computeIfPresent(msg, (k, o) -> o > 1 ? --o : null);// Reduce counter of records per message, remove on last
196+
197+
if (!pendingAcks.containsKey(msg)) {// Last record was commited in the group
198+
ackBuffer.add(msg);
199+
lastMessageAddedTime = System.currentTimeMillis(); // Update last message addition time
200+
201+
// Flush the buffer if it reaches the maximum buffer size
202+
if (ackBuffer.size() >= ackBufferSize) {
203+
flushAckBuffer();
204+
}
205+
log.debug("Buffer ack for message {}", msg);
206+
}
207+
}
208+
209+
private synchronized void flushAckBufferIfNeeded() {
210+
long currentTime = System.currentTimeMillis();
211+
if (!ackBuffer.isEmpty() && (currentTime - lastMessageAddedTime) >= ackTimeout) {
212+
flushAckBuffer();
213+
}
214+
}
215+
216+
private synchronized void flushAckBuffer() {
217+
for (BytesXMLMessage msg : ackBuffer) {
218+
msg.ackMessage();
219+
log.debug("Acknowledged message {}", msg);
220+
}
221+
222+
ackBuffer.clear(); // Clear the buffer after acknowledgment
223+
log.debug("Flushed acknowledgment buffer");
167224
}
168225

169226
@Override
@@ -183,6 +240,27 @@ public synchronized void stop() {
183240
}
184241
solSessionHandler = null; // At this point filling the ingress queue is stopped
185242
ingressMessages.clear(); // Remove all remaining ingressed messages, these will be no longer imported to Kafka
243+
recordToMessage.clear();
244+
245+
if (!pendingAcks.isEmpty()) {
246+
log.warn("Potential duplicates might be spotted");
247+
pendingAcks.forEach((s, m) -> log.warn("Dup: {}", m));
248+
pendingAcks.clear();
249+
}
250+
251+
// Flush remaining messages in the buffer
252+
flushAckBuffer();
253+
if (!flushScheduler.isShutdown()) {
254+
flushScheduler.shutdown();
255+
try {
256+
if (!flushScheduler.awaitTermination(500, TimeUnit.MILLISECONDS)) {
257+
flushScheduler.shutdownNow();
258+
}
259+
} catch (InterruptedException e) {
260+
flushScheduler.shutdownNow();
261+
}
262+
}
263+
186264
log.info("PubSub+ Source Connector stopped");
187265
}
188266

0 commit comments

Comments
 (0)