lzyLuke commented on code in PR #17502: URL: https://github.com/apache/kafka/pull/17502#discussion_r1801813685
########## metadata/src/main/java/org/apache/kafka/controller/BrokerHeartbeatTracker.java: ########## @@ -0,0 +1,137 @@ +/* + * 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; + +import org.apache.kafka.common.utils.Time; + +import java.util.Iterator; +import java.util.Map.Entry; +import java.util.Optional; +import java.util.OptionalLong; +import java.util.concurrent.ConcurrentHashMap; + +class BrokerHeartbeatTracker { + /** + * The clock to use. + */ + private final Time time; + + /** + * The broker session timeout in nanoseconds. + */ + private final long sessionTimeoutNs; + + /** + * Maps a broker ID and epoch to the last contact time in monotonic nanoseconds. + */ + private final ConcurrentHashMap<BrokerIdAndEpoch, Long> contactTimes; + + BrokerHeartbeatTracker(Time time, long sessionTimeoutNs) { + this.time = time; + this.sessionTimeoutNs = sessionTimeoutNs; + this.contactTimes = new ConcurrentHashMap<>(); + } + + Time time() { + return time; + } + + /** + * Update the contact time for the given broker ID and epoch to be the current time. + * + * @param idAndEpoch The broker ID and epoch. + */ + void updateContactTime(BrokerIdAndEpoch idAndEpoch) { + updateContactTime(idAndEpoch, time.nanoseconds()); + } + + /** + * Update the contact time for the given broker ID and epoch to be the given time. + * + * @param idAndEpoch The broker ID and epoch. + * @param timeNs The monotonic time in nanoseconds. + */ + void updateContactTime(BrokerIdAndEpoch idAndEpoch, long timeNs) { + contactTimes.put(idAndEpoch, timeNs); + } + + /** + * Get the contact time for the given broker ID and epoch. + * + * @param idAndEpoch The broker ID and epoch. + * @return The contact time, or Optional.empty if none is known. + */ + OptionalLong contactTime(BrokerIdAndEpoch idAndEpoch) { + Long value = contactTimes.get(idAndEpoch); + if (value == null) return OptionalLong.empty(); + return OptionalLong.of(value); + } + + /** + * Remove either one or zero expired brokers from the map. + * + * @return The expired broker that was removed, or Optional.empty if there was none. + */ + Optional<BrokerIdAndEpoch> maybeRemoveExpired() { + return maybeRemoveExpired(time.nanoseconds()); + } + + /** + * Remove either one or zero expired brokers from the map. + * + * @param nowNs The current time in monotonic nanoseconds. + * + * @return The expired broker that was removed, or Optional.empty if there was none. + */ + Optional<BrokerIdAndEpoch> maybeRemoveExpired(long nowNs) { + Iterator<Entry<BrokerIdAndEpoch, Long>> iterator = + contactTimes.entrySet().iterator(); + while (iterator.hasNext()) { Review Comment: ok, this is a concurrent map so it would not have a concurrent modifying exception, right? ########## metadata/src/main/java/org/apache/kafka/controller/BrokerHeartbeatManager.java: ########## @@ -82,23 +80,16 @@ static class BrokerHeartbeatState { */ private long controlledShutdownOffset; - /** - * The previous entry in the unfenced list, or null if the broker is not in that list. - */ - private BrokerHeartbeatState prev; - - /** - * The next entry in the unfenced list, or null if the broker is not in that list. - */ - private BrokerHeartbeatState next; - - BrokerHeartbeatState(int id) { + BrokerHeartbeatState( + int id, + boolean fenced, + long metadataOffset, + long controlledShutdownOffset Review Comment: what is this controlledShutdownOffset? And why we need this? ########## metadata/src/main/java/org/apache/kafka/controller/QuorumController.java: ########## @@ -1855,6 +1633,97 @@ private QuorumController( this.raftClient.register(metaLogListener); } + /** + * Register the writeNoOpRecord task. + * + * This task periodically writes a NoOpRecord to the metadata log, if the MetadataVersion + * supports it. + * + * @param maxIdleIntervalNs The period at which to write the NoOpRecord. + */ + private void registerWriteNoOpRecord(long maxIdleIntervalNs) { + periodicControl.registerTask(new PeriodicTask("writeNoOpRecord", + () -> { + ArrayList<ApiMessageAndVersion> records = new ArrayList<>(1); + if (featureControl.metadataVersion().isNoOpRecordSupported()) { + records.add(new ApiMessageAndVersion(new NoOpRecord(), (short) 0)); + } + return ControllerResult.of(records, false); + }, + maxIdleIntervalNs, + EnumSet.noneOf(PeriodicTaskFlag.class))); + } + + /** + * Calculate what the period should be for the maybeFenceStaleBroker task. + * + * @param sessionTimeoutNs The configured broker session timeout period in nanoseconds. + * + * @return The period for the maybeFenceStaleBroker task in nanoseconds. + */ + static long maybeFenceStaleBrokerPeriodNs(long sessionTimeoutNs) { + return Math.max(TimeUnit.MILLISECONDS.toNanos(1), sessionTimeoutNs / 4); Review Comment: what is the reason of mod `sessionTimeoutMs` by 4? ########## metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java: ########## @@ -1597,18 +1597,34 @@ public ControllerResult<Void> unregisterBroker(int brokerId) { return ControllerResult.of(records, null); } - ControllerResult<Void> maybeFenceOneStaleBroker() { - List<ApiMessageAndVersion> records = new ArrayList<>(); + ControllerResult<Boolean> maybeFenceOneStaleBroker() { BrokerHeartbeatManager heartbeatManager = clusterControl.heartbeatManager(); - heartbeatManager.findOneStaleBroker().ifPresent(brokerId -> { - // Even though multiple brokers can go stale at a time, we will process - // fencing one at a time so that the effect of fencing each broker is visible - // to the system prior to processing the next one - log.info("Fencing broker {} because its session has timed out.", brokerId); - handleBrokerFenced(brokerId, records); - heartbeatManager.fence(brokerId); - }); - return ControllerResult.of(records, null); + Optional<BrokerIdAndEpoch> idAndEpoch = heartbeatManager.tracker().maybeRemoveExpired(); + if (!idAndEpoch.isPresent()) { + log.debug("No stale brokers found."); + return ControllerResult.of(Collections.emptyList(), false); + } + int id = idAndEpoch.get().id(); + long epoch = idAndEpoch.get().epoch(); + if (!clusterControl.brokerRegistrations().containsKey(id)) { + log.info("Removing heartbeat tracker entry for unknown broker {} at epoch {}.", + id, epoch); + heartbeatManager.remove(id); + return ControllerResult.of(Collections.emptyList(), true); + } else if (clusterControl.brokerRegistrations().get(id).epoch() != epoch) { + log.info("Removing heartbeat tracker entry for broker {} at previous epoch {}. " + + "Current epoch is {}", id, epoch, + clusterControl.brokerRegistrations().get(id).epoch()); + return ControllerResult.of(Collections.emptyList(), true); + } + // Even though multiple brokers can go stale at a time, we will process + // fencing one at a time so that the effect of fencing each broker is visible + // to the system prior to processing the next one. + log.info("Fencing broker {} at epoch {} because its session has timed out.", id, epoch); Review Comment: should we change this log level to error? We fence broker because it has a session timeout issue like network connection right? ########## metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java: ########## @@ -511,6 +511,22 @@ BrokerFeature processRegistrationFeature( setMaxSupportedVersion(feature.maxSupportedVersion()); } + /** + * Track an incoming broker heartbeat. Unlike most functions, this one is not called from the main + * controller thread, so it can only access local, volatile and atomic data. Review Comment: qq: if not called from main controller thread, who will call this? -- 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