|
17 | 17 | package org.apache.kafka.clients;
|
18 | 18 |
|
19 | 19 | import org.apache.kafka.clients.admin.NewTopic;
|
| 20 | +import org.apache.kafka.clients.producer.ProducerRecord; |
| 21 | +import org.apache.kafka.common.config.TopicConfig; |
20 | 22 | import org.apache.kafka.common.test.ClusterInstance;
|
21 | 23 | import org.apache.kafka.common.test.api.ClusterConfigProperty;
|
22 | 24 | import org.apache.kafka.common.test.api.ClusterTest;
|
|
26 | 28 | import java.time.Duration;
|
27 | 29 | import java.util.List;
|
28 | 30 | import java.util.Map;
|
| 31 | +import java.util.concurrent.ExecutionException; |
29 | 32 | import java.util.concurrent.TimeUnit;
|
30 | 33 | import java.util.concurrent.TimeoutException;
|
31 | 34 |
|
32 | 35 | import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
|
| 36 | +import static org.junit.jupiter.api.Assertions.assertEquals; |
33 | 37 | import static org.junit.jupiter.api.Assertions.assertThrows;
|
34 | 38 |
|
35 | 39 | public class ClientRebootstrapTest {
|
@@ -94,4 +98,74 @@ public void testAdminRebootstrapDisabled(ClusterInstance clusterInstance) {
|
94 | 98 | // Since the brokers cached during the bootstrap are offline, the admin client needs to wait the default timeout for other threads.
|
95 | 99 | admin.close(Duration.ZERO);
|
96 | 100 | }
|
| 101 | + |
| 102 | + @ClusterTest( |
| 103 | + brokers = REPLICAS, |
| 104 | + types = {Type.KRAFT}, |
| 105 | + serverProperties = { |
| 106 | + @ClusterConfigProperty(key = TopicConfig.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG, value = "true"), |
| 107 | + @ClusterConfigProperty(key = GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, value = "2") |
| 108 | + } |
| 109 | + ) |
| 110 | + public void testProducerRebootstrap(ClusterInstance clusterInstance) throws ExecutionException, InterruptedException { |
| 111 | + try (var admin = clusterInstance.admin()) { |
| 112 | + admin.createTopics(List.of(new NewTopic(TOPIC, 1, (short) REPLICAS))); |
| 113 | + } |
| 114 | + |
| 115 | + var broker0 = 0; |
| 116 | + var broker1 = 1; |
| 117 | + |
| 118 | + // It's ok to shut the leader down, cause the reelection is small enough to the producer timeout. |
| 119 | + clusterInstance.shutdownBroker(broker0); |
| 120 | + |
| 121 | + try (var producer = clusterInstance.producer()) { |
| 122 | + // Only the broker 1 is available for the producer during the bootstrap. |
| 123 | + var recordMetadata0 = producer.send(new ProducerRecord<>(TOPIC, "value 0".getBytes())).get(); |
| 124 | + assertEquals(0, recordMetadata0.offset()); |
| 125 | + |
| 126 | + clusterInstance.shutdownBroker(broker1); |
| 127 | + clusterInstance.startBroker(broker0); |
| 128 | + |
| 129 | + // Current broker 1 is offline. |
| 130 | + // However, the broker 0 from the bootstrap list is online. |
| 131 | + // Should be able to produce records. |
| 132 | + var recordMetadata1 = producer.send(new ProducerRecord<>(TOPIC, "value 1".getBytes())).get(); |
| 133 | + assertEquals(0, recordMetadata1.offset()); |
| 134 | + } |
| 135 | + } |
| 136 | + |
| 137 | + @ClusterTest( |
| 138 | + brokers = REPLICAS, |
| 139 | + types = {Type.KRAFT}, |
| 140 | + serverProperties = { |
| 141 | + @ClusterConfigProperty(key = TopicConfig.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG, value = "true"), |
| 142 | + @ClusterConfigProperty(key = GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, value = "2") |
| 143 | + } |
| 144 | + ) |
| 145 | + public void testProducerRebootstrapDisabled(ClusterInstance clusterInstance) throws ExecutionException, InterruptedException { |
| 146 | + try (var admin = clusterInstance.admin()) { |
| 147 | + admin.createTopics(List.of(new NewTopic(TOPIC, 1, (short) REPLICAS))); |
| 148 | + } |
| 149 | + |
| 150 | + var broker0 = 0; |
| 151 | + var broker1 = 1; |
| 152 | + |
| 153 | + // It's ok to shut the leader down, cause the reelection is small enough to the producer timeout. |
| 154 | + clusterInstance.shutdownBroker(broker0); |
| 155 | + |
| 156 | + var producer = clusterInstance.producer(Map.of(CommonClientConfigs.METADATA_RECOVERY_STRATEGY_CONFIG, "none")); |
| 157 | + |
| 158 | + // Only the broker 1 is available for the producer during the bootstrap. |
| 159 | + var recordMetadata0 = producer.send(new ProducerRecord<>(TOPIC, "value 0".getBytes())).get(); |
| 160 | + assertEquals(0, recordMetadata0.offset()); |
| 161 | + |
| 162 | + clusterInstance.shutdownBroker(broker1); |
| 163 | + clusterInstance.startBroker(broker0); |
| 164 | + |
| 165 | + // The broker 1, originally cached during the bootstrap, is offline. |
| 166 | + // As a result, the producer will throw a TimeoutException when trying to send a message. |
| 167 | + assertThrows(TimeoutException.class, () -> producer.send(new ProducerRecord<>(TOPIC, "value 1".getBytes())).get(5, TimeUnit.SECONDS)); |
| 168 | + // Since the brokers cached during the bootstrap are offline, the producer needs to wait the default timeout for other threads. |
| 169 | + producer.close(Duration.ZERO); |
| 170 | + } |
97 | 171 | }
|
0 commit comments