Skip to content

Commit b9d5597

Browse files
KAFKA-17830 Cover unit tests for TBRLMM init failure scenarios (#19076)
Add unit tests for TBRLMM when initializing clients. Reviewers: Chia-Ping Tsai <[email protected]>
1 parent 84b8fec commit b9d5597

File tree

2 files changed

+18
-0
lines changed

2 files changed

+18
-0
lines changed

storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManager.java

+4
Original file line numberDiff line numberDiff line change
@@ -555,6 +555,10 @@ boolean isInitialized() {
555555
return initialized.get();
556556
}
557557

558+
boolean isInitializationFailed() {
559+
return initializationFailed;
560+
}
561+
558562
private void ensureInitializedAndNotClosed() {
559563
if (initializationFailed) {
560564
// If initialization is failed, shutdown the broker.

storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerTest.java

+14
Original file line numberDiff line numberDiff line change
@@ -29,12 +29,14 @@
2929
import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
3030
import org.apache.kafka.server.log.remote.storage.RemoteResourceNotFoundException;
3131
import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
32+
import org.apache.kafka.test.TestUtils;
3233

3334
import org.junit.jupiter.api.AfterEach;
3435

3536
import java.io.IOException;
3637
import java.util.Arrays;
3738
import java.util.Collections;
39+
import java.util.Map;
3840
import java.util.concurrent.CountDownLatch;
3941
import java.util.concurrent.ExecutionException;
4042
import java.util.concurrent.TimeUnit;
@@ -307,4 +309,16 @@ public void testRemoteLogSizeCalculationWithSegmentsHavingNonExistentEpochs() th
307309
assertEquals(0, topicBasedRemoteLogMetadataManager.remoteLogSize(topicIdPartition, 9001));
308310
}
309311

312+
@ClusterTest
313+
public void testInitializationFailure() throws IOException, InterruptedException {
314+
try (TopicBasedRemoteLogMetadataManager rlmm = new TopicBasedRemoteLogMetadataManager()) {
315+
// configure rlmm without bootstrap servers, so it will fail to initialize admin client.
316+
Map<String, Object> configs = Map.of(
317+
TopicBasedRemoteLogMetadataManagerConfig.LOG_DIR, TestUtils.tempDirectory("rlmm_segs_").getAbsolutePath(),
318+
TopicBasedRemoteLogMetadataManagerConfig.BROKER_ID, 0
319+
);
320+
rlmm.configure(configs);
321+
TestUtils.waitForCondition(rlmm::isInitializationFailed, "Initialization should fail");
322+
}
323+
}
310324
}

0 commit comments

Comments
 (0)