kamalcph commented on code in PR #20007:
URL: https://github.com/apache/kafka/pull/20007#discussion_r2227573033


##########
storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManager.java:
##########
@@ -368,6 +368,12 @@ private void 
initializeResources(TopicBasedRemoteLogMetadataManagerConfig rlmmCo
         } catch (KafkaException e) {
             log.error("Encountered error while initializing topic-based RLMM 
resources", e);
             initializationFailed = true;
+        } finally {
+            Utils.closeQuietly(adminClient, "AdminClient");
+            if (initializationFailed) {

Review Comment:
   The `initializationFailed` variable can be moved from global to local scope.



##########
storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerTest.java:
##########
@@ -336,14 +340,37 @@ public void 
testRemoteLogSizeCalculationWithSegmentsHavingNonExistentEpochs() th
 
     @ClusterTest
     public void testInitializationFailure() throws IOException, 
InterruptedException {
+        // Set up a custom exit procedure for testing
+        final AtomicBoolean exitCalled = new AtomicBoolean(false);
+        final AtomicInteger exitCode = new AtomicInteger(-1);
+        final AtomicReference<String> exitMessage = new AtomicReference<>();
+        
+        // Set custom exit procedure that won't actually exit the process
+        Exit.setExitProcedure((statusCode, message) -> {
+            exitCalled.set(true);
+            exitCode.set(statusCode);
+            exitMessage.set(message);
+        });
+
         try (TopicBasedRemoteLogMetadataManager rlmm = new 
TopicBasedRemoteLogMetadataManager()) {
             // configure rlmm without bootstrap servers, so it will fail to 
initialize admin client.
             Map<String, Object> configs = Map.of(
                 TopicBasedRemoteLogMetadataManagerConfig.LOG_DIR, 
TestUtils.tempDirectory("rlmm_segs_").getAbsolutePath(),
                 TopicBasedRemoteLogMetadataManagerConfig.BROKER_ID, 0
             );
             rlmm.configure(configs);
-            TestUtils.waitForCondition(rlmm::isInitializationFailed, 
"Initialization should fail");
+            
+            // Wait for initialization failure and exit procedure to be called
+            TestUtils.waitForCondition(() -> exitCalled.get(), 
+                "Exit procedure should be called due to initialization 
failure");
+            
+            // Verify exit code and initialization failure status
+            assertTrue(exitCalled.get(), "Exit procedure should have been 
called");

Review Comment:
   This assert and the above are same.



##########
storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManager.java:
##########
@@ -368,6 +368,12 @@ private void 
initializeResources(TopicBasedRemoteLogMetadataManagerConfig rlmmCo
         } catch (KafkaException e) {
             log.error("Encountered error while initializing topic-based RLMM 
resources", e);
             initializationFailed = true;
+        } finally {
+            Utils.closeQuietly(adminClient, "AdminClient");

Review Comment:
   No need to close `Admin` instance explicitly as it is enclosed in 
try-with-resources. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to