hachikuji commented on a change in pull request #9839: URL: https://github.com/apache/kafka/pull/9839#discussion_r553749199
########## File path: core/src/main/scala/kafka/raft/RaftManager.scala ########## @@ -0,0 +1,279 @@ +/* + * 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 kafka.raft + +import java.io.File +import java.nio.file.Files +import java.util.Random +import java.util.concurrent.CompletableFuture + +import kafka.log.{Log, LogConfig, LogManager} +import kafka.raft.KafkaRaftManager.RaftIoThread +import kafka.server.{BrokerTopicStats, KafkaConfig, KafkaServer, LogDirFailureChannel} +import kafka.utils.timer.SystemTimer +import kafka.utils.{KafkaScheduler, Logging, ShutdownableThread} +import org.apache.kafka.clients.{ApiVersions, ClientDnsLookup, ManualMetadataUpdater, NetworkClient} +import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.metrics.Metrics +import org.apache.kafka.common.network.{ChannelBuilders, NetworkReceive, Selectable, Selector} +import org.apache.kafka.common.protocol.ApiMessage +import org.apache.kafka.common.requests.RequestHeader +import org.apache.kafka.common.security.JaasContext +import org.apache.kafka.common.utils.{LogContext, Time} +import org.apache.kafka.raft.{FileBasedStateStore, KafkaRaftClient, QuorumState, RaftClient, RaftConfig, RaftRequest, RecordSerde} + +import scala.jdk.CollectionConverters._ + +object KafkaRaftManager { + class RaftIoThread( + client: KafkaRaftClient[_] + ) extends ShutdownableThread( + name = "raft-io-thread", + isInterruptible = false + ) { + override def doWork(): Unit = { + client.poll() + } + + override def initiateShutdown(): Boolean = { + if (super.initiateShutdown()) { + client.shutdown(5000).whenComplete { (_, exception) => + if (exception != null) { + error("Graceful shutdown of RaftClient failed", exception) + } else { + info("Completed graceful shutdown of RaftClient") + } + } + true + } else { + false + } + } + + override def isRunning: Boolean = { + client.isRunning && !isThreadFailed Review comment: It's a good question and I was debating it earlier today. It's worth keeping in mind that `KafkaRaftClient` doesn't really own the resources that would need closing. Sockets and files are owned by `RaftManager`, so they will still get closed properly. The one exception is de-registration of metrics in `KafkaRaftMetrics.close`. The main purpose of `shutdown` is to attempt a graceful departure from the quorum. But after we after we have hit an unexpected exception and are left in an undetermined state, do we still want to try the graceful shutdown? I was leaning toward 'no'. Perhaps what this suggests is that we should have a separate `close` method which can handle cleanup of any resources that `KafkaRaftClient` owns today (and might own in the future). If we hit an uncaught exception, then we can forego the call to `shutdown`, but we can ensure that `close` always gets called. How does that sound? ---------------------------------------------------------------- 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org