|
53 | 53 | import java.util.function.Function;
|
54 | 54 | import lombok.Cleanup;
|
55 | 55 | import lombok.extern.slf4j.Slf4j;
|
| 56 | +import org.apache.commons.lang3.RandomStringUtils; |
56 | 57 | import org.apache.kafka.clients.admin.AbortTransactionSpec;
|
57 | 58 | import org.apache.kafka.clients.admin.AdminClient;
|
58 | 59 | import org.apache.kafka.clients.admin.DescribeProducersResult;
|
@@ -1206,6 +1207,7 @@ private KafkaConsumer<Integer, String> buildTransactionConsumer(String groupId,
|
1206 | 1207 | consumerProps.put(ConsumerConfig.GROUP_ID_CONFIG, groupId);
|
1207 | 1208 | consumerProps.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
|
1208 | 1209 | consumerProps.put(ConsumerConfig.ISOLATION_LEVEL_CONFIG, isolation);
|
| 1210 | + consumerProps.put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, 10); |
1209 | 1211 | addCustomizeProps(consumerProps);
|
1210 | 1212 |
|
1211 | 1213 | return new KafkaConsumer<>(consumerProps);
|
@@ -1746,4 +1748,64 @@ protected void addCustomizeProps(Properties producerProps) {
|
1746 | 1748 | // No-op
|
1747 | 1749 | }
|
1748 | 1750 |
|
| 1751 | + @DataProvider(name = "isolationProvider") |
| 1752 | + protected Object[][] isolationProvider() { |
| 1753 | + return new Object[][]{ |
| 1754 | + {"read_committed"}, |
| 1755 | + {"read_uncommitted"}, |
| 1756 | + }; |
| 1757 | + } |
| 1758 | + |
| 1759 | + @Test(dataProvider = "isolationProvider", timeOut = 1000 * 30) |
| 1760 | + public void readUnstableMessagesTest(String isolation) throws InterruptedException, ExecutionException { |
| 1761 | + String topic = "unstable-message-test-" + RandomStringUtils.randomAlphabetic(5); |
| 1762 | + |
| 1763 | + KafkaConsumer<Integer, String> consumer = buildTransactionConsumer("unstable-read", isolation); |
| 1764 | + consumer.subscribe(Collections.singleton(topic)); |
| 1765 | + |
| 1766 | + String tnxId = "txn-" + RandomStringUtils.randomAlphabetic(5); |
| 1767 | + KafkaProducer<Integer, String> producer = buildTransactionProducer(tnxId); |
| 1768 | + producer.initTransactions(); |
| 1769 | + |
| 1770 | + String baseMsg = "test msg commit - "; |
| 1771 | + producer.beginTransaction(); |
| 1772 | + producer.send(new ProducerRecord<>(topic, baseMsg + 0)).get(); |
| 1773 | + producer.send(new ProducerRecord<>(topic, baseMsg + 1)).get(); |
| 1774 | + producer.flush(); |
| 1775 | + |
| 1776 | + AtomicInteger messageCount = new AtomicInteger(0); |
| 1777 | + // make sure consumer can't receive unstable messages in `read_committed` mode |
| 1778 | + readAndCheckMessages(consumer, baseMsg, messageCount, isolation.equals("read_committed") ? 0 : 2); |
| 1779 | + |
| 1780 | + producer.commitTransaction(); |
| 1781 | + producer.beginTransaction(); |
| 1782 | + // these two unstable message shouldn't be received in `read_committed` mode |
| 1783 | + producer.send(new ProducerRecord<>(topic, baseMsg + 2)).get(); |
| 1784 | + producer.send(new ProducerRecord<>(topic, baseMsg + 3)).get(); |
| 1785 | + producer.flush(); |
| 1786 | + |
| 1787 | + readAndCheckMessages(consumer, baseMsg, messageCount, isolation.equals("read_committed") ? 2 : 4); |
| 1788 | + |
| 1789 | + consumer.close(); |
| 1790 | + producer.close(); |
| 1791 | + } |
| 1792 | + |
| 1793 | + private void readAndCheckMessages(KafkaConsumer<Integer, String> consumer, String baseMsg, |
| 1794 | + AtomicInteger messageCount, int expectedMessageCount) { |
| 1795 | + while (true) { |
| 1796 | + ConsumerRecords<Integer, String> records = consumer.poll(Duration.ofSeconds(3)); |
| 1797 | + if (records.isEmpty()) { |
| 1798 | + break; |
| 1799 | + } |
| 1800 | + for (ConsumerRecord<Integer, String> record : records) { |
| 1801 | + assertEquals(record.value(), baseMsg + messageCount.getAndIncrement()); |
| 1802 | + } |
| 1803 | + } |
| 1804 | + // make sure there is no message can be received |
| 1805 | + ConsumerRecords<Integer, String> records = consumer.poll(Duration.ofSeconds(3)); |
| 1806 | + assertTrue(records.isEmpty()); |
| 1807 | + // make sure only receive the expected number of stable messages |
| 1808 | + assertEquals(messageCount.get(), expectedMessageCount); |
| 1809 | + } |
| 1810 | + |
1749 | 1811 | }
|
0 commit comments