Compare commits

...

14 Commits

Author SHA1 Message Date
Chang-Yu Huang bffcbb828c
Merge f9703f7e8b into d76442e5a6 2025-10-05 10:58:24 +05:30
Chang-Yu Huang f9703f7e8b
Merge branch 'apache:trunk' into KAFKA-17103 2025-08-20 15:08:56 -04:00
ChangYu Huang cf1f13b0c2 Remove additional thread in testTopicExpiryInMetadata 2025-08-20 12:30:27 -04:00
ChangYu Huang ff404d01ec Remove additional thread in testTopicNotExistingInMetadata 2025-08-20 12:24:59 -04:00
ChangYu Huang 69975c02ae Remove additional thread in testTopicRefreshInMetadata 2025-08-20 12:18:01 -04:00
ChangYu Huang 4fa33f2d0f Add a prepared metadata count 2025-08-20 12:09:34 -04:00
ChangYu Huang f15d2ef460 Fix value mistake 2025-08-20 11:39:37 -04:00
ChangYu Huang fe3df3c168 Remove debug message 2025-08-19 22:09:40 -04:00
Chang-Yu Huang 5817251104
Merge branch 'apache:trunk' into KAFKA-17103 2025-08-19 21:33:23 -04:00
ChangYu Huang d1ab4b41b9 Advance mock time to prevent sender waiting for transactional response infinitely 2025-08-19 16:39:55 -04:00
ChangYu Huang f91dc5a239 Remove busy waiting in testTopicExpiryInMetadata 2025-08-19 16:24:09 -04:00
ChangYu Huang 2b68e44a35 Remove busy waiting in testTopicNotExistingInMetadata 2025-08-19 12:44:27 -04:00
ChangYu Huang 9922d9dacd Remove busy waiting testTopicRefreshInMetadata 2025-08-19 12:44:27 -04:00
ChangYu Huang 39fb8a0f95 Add more controls in MockClient 2025-08-19 12:44:08 -04:00
2 changed files with 74 additions and 72 deletions

View File

