Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -300,9 +300,18 @@ protected void ackUntil(Record<GenericObject> lastNotFlushed,
partitionOffset.put(tp.partition(), e.getValue().offset());
}

int ackRequestedCount = 0;
for (Record<GenericObject> r : pendingFlushQueue) {
final String topic = sanitizeNameIfNeeded(r.getTopicName().orElse(topicName), sanitizeTopicName);
final int partition = r.getPartitionIndex().orElse(0);
final String topic;
final int partition;
if (shouldCollapsePartitionedTopic(r)) {
TopicName tn = TopicName.get(r.getTopicName().get());
partition = tn.getPartitionIndex();
topic = sanitizeNameIfNeeded(tn.getPartitionedTopicName(), sanitizeTopicName);
} else {
partition = r.getPartitionIndex().orElse(0);
topic = sanitizeNameIfNeeded(r.getTopicName().orElse(topicName), sanitizeTopicName);
}

Long lastCommittedOffset = null;
if (topicOffsets.containsKey(topic)) {
Expand All @@ -326,15 +335,26 @@ protected void ackUntil(Record<GenericObject> lastNotFlushed,
}

cb.accept(r);
ackRequestedCount++;
pendingFlushQueue.remove(r);
currentBatchSize.addAndGet(-1 * r.getMessage().get().size());
if (r == lastNotFlushed) {
break;
}
}
if (log.isDebugEnabled()) {
log.debug("ackRequestedCount: {}, committedOffsets: {}", ackRequestedCount, committedOffsets);
}
}

private long getMessageOffset(Record<GenericObject> sourceRecord) {
private boolean shouldCollapsePartitionedTopic(Record<GenericObject> r) {
return collapsePartitionedTopics
&& r.getTopicName().isPresent()
&& TopicName.get(r.getTopicName().get()).isPartitioned();
}

@VisibleForTesting
long getMessageOffset(Record<GenericObject> sourceRecord) {

if (sourceRecord.getMessage().isPresent()) {
// Use index added by org.apache.pulsar.common.intercept.AppendIndexMetadataInterceptor if present.
Expand Down Expand Up @@ -440,9 +460,7 @@ protected SinkRecord toSinkRecord(Record<GenericObject> sourceRecord) {
final int partition;
final String topic;

if (collapsePartitionedTopics
&& sourceRecord.getTopicName().isPresent()
&& TopicName.get(sourceRecord.getTopicName().get()).isPartitioned()) {
if (shouldCollapsePartitionedTopic(sourceRecord)) {
TopicName tn = TopicName.get(sourceRecord.getTopicName().get());
partition = tn.getPartitionIndex();
topic = sanitizeNameIfNeeded(tn.getPartitionedTopicName(), sanitizeTopicName);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1682,6 +1682,67 @@ private void testCollapsePartitionedTopic(boolean isEnabled,
sink.close();
}

@Test
public void testAckUntilWithCollapsePartitionedTopics() throws Exception {
testAckUntil(true,
"persistent://a/b/fake-topic-partition-0",
"persistent://a/b/fake-topic",
0);
}

@Test
public void testAckUntilWithoutCollapsePartitionedTopics() throws Exception {
// Note: Without collapsePartitionedTopics expectedPartition in the committedOffsets will always be 0
testAckUntil(false,
"persistent://a/b/fake-topic-partition-1",
"persistent://a/b/fake-topic-partition-1",
0);
}

private void testAckUntil(boolean collapseEnabled,
String pulsarTopic,
String expectedKafkaTopic,
int expectedPartition) throws Exception {
// Setup sink with given collapseEnabled value
props.put("kafkaConnectorSinkClass", SchemaedFileStreamSinkConnector.class.getCanonicalName());
props.put("collapsePartitionedTopics", Boolean.toString(collapseEnabled));
KafkaConnectSink sink = new KafkaConnectSink();
sink.open(props, context);

// Create pulsar record with given pulsarTopic and expectedPartition
Message msg = mock(MessageImpl.class);
when(msg.getMessageId()).thenReturn(new MessageIdImpl(1, 1, expectedPartition));
when(msg.getValue()).thenReturn(null);

AtomicInteger ackCount = new AtomicInteger(0);

Record<GenericObject> record = PulsarRecord.<GenericObject>builder()
.topicName(pulsarTopic)
.message(msg)
.ackFunction(ackCount::incrementAndGet)
.failFunction(() -> {})
.build();

// Add the pulsar record to pendingFlushQueue
sink.pendingFlushQueue.add(record);

// Build committedOffsets with the given expectedKafkaTopic and expectedPartition
Map<TopicPartition, OffsetAndMetadata> committedOffsets = new HashMap<>();
committedOffsets.put(
new TopicPartition(expectedKafkaTopic, expectedPartition),
new OffsetAndMetadata(sink.getMessageOffset(record))
);

// Trigger actUntil manually
sink.ackUntil(record, committedOffsets, Record::ack);

// Assert that the ackFunction runnable of the record is called and pendingFlushQueue is empty
Assert.assertEquals(ackCount.get(), 1);
Assert.assertTrue(sink.pendingFlushQueue.isEmpty());

sink.close();
}

@SneakyThrows
private java.util.Date getDateFromString(String dateInString) {
SimpleDateFormat formatter = new SimpleDateFormat("dd/MM/yyyy hh:mm:ss");
Expand Down
Loading