jsancio commented on code in PR #15119:
URL: https://github.com/apache/kafka/pull/15119#discussion_r1442310315


##########
raft/src/main/java/org/apache/kafka/raft/KafkaRaftClientDriver.java:
##########
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.raft;
+
+import org.apache.kafka.common.protocol.ApiMessage;
+import org.apache.kafka.common.requests.RequestHeader;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.server.fault.FaultHandler;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+
+import java.util.concurrent.CompletableFuture;
+
+
+/**
+ * A single-threaded driver for {@link KafkaRaftClient}.
+ *
+ * @param <T> See {@link KafkaRaftClient<T>}
+ */
+public class KafkaRaftClientDriver<T> extends ShutdownableThread {
+    private final Logger log;
+    private final KafkaRaftClient<T> client;
+    private final FaultHandler fatalFaultHandler;
+
+    public KafkaRaftClientDriver(
+        KafkaRaftClient<T> client,
+        String threadNamePrefix,
+        FaultHandler fatalFaultHandler,
+        LogContext logContext
+    ) {
+        super(threadNamePrefix + "-io-thread", false);
+        this.client = client;
+        this.fatalFaultHandler = fatalFaultHandler;
+        this.log = logContext.logger(KafkaRaftClientDriver.class);
+    }
+
+    @Override
+    public void doWork() {
+        try {
+            client.poll();
+        } catch (Throwable t) {
+            throw fatalFaultHandler.handleFault("Unexpected error in raft IO 
thread", t);
+        }
+    }
+
+    @Override
+    public boolean initiateShutdown() {
+        if (super.initiateShutdown()) {
+            client.shutdown(5000).whenComplete((na, exception) -> {
+                if (exception != null) {
+                    log.error("Graceful shutdown of RaftClient failed", 
exception);
+                } else {
+                    log.info("Completed graceful shutdown of RaftClient");
+                }
+            });
+            return true;
+        } else {
+            return false;
+        }
+    }
+
+    /**
+     * Shutdown the thread. In addition to stopping any utilized threads, this 
will
+     * close the {@link KafkaRaftClient} instance.
+     */
+    @Override
+    public void shutdown() throws InterruptedException {
+        try {
+            super.shutdown();
+        } finally {
+            client.close();
+        }
+    }
+
+    @Override
+    public boolean isRunning() {
+        return client.isRunning() && !isThreadFailed();
+    }
+
+    public CompletableFuture<ApiMessage> handleRequest(
+        RequestHeader header,
+        ApiMessage request,
+        long createdTimeMs
+    ) {
+        RaftRequest.Inbound inboundRequest = new RaftRequest.Inbound(
+            header.correlationId(),
+            request,
+            createdTimeMs
+        );
+
+        client.handle(inboundRequest);
+
+        return inboundRequest.completion.thenApply(response -> response.data);

Review Comment:
   How about:
   ```java
           return inboundRequest.completion.thenApply(RaftMessage::data));
   ```



##########
raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientDriverTest.java:
##########
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.raft;
+
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.server.fault.MockFaultHandler;
+import org.junit.jupiter.api.Test;
+import org.mockito.Mockito;
+
+import java.util.concurrent.CompletableFuture;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+class KafkaRaftClientDriverTest {
+
+    @Test
+    public void testShutdown() throws Exception {
+        @SuppressWarnings("unchecked")
+        KafkaRaftClient<String> raftClient = (KafkaRaftClient<String>) 
Mockito.mock(KafkaRaftClient.class);
+        MockFaultHandler faultHandler = new 
MockFaultHandler("TestFaultHandler");
+        KafkaRaftClientDriver<String> driver = new KafkaRaftClientDriver<>(
+            raftClient,
+            "test-raft",
+            faultHandler,
+            new LogContext()
+        );
+
+        when(raftClient.isRunning()).thenReturn(true);
+        assertTrue(driver.isRunning());
+
+        CompletableFuture<Void> shutdownFuture = new CompletableFuture<>();
+        when(raftClient.shutdown(5000)).thenReturn(shutdownFuture);
+
+        driver.initiateShutdown();
+        assertTrue(driver.isRunning());
+        assertTrue(driver.isShutdownInitiated());
+        verify(raftClient).shutdown(5000);
+
+        shutdownFuture.complete(null);
+        when(raftClient.isRunning()).thenReturn(false);
+        driver.run();
+        assertFalse(driver.isRunning());
+        assertTrue(driver.isShutdownComplete());
+        assertNull(faultHandler.firstException());
+
+        driver.shutdown();
+        verify(raftClient).close();
+    }
+
+

Review Comment:
   Extra newline.



##########
raft/src/main/java/org/apache/kafka/raft/KafkaRaftClientDriver.java:
##########
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.raft;
+
+import org.apache.kafka.common.protocol.ApiMessage;
+import org.apache.kafka.common.requests.RequestHeader;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.server.fault.FaultHandler;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+
+import java.util.concurrent.CompletableFuture;
+
+

Review Comment:
   Extra newline.



##########
raft/src/main/java/org/apache/kafka/raft/KafkaRaftClientDriver.java:
##########
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.raft;
+
+import org.apache.kafka.common.protocol.ApiMessage;
+import org.apache.kafka.common.requests.RequestHeader;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.server.fault.FaultHandler;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+
+import java.util.concurrent.CompletableFuture;
+
+
+/**
+ * A single-threaded driver for {@link KafkaRaftClient}.
+ *
+ * @param <T> See {@link KafkaRaftClient<T>}
+ */
+public class KafkaRaftClientDriver<T> extends ShutdownableThread {
+    private final Logger log;
+    private final KafkaRaftClient<T> client;
+    private final FaultHandler fatalFaultHandler;
+
+    public KafkaRaftClientDriver(
+        KafkaRaftClient<T> client,
+        String threadNamePrefix,
+        FaultHandler fatalFaultHandler,
+        LogContext logContext
+    ) {
+        super(threadNamePrefix + "-io-thread", false);
+        this.client = client;
+        this.fatalFaultHandler = fatalFaultHandler;
+        this.log = logContext.logger(KafkaRaftClientDriver.class);
+    }
+
+    @Override
+    public void doWork() {
+        try {
+            client.poll();
+        } catch (Throwable t) {
+            throw fatalFaultHandler.handleFault("Unexpected error in raft IO 
thread", t);
+        }
+    }
+
+    @Override
+    public boolean initiateShutdown() {
+        if (super.initiateShutdown()) {
+            client.shutdown(5000).whenComplete((na, exception) -> {
+                if (exception != null) {
+                    log.error("Graceful shutdown of RaftClient failed", 
exception);
+                } else {
+                    log.info("Completed graceful shutdown of RaftClient");
+                }
+            });
+            return true;
+        } else {
+            return false;
+        }
+    }
+
+    /**
+     * Shutdown the thread. In addition to stopping any utilized threads, this 
will
+     * close the {@link KafkaRaftClient} instance.
+     */
+    @Override
+    public void shutdown() throws InterruptedException {
+        try {
+            super.shutdown();
+        } finally {
+            client.close();

Review Comment:
   Should this type `close` the `KafkaRaftClient`? In the `raft` module we 
generally establish `close`/`shutdown` ownership based on if the type created 
the object or if it is was passed in through a function/constructor.
   
   Take a look at `KafkaRaftClient` for an example. `KafkaRaftClient` doesn't 
close `ReplicatedLog`, `NetworkChannel`, etc because they were passed through 
the constructor. In the other hand `KafkaRaftClient` does close 
`kafkaRaftMetrics` and `memoryPool` because they were created by 
`KafkaRaftClient`.



-- 
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