@ -39,6 +39,8 @@ import java.util.Queue;
import java.util.Set;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ConcurrentLinkedDeque;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.function.Consumer;
import java.util.stream.Collectors;
@ -72,6 +74,8 @@ public class MockClient implements KafkaClient {
private int correlation;
private Runnable wakeupHook;
private boolean advanceTimeDuringPoll;
private boolean shouldUpdateWithCurrentMetadata = true;
private CountDownLatch isMetadataUpdateNeeded = new CountDownLatch(1);
private final Time time;
private final MockMetadataUpdater metadataUpdater;
private final Map<String, ConnectionState> connections = new HashMap<>();
@ -192,6 +196,10 @@ public class MockClient implements KafkaClient {
this.disconnectFuture = disconnectFuture;
}
public void setShouldUpdateWithCurrentMetadata(boolean shouldUpdateWithCurrentMetadata) {
this.shouldUpdateWithCurrentMetadata = shouldUpdateWithCurrentMetadata;
}
@Override
public void disconnect(String node) {
disconnect(node, false);
@ -329,8 +337,10 @@ public class MockClient implements KafkaClient {
MetadataUpdate metadataUpdate = metadataUpdates.poll();
if (metadataUpdate != null) {
metadataUpdater.update(time, metadataUpdate);
} else {
} else if (shouldUpdateWithCurrentMetadata) {
metadataUpdater.updateWithCurrentMetadata(time);
} else {
isMetadataUpdateNeeded.countDown();
}
}
@ -350,6 +360,14 @@ public class MockClient implements KafkaClient {
return copy;
}
public boolean awaitMetadataUpdateRequest(long timeoutMs) throws InterruptedException {
if (isMetadataUpdateNeeded.await(timeoutMs, TimeUnit.MILLISECONDS)) {
isMetadataUpdateNeeded = new CountDownLatch(1);
return true;
}
return false;
}
private long elapsedTimeMs(long currentTimeMs, long startTimeMs) {
return Math.max(0, currentTimeMs - startTimeMs);
}
@ -523,6 +541,10 @@ public class MockClient implements KafkaClient {
metadataUpdates.add(new MetadataUpdate(updateResponse, expectMatchMetadataTopics));
}
public int preparedMetadataUpdatesCount() {
return metadataUpdates.size();
}
public void updateMetadata(MetadataResponse updateResponse) {
metadataUpdater.update(time, new MetadataUpdate(updateResponse, false));
}

View File

@ -128,13 +128,11 @@ import java.util.Map;
import java.util.Optional;
import java.util.Properties;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.Exchanger;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
import java.util.stream.Collectors;
@ -961,10 +959,9 @@ public class KafkaProducerTest {
@Test
public void testTopicRefreshInMetadata() throws InterruptedException {
Map<String, Object> configs = new HashMap<>();
final String maxBlockMs = "600000";
configs.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9999");
configs.put(ProducerConfig.MAX_BLOCK_MS_CONFIG, "600000");
// test under normal producer for simplicity
configs.put(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, false);
configs.put(ProducerConfig.MAX_BLOCK_MS_CONFIG, maxBlockMs);
long refreshBackoffMs = 500L;
long refreshBackoffMaxMs = 5000L;
long metadataExpireMs = 60000L;
@ -972,35 +969,35 @@ public class KafkaProducerTest {
final Time time = new MockTime();
final ProducerMetadata metadata = new ProducerMetadata(refreshBackoffMs, refreshBackoffMaxMs, metadataExpireMs, metadataIdleMs,
new LogContext(), new ClusterResourceListeners(), time);
final String warmupTopic = "warmup-topic";
final String topic = "topic";
MockClient client = new MockClient(time, metadata);
client.setShouldUpdateWithCurrentMetadata(false);
client.advanceTimeDuringPoll(true);
try (KafkaProducer<String, String> producer = kafkaProducer(configs,
new StringSerializer(), new StringSerializer(), metadata, new MockClient(time, metadata), null, time)) {
new StringSerializer(), new StringSerializer(), metadata, client, null, time)) {
AtomicBoolean running = new AtomicBoolean(true);
Thread t = new Thread(() -> {
long startTimeMs = System.currentTimeMillis();
while (running.get()) {
while (!metadata.updateRequested() && System.currentTimeMillis() - startTimeMs < 100)
Thread.yield();
MetadataResponse updateResponse = RequestTestUtils.metadataUpdateWith("kafka-cluster", 1,
singletonMap(topic, Errors.UNKNOWN_TOPIC_OR_PARTITION), emptyMap());
metadata.updateWithCurrentRequestVersion(updateResponse, false, time.milliseconds());
time.sleep(60 * 1000L);
}
});
t.start();
MetadataResponse updateResponse = RequestTestUtils.metadataUpdateWith(1, singletonMap(warmupTopic, 1));
client.updateMetadata(updateResponse);
final int preparedUpdatesCount = 100;
updateResponse = RequestTestUtils.metadataUpdateWith("kafka-cluster", 1,
singletonMap(topic, Errors.UNKNOWN_TOPIC_OR_PARTITION), emptyMap());
for (int i = 0; i < preparedUpdatesCount; i++) {
client.prepareMetadataUpdate(updateResponse);
}
Throwable throwable = assertThrows(TimeoutException.class, () -> producer.partitionsFor(topic));
assertInstanceOf(UnknownTopicOrPartitionException.class, throwable.getCause());
running.set(false);
t.join();
assertTrue(preparedUpdatesCount > client.preparedMetadataUpdatesCount());
}
}
@Test
public void testTopicNotExistingInMetadata() throws InterruptedException {
public void testTopicNotExistingInMetadata() {
Map<String, Object> configs = new HashMap<>();
final String maxBlockMs = "30000";
configs.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9999");
configs.put(ProducerConfig.MAX_BLOCK_MS_CONFIG, "30000");
configs.put(ProducerConfig.MAX_BLOCK_MS_CONFIG, maxBlockMs);
long refreshBackoffMs = 500L;
long refreshBackoffMaxMs = 5000L;
long metadataExpireMs = 60000L;
@ -1008,39 +1005,33 @@ public class KafkaProducerTest {
final Time time = new MockTime();
final ProducerMetadata metadata = new ProducerMetadata(refreshBackoffMs, refreshBackoffMaxMs, metadataExpireMs, metadataIdleMs,
new LogContext(), new ClusterResourceListeners(), time);
final String warmupTopic = "warmup-topic";
final String topic = "topic";
MockClient client = new MockClient(time, metadata);
client.setShouldUpdateWithCurrentMetadata(false);
client.advanceTimeDuringPoll(true);
try (KafkaProducer<String, String> producer = kafkaProducer(configs, new StringSerializer(),
new StringSerializer(), metadata, new MockClient(time, metadata), null, time)) {
new StringSerializer(), metadata, client, null, time)) {
Exchanger<Void> exchanger = new Exchanger<>();
MetadataResponse updateResponse = RequestTestUtils.metadataUpdateWith(1, singletonMap(warmupTopic, 1));
client.updateMetadata(updateResponse);
updateResponse = RequestTestUtils.metadataUpdateWith("kafka-cluster", 1,
singletonMap(topic, Errors.UNKNOWN_TOPIC_OR_PARTITION), emptyMap());
client.prepareMetadataUpdate(updateResponse);
Thread t = new Thread(() -> {
try {
// Update the metadata with non-existing topic.
MetadataResponse updateResponse = RequestTestUtils.metadataUpdateWith("kafka-cluster", 1,
singletonMap(topic, Errors.UNKNOWN_TOPIC_OR_PARTITION), emptyMap());
metadata.updateWithCurrentRequestVersion(updateResponse, false, time.milliseconds());
exchanger.exchange(null);
while (!metadata.updateRequested())
Thread.sleep(100);
time.sleep(30 * 1000L);
} catch (Exception e) {
throw new RuntimeException(e);
}
});
t.start();
exchanger.exchange(null);
Throwable throwable = assertThrows(TimeoutException.class, () -> producer.partitionsFor(topic));
assertInstanceOf(UnknownTopicOrPartitionException.class, throwable.getCause());
t.join();
assertEquals(0, client.preparedMetadataUpdatesCount());
}
}
@Test
public void testTopicExpiryInMetadata() throws InterruptedException {
Map<String, Object> configs = new HashMap<>();
final String maxBlockMs = "300000";
configs.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9999");
configs.put(ProducerConfig.MAX_BLOCK_MS_CONFIG, "30000");
configs.put(ProducerConfig.MAX_BLOCK_MS_CONFIG, maxBlockMs);
long refreshBackoffMs = 500L;
long refreshBackoffMaxMs = 5000L;
long metadataExpireMs = 60000L;
@ -1049,39 +1040,28 @@ public class KafkaProducerTest {
final ProducerMetadata metadata = new ProducerMetadata(refreshBackoffMs, refreshBackoffMaxMs, metadataExpireMs, metadataIdleMs,
new LogContext(), new ClusterResourceListeners(), time);
final String topic = "topic";
final String warmupTopic = "warmup-topic";
MockClient client = new MockClient(time, metadata);
client.setShouldUpdateWithCurrentMetadata(false);
client.advanceTimeDuringPoll(true);
try (KafkaProducer<String, String> producer = kafkaProducer(configs, new StringSerializer(),
new StringSerializer(), metadata, new MockClient(time, metadata), null, time)) {
new StringSerializer(), metadata, client, null, time)) {
Exchanger<Void> exchanger = new Exchanger<>();
MetadataResponse updateResponse = RequestTestUtils.metadataUpdateWith(1, singletonMap(warmupTopic, 1));
client.updateMetadata(updateResponse);
Thread t = new Thread(() -> {
try {
exchanger.exchange(null); // 1
while (!metadata.updateRequested())
Thread.sleep(100);
MetadataResponse updateResponse = RequestTestUtils.metadataUpdateWith(1, singletonMap(topic, 1));
metadata.updateWithCurrentRequestVersion(updateResponse, false, time.milliseconds());
exchanger.exchange(null); // 2
time.sleep(120 * 1000L);
// Update the metadata again, but it should be expired at this point.
updateResponse = RequestTestUtils.metadataUpdateWith(1, singletonMap(topic, 1));
metadata.updateWithCurrentRequestVersion(updateResponse, false, time.milliseconds());
exchanger.exchange(null); // 3
while (!metadata.updateRequested())
Thread.sleep(100);
time.sleep(30 * 1000L);
} catch (Exception e) {
throw new RuntimeException(e);
}
});
t.start();
exchanger.exchange(null); // 1
updateResponse = RequestTestUtils.metadataUpdateWith(1, singletonMap(topic, 1));
client.prepareMetadataUpdate(updateResponse);
assertNotNull(producer.partitionsFor(topic));
exchanger.exchange(null); // 2
exchanger.exchange(null); // 3
// Update the metadata again, but it should be expired at this point.
time.sleep(120 * 1000L);
updateResponse = RequestTestUtils.metadataUpdateWith(1, singletonMap(warmupTopic, 1));
client.updateMetadata(updateResponse);
assertFalse(client.awaitMetadataUpdateRequest(0));
assertThrows(TimeoutException.class, () -> producer.partitionsFor(topic));
t.join();
assertTrue(client.awaitMetadataUpdateRequest(0));
}
}