-
Notifications
You must be signed in to change notification settings - Fork 13.6k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
KAFKA-15019: Improve handling of broker heartbeat timeouts #13759
Changes from 1 commit
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -1384,7 +1384,9 @@ ApiError electLeader(String topic, int partitionId, ElectionType electionType, | |
} | ||
|
||
ControllerResult<BrokerHeartbeatReply> processBrokerHeartbeat( | ||
BrokerHeartbeatRequestData request, long registerBrokerRecordOffset) { | ||
BrokerHeartbeatRequestData request, | ||
long registerBrokerRecordOffset | ||
) { | ||
int brokerId = request.brokerId(); | ||
long brokerEpoch = request.brokerEpoch(); | ||
clusterControl.checkBrokerEpoch(brokerId, brokerEpoch); | ||
|
@@ -1419,6 +1421,15 @@ ControllerResult<BrokerHeartbeatReply> processBrokerHeartbeat( | |
return ControllerResult.of(records, reply); | ||
} | ||
|
||
void processExpiredBrokerHeartbeat(BrokerHeartbeatRequestData request) { | ||
int brokerId = request.brokerId(); | ||
clusterControl.checkBrokerEpoch(brokerId, request.brokerEpoch()); | ||
clusterControl.heartbeatManager().touch(brokerId, | ||
clusterControl.brokerRegistrations().get(brokerId).fenced(), | ||
request.currentMetadataOffset()); | ||
log.error("processExpiredBrokerHeartbeat: timed out heartbeat from broker {}.", brokerId); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. What is the operator meant to do when they see this error message? Perhaps we can explain the implications in the log message? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I changed it to "controller event queue overloaded. timed out heartbeat from broker {}" |
||
} | ||
|
||
public ControllerResult<Void> unregisterBroker(int brokerId) { | ||
BrokerRegistration registration = clusterControl.brokerRegistrations().get(brokerId); | ||
if (registration == null) { | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,46 @@ | ||
/* | ||
* 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.controller.errors; | ||
|
||
import org.apache.kafka.common.errors.TimeoutException; | ||
|
||
import java.util.concurrent.ExecutionException; | ||
|
||
|
||
public class ControllerExceptions { | ||
/** | ||
* Check if an exception is a normal timeout exception. | ||
* | ||
* @param exception The exception to check. | ||
* @return True if the exception is a normal timeout exception and NOT a timeout | ||
* exception generated by the controller shutting down. | ||
*/ | ||
public static boolean isNormalTimeoutException(Throwable exception) { | ||
if (exception == null) return false; | ||
if (exception instanceof ExecutionException) { | ||
exception = exception.getCause(); | ||
if (exception == null) return false; | ||
} | ||
if (!(exception instanceof TimeoutException)) return false; | ||
if (exception.getMessage() != null && | ||
exception.getMessage().equals("The controller is shutting down.")) { | ||
return false; | ||
} | ||
return true; | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This seems brittle. Is there a way we can identify the specific thing we are interested in? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I removed the check for the exception text. (It wasn't actually doing anything here in any case!) This is something I'll revisit in a follow-on PR (there are a few corner cases around error handling and queue shutdown to fix up) |
||
} | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
What is the purpose of this line? Seems it is an unnecessary extra pause/resume, and the test passes without it. Can we remove it?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Removed