Skip to content
This repository was archived by the owner on Jan 24, 2024. It is now read-only.
Merged
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 @@ -810,12 +810,6 @@ public CompletableFuture<Map<TopicPartition, Errors>> handleCommitOffsets(
});
});

result.whenCompleteAsync((ignore, e) ->{
if (e == null){
offsetAcker.ackOffsets(groupId, offsetMetadata);
}
});

return result;
}

Expand Down Expand Up @@ -850,6 +844,7 @@ private CompletableFuture<Map<TopicPartition, Errors>> doCommitOffsets(
// The group is only using Kafka to store offsets.
// Also, for transactional offset commits we don't need to validate group membership
// and the generation.
offsetAcker.ackOffsets(group.groupId(), offsetMetadata);
return groupManager.storeOffsets(group, memberId, offsetMetadata, producerId, producerEpoch);
} else if (group.is(CompletingRebalance)) {
return CompletableFuture.completedFuture(
Expand All @@ -866,6 +861,7 @@ private CompletableFuture<Map<TopicPartition, Errors>> doCommitOffsets(
} else {
MemberMetadata member = group.get(memberId);
completeAndScheduleNextHeartbeatExpiration(group, member);
offsetAcker.ackOffsets(group.groupId(), offsetMetadata);
return groupManager.storeOffsets(
group, memberId, offsetMetadata
);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,11 +15,13 @@

import static org.testng.Assert.assertEquals;
import static org.testng.Assert.assertTrue;
import static org.testng.Assert.fail;

import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import lombok.Cleanup;
import lombok.extern.slf4j.Slf4j;
import org.apache.kafka.clients.consumer.KafkaConsumer;
import org.apache.kafka.clients.producer.KafkaProducer;
import org.apache.pulsar.client.admin.PulsarAdminException;
Expand All @@ -28,6 +30,7 @@
/**
* Basic end-to-end test with `entryFormat=kafka`.
*/
@Slf4j
public class BasicEndToEndKafkaTest extends BasicEndToEndTestBase {

public BasicEndToEndKafkaTest() {
Expand Down Expand Up @@ -61,16 +64,22 @@ public void testDeleteClosedTopics() throws Exception {

try {
admin.topics().deletePartitionedTopic(topic);
fail();
} catch (PulsarAdminException e) {
assertTrue(e.getMessage().contains("Topic has active producers/subscriptions"));
log.info("Failed to delete partitioned topic \"{}\": {}", topic, e.getMessage());
assertTrue(e.getMessage().contains("Topic has active producers/subscriptions")
|| e.getMessage().contains("Partitioned topic does not exist"));
}

final KafkaConsumer<String, String> kafkaConsumer1 = newKafkaConsumer(topic, "sub-1");
assertEquals(receiveMessages(kafkaConsumer1, expectedMessages.size()), expectedMessages);
try {
admin.topics().deletePartitionedTopic(topic);
fail();
} catch (PulsarAdminException e) {
assertTrue(e.getMessage().contains("Topic has active producers/subscriptions"));
log.info("Failed to delete partitioned topic \"{}\": {}", topic, e.getMessage());
assertTrue(e.getMessage().contains("Topic has active producers/subscriptions")
|| e.getMessage().contains("Partitioned topic does not exist"));
}

final KafkaConsumer<String, String> kafkaConsumer2 = newKafkaConsumer(topic, "sub-2");
Expand All @@ -80,8 +89,11 @@ public void testDeleteClosedTopics() throws Exception {
kafkaConsumer1.close();
try {
admin.topics().deletePartitionedTopic(topic);
fail();
} catch (PulsarAdminException e) {
assertTrue(e.getMessage().contains("Topic has active producers/subscriptions"));
log.info("Failed to delete partitioned topic \"{}\": {}", topic, e.getMessage());
assertTrue(e.getMessage().contains("Topic has active producers/subscriptions")
|| e.getMessage().contains("Partitioned topic does not exist"));
}

kafkaConsumer2.close();
Expand Down