From 859aa11295d83d14552ac8fda7185972d9564a58 Mon Sep 17 00:00:00 2001 From: Akhilesh Chaganti Date: Wed, 14 Dec 2022 18:29:56 -0800 Subject: [PATCH 01/30] KAFKA-14493: Introduce Zk to KRaft migration state machine STUBs in KRaft controller. This patch introduces a preliminary state machine that can be used by KRaft controller to drive online migration from Zk to KRaft. MigrationState -- Defines the states we can have while migration from Zk to KRaft. KRaftMigrationDriver -- Defines the state transitions, and events to handle actions like controller change, metadata change, broker change and have interfaces through which it claims Zk controllership, performs zk writes and sends RPCs to ZkBrokers. MigrationClient -- Interface that defines the functions used to claim and relinquish Zk controllership, read to and write from Zk. BrokersRpcClient -- Interface that defines the functions used to send RPCs to Zk brokers. --- .../scala/kafka/zk/ZkMigrationClient.scala | 18 +++++++++++++++--- .../metadata/migration/MigrationClient.java | 13 +++++++++++++ .../migration/ZkMigrationLeadershipState.java | 18 ++++++++++++++++++ 3 files changed, 46 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/kafka/zk/ZkMigrationClient.scala b/core/src/main/scala/kafka/zk/ZkMigrationClient.scala index 017f773ee21c8..d6c2149846413 100644 --- a/core/src/main/scala/kafka/zk/ZkMigrationClient.scala +++ b/core/src/main/scala/kafka/zk/ZkMigrationClient.scala @@ -1,4 +1,4 @@ -/** +/* * 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. @@ -28,6 +28,7 @@ import org.apache.kafka.common.metadata.ClientQuotaRecord.EntityData import org.apache.kafka.common.metadata._ import org.apache.kafka.common.quota.ClientQuotaEntity import org.apache.kafka.common.{TopicPartition, Uuid} +import org.apache.kafka.image.{MetadataDelta, MetadataImage} import org.apache.kafka.metadata.{LeaderRecoveryState, PartitionRegistration} import org.apache.kafka.metadata.migration.{MigrationClient, ZkMigrationLeadershipState} import org.apache.kafka.server.common.{ApiMessageAndVersion, MetadataVersion, ProducerIdsBlock} @@ -40,7 +41,10 @@ import java.util.function.Consumer import scala.collection.Seq import scala.jdk.CollectionConverters._ - +/** + * Migration client in KRaft controller responsible for handling communication to Zookeeper and + * the ZkBrokers present in the cluster. + */ class ZkMigrationClient(zkClient: KafkaZkClient) extends MigrationClient with Logging { override def getOrCreateMigrationRecoveryState(initialState: ZkMigrationLeadershipState): ZkMigrationLeadershipState = { @@ -54,7 +58,8 @@ class ZkMigrationClient(zkClient: KafkaZkClient) extends MigrationClient with Lo override def claimControllerLeadership(state: ZkMigrationLeadershipState): ZkMigrationLeadershipState = { zkClient.tryRegisterKRaftControllerAsActiveController(state.kraftControllerId(), state.kraftControllerEpoch()) match { - case SuccessfulRegistrationResult(_, controllerEpochZkVersion) => state.withControllerZkVersion(controllerEpochZkVersion) + case SuccessfulRegistrationResult(_, controllerEpochZkVersion) => + zkClient.getOrCreateMigrationState(state.withControllerZkVersion(controllerEpochZkVersion)) case FailedRegistrationResult() => state.withControllerZkVersion(-1) } } @@ -435,4 +440,11 @@ class ZkMigrationClient(zkClient: KafkaZkClient) extends MigrationClient with Lo state } } + + override def writeMetadataDeltaToZookeeper(delta: MetadataDelta, + image: MetadataImage, + state: ZkMigrationLeadershipState): ZkMigrationLeadershipState = { + // TODO + state + } } diff --git a/metadata/src/main/java/org/apache/kafka/metadata/migration/MigrationClient.java b/metadata/src/main/java/org/apache/kafka/metadata/migration/MigrationClient.java index 5eccbc70625ce..1710458cd57fd 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/migration/MigrationClient.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/migration/MigrationClient.java @@ -17,6 +17,8 @@ package org.apache.kafka.metadata.migration; import org.apache.kafka.common.Uuid; +import org.apache.kafka.image.MetadataDelta; +import org.apache.kafka.image.MetadataImage; import org.apache.kafka.metadata.PartitionRegistration; import org.apache.kafka.server.common.ApiMessageAndVersion; @@ -88,4 +90,15 @@ ZkMigrationLeadershipState updateTopicPartitions( Set readBrokerIds(); Set readBrokerIdsFromTopicAssignments(); + + /** + * Convert the Metadata delta to Zookeeper writes and persist the changes. On successful + * write, update the migration state with new metadata offset and epoch. + * @param delta Changes in the cluster metadata + * @param image New metadata after the changes in `delta` are applied + * @param state Current migration state before writing to Zookeeper. + */ + ZkMigrationLeadershipState writeMetadataDeltaToZookeeper(MetadataDelta delta, + MetadataImage image, + ZkMigrationLeadershipState state); } diff --git a/metadata/src/main/java/org/apache/kafka/metadata/migration/ZkMigrationLeadershipState.java b/metadata/src/main/java/org/apache/kafka/metadata/migration/ZkMigrationLeadershipState.java index b6217ee80c2dd..8c7af0dfead20 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/migration/ZkMigrationLeadershipState.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/migration/ZkMigrationLeadershipState.java @@ -16,6 +16,8 @@ */ package org.apache.kafka.metadata.migration; +import org.apache.kafka.raft.OffsetAndEpoch; + import java.util.Objects; /** @@ -71,6 +73,18 @@ public ZkMigrationLeadershipState withNewKRaftController(int controllerId, int c this.kraftMetadataEpoch, this.lastUpdatedTimeMs, this.migrationZkVersion, this.controllerZkVersion); } + public ZkMigrationLeadershipState withKRaftMetadataOffsetAndEpoch(long metadataOffset, + int metadataEpoch) { + return new ZkMigrationLeadershipState( + this.kraftControllerId, + this.kraftControllerEpoch, + metadataOffset, + metadataEpoch, + this.lastUpdatedTimeMs, + this.migrationZkVersion, + this.controllerZkVersion); + } + public int kraftControllerId() { return kraftControllerId; } @@ -103,6 +117,10 @@ public boolean zkMigrationComplete() { return kraftMetadataOffset > 0; } + public OffsetAndEpoch offsetAndEpoch() { + return new OffsetAndEpoch(kraftMetadataOffset, kraftMetadataEpoch); + } + @Override public String toString() { return "ZkMigrationLeadershipState{" + From fcde47a75384bb40d9394ec9c3186084af2a314c Mon Sep 17 00:00:00 2001 From: Akhilesh Chaganti Date: Wed, 14 Dec 2022 18:49:30 -0800 Subject: [PATCH 02/30] add missing files --- .../metadata/migration/BrokersRpcClient.java | 39 ++ .../migration/KRaftMigrationDriver.java | 397 ++++++++++++++++++ .../metadata/migration/MigrationState.java | 56 +++ 3 files changed, 492 insertions(+) create mode 100644 metadata/src/main/java/org/apache/kafka/metadata/migration/BrokersRpcClient.java create mode 100644 metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java create mode 100644 metadata/src/main/java/org/apache/kafka/metadata/migration/MigrationState.java diff --git a/metadata/src/main/java/org/apache/kafka/metadata/migration/BrokersRpcClient.java b/metadata/src/main/java/org/apache/kafka/metadata/migration/BrokersRpcClient.java new file mode 100644 index 0000000000000..3d7f21ce956de --- /dev/null +++ b/metadata/src/main/java/org/apache/kafka/metadata/migration/BrokersRpcClient.java @@ -0,0 +1,39 @@ +/* + * 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.metadata.migration; + +import org.apache.kafka.image.MetadataDelta; +import org.apache.kafka.image.MetadataImage; + +import java.util.Optional; + +public interface BrokersRpcClient { + void addZkBroker(); + + void removeZkBroker(); + + void publishMetadata(MetadataImage image, + Optional deltaOpt); + + void sendRPCsToBrokersFromMetadataDelta(MetadataDelta delta, + MetadataImage image, + int controllerEpoch); + + void sendRPCsToBrokersFromMetadataImage(MetadataImage image, int controllerEpoch); + + void reset(); +} diff --git a/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java b/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java new file mode 100644 index 0000000000000..da24850ac5009 --- /dev/null +++ b/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java @@ -0,0 +1,397 @@ +/* + * 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.metadata.migration; + +import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.image.MetadataDelta; +import org.apache.kafka.image.MetadataImage; +import org.apache.kafka.queue.EventQueue; +import org.apache.kafka.queue.KafkaEventQueue; + + +import org.apache.kafka.raft.OffsetAndEpoch; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.function.Function; + +import static java.util.concurrent.TimeUnit.NANOSECONDS; +import static java.util.concurrent.TimeUnit.SECONDS; + +/** + * This class orchestrates and manages the state related to a ZK to KRaft migration. An event thread is used to + * serialize events coming from various threads and listeners. + */ +public class KRaftMigrationDriver { + private final Time time; + private final Logger log; + private final int nodeId; + private final MigrationClient zkMigrationClient; + private final BrokersRpcClient rpcClient; + private final KafkaEventQueue eventQueue; + private volatile MigrationState migrationState; + private volatile ZkMigrationLeadershipState migrationLeadershipState; + private volatile MetadataDelta delta; + private volatile MetadataImage image; + + public KRaftMigrationDriver(int nodeId, MigrationClient zkMigrationClient, BrokersRpcClient rpcClient) { + this.nodeId = nodeId; + this.time = Time.SYSTEM; + this.log = LoggerFactory.getLogger(KRaftMigrationDriver.class); + this.migrationState = MigrationState.UNINITIALIZED; + this.migrationLeadershipState = ZkMigrationLeadershipState.EMPTY; + this.zkMigrationClient = zkMigrationClient; + this.rpcClient = rpcClient; + this.eventQueue = new KafkaEventQueue(Time.SYSTEM, new LogContext("KRaftMigrationDriver"), "kraft-migration"); + this.delta = null; + this.image = MetadataImage.EMPTY; + } + + public void start() { + eventQueue.prepend(new PollEvent()); + } + + public void shutdown() throws InterruptedException { + eventQueue.close(); + } + + public void handleLeaderChange(boolean isActive, int epoch) { + eventQueue.append(new KRaftLeaderEvent(isActive, epoch)); + } + + public void publishMetadata(MetadataDelta delta, MetadataImage image) { + eventQueue.append(new MetadataChangeEvent(delta, image)); + } + + private void initializeMigrationState() { + log.info("Recovering migration state"); + apply("Recovery", zkMigrationClient::getOrCreateMigrationRecoveryState); + String maybeDone = migrationLeadershipState.zkMigrationComplete() ? "done" : "not done"; + log.info("Recovered migration state {}. ZK migration is {}.", migrationLeadershipState, maybeDone); + // Let's transition to INACTIVE state and wait for leadership events. + transitionTo(MigrationState.INACTIVE); + } + + private boolean isControllerQuorumReadyForMigration() { + // TODO + return false; + } + + private boolean areZkBrokersReadyForMigration() { + // TODO: Check available broker registrations and known topic assignments to confirm of + // all Zk brokers are registered before beginning migration. + return false; + } + + private void apply(String name, Function stateMutator) { + ZkMigrationLeadershipState beforeState = this.migrationLeadershipState; + ZkMigrationLeadershipState afterState = stateMutator.apply(beforeState); + log.debug("{} transitioned from {} to {}", name, beforeState, afterState); + this.migrationLeadershipState = afterState; + } + + private boolean isValidStateChange(MigrationState newState) { + if (migrationState == newState) + return true; + switch (migrationState) { + case UNINITIALIZED: + case DUAL_WRITE: + return newState == MigrationState.INACTIVE; + case INACTIVE: + return newState == MigrationState.WAIT_FOR_CONTROLLER_QUORUM; + case WAIT_FOR_CONTROLLER_QUORUM: + return + newState == MigrationState.INACTIVE || + newState == MigrationState.BECOME_CONTROLLER; + case BECOME_CONTROLLER: + return + newState == MigrationState.INACTIVE || + newState == MigrationState.WAIT_FOR_BROKERS || + newState == MigrationState.KRAFT_CONTROLLER_TO_BROKER_COMM; + case WAIT_FOR_BROKERS: + return + newState == MigrationState.INACTIVE || + newState == MigrationState.ZK_MIGRATION; + case ZK_MIGRATION: + return + newState == MigrationState.INACTIVE || + newState == MigrationState.KRAFT_CONTROLLER_TO_BROKER_COMM; + case KRAFT_CONTROLLER_TO_BROKER_COMM: + return + newState == MigrationState.INACTIVE || + newState == MigrationState.DUAL_WRITE; + default: + log.error("Migration driver trying to transition from an unknown state {}", migrationState); + return false; + } + } + + private void transitionTo(MigrationState newState) { + if (!isValidStateChange(newState)) { + log.error("Error transition in migration driver from {} to {}", migrationState, newState); + return; + } + log.debug("{} transitioning from {} to {} state", nodeId, migrationState, newState); + switch (newState) { + case UNINITIALIZED: + // No state can transition to UNITIALIZED. + throw new IllegalStateException("Illegal transition from " + migrationState + " to " + newState + " " + + "state in Zk to KRaft migration"); + case INACTIVE: + // Any state can go to INACTIVE. + break; + } + migrationState = newState; + } + + // Events handled by Migration Driver. + + class PollEvent implements EventQueue.Event { + @Override + public void run() throws Exception { + switch (migrationState) { + case UNINITIALIZED: + initializeMigrationState(); + break; + case INACTIVE: + // Nothing to do when the driver is inactive. We need to wait on the + // controller node's state to move forward. + break; + case WAIT_FOR_CONTROLLER_QUORUM: + eventQueue.append(new WaitForControllerQuorumEvent()); + break; + case BECOME_CONTROLLER: + eventQueue.append(new BecomeZkControllerEvent()); + break; + case WAIT_FOR_BROKERS: + eventQueue.append(new WaitForZkBrokersEvent()); + break; + case ZK_MIGRATION: + eventQueue.append(new MigrateMetadataEvent()); + break; + case KRAFT_CONTROLLER_TO_BROKER_COMM: + eventQueue.append(new SendRPCsToBrokersEvent()); + break; + case DUAL_WRITE: + // Nothing to do in the PollEvent. If there's metadata change, we use + // MetadataChange event to drive the writes to Zookeeper. + break; + } + + // Poll again after some time + long deadline = time.nanoseconds() + NANOSECONDS.convert(1, SECONDS); + eventQueue.scheduleDeferred( + "poll", + new EventQueue.DeadlineFunction(deadline), + new PollEvent()); + } + + @Override + public void handleException(Throwable e) { + log.error("Had an exception in " + this.getClass().getSimpleName(), e); + } + } + + class KRaftLeaderEvent implements EventQueue.Event { + private final boolean isActive; + private final int kraftControllerEpoch; + + KRaftLeaderEvent(boolean isActive, int kraftControllerEpoch) { + this.isActive = isActive; + this.kraftControllerEpoch = kraftControllerEpoch; + } + @Override + public void run() throws Exception { + // We can either the the active controller or just resigned being the controller. + switch (migrationState) { + case UNINITIALIZED: + // Poll and retry after initialization + long deadline = time.nanoseconds() + NANOSECONDS.convert(10, SECONDS); + eventQueue.scheduleDeferred( + "poll", + new EventQueue.DeadlineFunction(deadline), + this); + break; + default: + if (!isActive) { + apply("KRaftLeaderEvent is active", + state -> state.withControllerZkVersion(ZkMigrationLeadershipState.EMPTY.controllerZkVersion())); + transitionTo(MigrationState.INACTIVE); + } else { + // Apply the new KRaft state + apply("KRaftLeaderEvent not active", state -> state.withNewKRaftController(nodeId, kraftControllerEpoch)); + // Before becoming the controller fo ZkBrokers, we need to make sure the + // Controller Quorum can handle migration. + transitionTo(MigrationState.WAIT_FOR_CONTROLLER_QUORUM); + } + break; + } + } + + @Override + public void handleException(Throwable e) { + log.error("Had an exception in " + this.getClass().getSimpleName(), e); + } + } + + class WaitForControllerQuorumEvent implements EventQueue.Event { + + @Override + public void run() throws Exception { + switch (migrationState) { + case WAIT_FOR_CONTROLLER_QUORUM: + if (isControllerQuorumReadyForMigration()) { + log.debug("Controller Quorum is ready for Zk to KRaft migration"); + // Note that leadership would not change here. Hence we do not need to + // `apply` any leadership state change. + transitionTo(MigrationState.BECOME_CONTROLLER); + } + break; + default: + // Ignore the event as we're not trying to become controller anymore. + break; + } + } + + @Override + public void handleException(Throwable e) { + log.error("Had an exception in " + this.getClass().getSimpleName(), e); + } + } + + class BecomeZkControllerEvent implements EventQueue.Event { + @Override + public void run() throws Exception { + switch (migrationState) { + case BECOME_CONTROLLER: + // TODO: Handle unhappy path. + apply("BecomeZkLeaderEvent", zkMigrationClient::claimControllerLeadership); + if (!migrationLeadershipState.zkMigrationComplete()) { + transitionTo(MigrationState.WAIT_FOR_BROKERS); + } else { + transitionTo(MigrationState.KRAFT_CONTROLLER_TO_BROKER_COMM); + } + break; + default: + // Ignore the event as we're not trying to become controller anymore. + break; + } + } + + @Override + public void handleException(Throwable e) { + log.error("Had an exception in " + this.getClass().getSimpleName(), e); + } + } + + class WaitForZkBrokersEvent implements EventQueue.Event { + @Override + public void run() throws Exception { + switch (migrationState) { + case WAIT_FOR_BROKERS: + if (areZkBrokersReadyForMigration()) { + log.debug("Zk brokers are registered and ready for migration"); + transitionTo(MigrationState.ZK_MIGRATION); + } + break; + default: + // Ignore the event as we're not in the appropriate state anymore. + break; + } + } + + @Override + public void handleException(Throwable e) { + log.error("Had an exception in " + this.getClass().getSimpleName(), e); + } + } + + class MigrateMetadataEvent implements EventQueue.Event { + @Override + public void run() throws Exception { + // TODO: Do actual zk write. + OffsetAndEpoch offsetAndEpochAfterMigration = new OffsetAndEpoch(-1, -1); + log.debug("Completed migrating metadata from Zookeeper. Current offset is {} and " + + "epoch is {}", offsetAndEpochAfterMigration.offset(), + offsetAndEpochAfterMigration.epoch()); + ZkMigrationLeadershipState newState = migrationLeadershipState.withKRaftMetadataOffsetAndEpoch( + offsetAndEpochAfterMigration.offset(), + offsetAndEpochAfterMigration.epoch()); + apply("Migrate metadata from Zk", state -> zkMigrationClient.setMigrationRecoveryState(newState)); + transitionTo(MigrationState.KRAFT_CONTROLLER_TO_BROKER_COMM); + } + + @Override + public void handleException(Throwable e) { + log.error("Had an exception in " + this.getClass().getSimpleName(), e); + } + } + + class SendRPCsToBrokersEvent implements EventQueue.Event { + + @Override + public void run() throws Exception { + switch (migrationState) { + case KRAFT_CONTROLLER_TO_BROKER_COMM: + if (image.highestOffsetAndEpoch().compareTo(migrationLeadershipState.offsetAndEpoch()) >= 0) { + rpcClient.sendRPCsToBrokersFromMetadataImage(image, + migrationLeadershipState.kraftControllerEpoch()); + // Migration leadership state doesn't change since we're not doing any Zk + // writes. + transitionTo(MigrationState.DUAL_WRITE); + } + break; + default: + // Ignore sending RPCs to the brokers since we're no longer in the state. + break; + } + } + } + + class MetadataChangeEvent implements EventQueue.Event { + private final MetadataDelta delta; + private final MetadataImage image; + MetadataChangeEvent(MetadataDelta delta, MetadataImage image) { + this.delta = delta; + this.image = image; + } + + @Override + public void run() throws Exception { + KRaftMigrationDriver.this.image = image; + KRaftMigrationDriver.this.delta = delta; + + switch (migrationState) { + case DUAL_WRITE: + if (image.highestOffsetAndEpoch().compareTo(migrationLeadershipState.offsetAndEpoch()) >= 0) { + apply("Write MetadataDelta to Zk", + state -> zkMigrationClient.writeMetadataDeltaToZookeeper(delta, image, state)); + // TODO: Unhappy path: Probably relinquish leadership and let new controller + // retry the write? + rpcClient.sendRPCsToBrokersFromMetadataDelta(delta, image, + migrationLeadershipState.kraftControllerEpoch()); + } + break; + default: + log.debug("Received metadata change, but the controller is not in dual-write " + + "mode. Ignoring the change to be replicated to Zookeeper"); + break; + } + } + } +} diff --git a/metadata/src/main/java/org/apache/kafka/metadata/migration/MigrationState.java b/metadata/src/main/java/org/apache/kafka/metadata/migration/MigrationState.java new file mode 100644 index 0000000000000..2935d678f2aa5 --- /dev/null +++ b/metadata/src/main/java/org/apache/kafka/metadata/migration/MigrationState.java @@ -0,0 +1,56 @@ +/* + * 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.metadata.migration; + +/** + * UNINITIALIZED───────────────►INACTIVE◄────────────────DUAL_WRITE◄────────────────────────┐ + * │ ▲ │ + * │ │ │ + * │ │ │ + * │ │◄───────────────────────────────────────────────────────┤ + * │ │ │ + * ▼ │ │ + * WAIT_FOR_CONTROLLER_QUORUM───────────┘◄────────────────────ZK_MIGRATION────────────►KRAFT_CONTROLLER_TO_BROKER_COMM + * │ ▲ ▲ ▲ + * │ │ │ │ + * │ │ │ │ + * │ │◄────────────────────────┤ │ + * │ │ │ │ + * ▼ │ │ │ + * BECOME_CONTROLLER───────────────────►└────────────────────►WAIT_FOR_BROKERS───────────────────┘ + */ +public enum MigrationState { + UNINITIALIZED(false), // Initial state. + INACTIVE(false), // State when not the active controller. + WAIT_FOR_CONTROLLER_QUORUM(false), // Ensure all the quorum nodes are ready for migration. + BECOME_CONTROLLER(false), // Become controller for the Zk Brokers. + WAIT_FOR_BROKERS(true), // Wait for Zk brokers to be ready for migration. + ZK_MIGRATION(true), // The cluster has satisfied the migration criteria + KRAFT_CONTROLLER_TO_BROKER_COMM(true), // First communication from Controller to send full RPCs to the Zk brokers. + DUAL_WRITE(true); // The data has been migrated + + private final boolean isActiveController; + + MigrationState(boolean isActiveController) { + this.isActiveController = isActiveController; + } + + boolean isActiveController() { + return isActiveController; + } +} From 13c82dff828a5efb55651857018fa9f07e39f717 Mon Sep 17 00:00:00 2001 From: David Arthur Date: Thu, 15 Dec 2022 13:43:19 -0500 Subject: [PATCH 03/30] MINOR: ControllerServer should use the new metadata loader and snapshot generator. This PR introduces the new metadata loader and snapshot generator. For the time being, they are only used by the controller, but a PR for the broker will come soon. The new metadata loader supports adding and removing publishers dynamically. (In contrast, the old loader only supported adding a single publisher.) It also passes along more information about each new image that is published. This information can be found in the LogDeltaManifest and SnapshotManifest classes. The new snapshot generator replaces the previous logic for generating snapshots in QuorumController.java and associated classes. The new generator is intended to be shared between the broker and the controller, so it is decoupled from both. There are a few small changes to the old snapshot generator in this PR. Specifically, we move the batch processing time and batch size metrics out of BrokerMetadataListener.scala and into BrokerServerMetrics.scala. --- checkstyle/import-control.xml | 6 +- .../scala/kafka/server/ControllerServer.scala | 3 +- .../scala/kafka/server/SharedServer.scala | 77 ++- .../metadata/BrokerMetadataListener.scala | 39 +- .../server/metadata/BrokerServerMetrics.scala | 59 ++- .../test/java/kafka/test/MockController.java | 5 - .../metadata/BrokerServerMetricsTest.scala | 21 +- .../server/DynamicConfigChangeTest.scala | 4 +- .../metadata/BrokerMetadataListenerTest.scala | 8 +- .../kafka/controller/AclControlManager.java | 18 - .../controller/ClientQuotaControlManager.java | 42 -- .../controller/ClusterControlManager.java | 60 --- .../ConfigurationControlManager.java | 36 -- .../apache/kafka/controller/Controller.java | 8 - .../controller/FeatureControlManager.java | 44 -- .../kafka/controller/LogReplayTracker.java | 3 +- .../controller/ProducerIdControlManager.java | 24 +- .../kafka/controller/QuorumController.java | 307 +----------- .../controller/ReplicationControlManager.java | 34 -- .../kafka/controller/SnapshotGenerator.java | 132 ----- .../kafka/image/loader/LogDeltaManifest.java | 106 ++++ .../kafka/image/loader/MetadataLoader.java | 473 ++++++++++++++++++ .../image/loader/MetadataLoaderMetrics.java | 46 ++ .../kafka/image/loader/SnapshotManifest.java | 77 +++ .../image/publisher/MetadataPublisher.java | 72 +++ .../image/publisher/SnapshotEmitter.java | 123 +++++ .../image/publisher/SnapshotGenerator.java | 275 ++++++++++ .../controller/AclControlManagerTest.java | 24 +- .../ClientQuotaControlManagerTest.java | 59 ++- .../controller/ClusterControlManagerTest.java | 27 +- .../ConfigurationControlManagerTest.java | 8 - .../controller/FeatureControlManagerTest.java | 38 +- .../ProducerIdControlManagerTest.java | 30 +- .../controller/QuorumControllerTest.java | 401 ++------------- .../ReplicationControlManagerTest.java | 11 - .../controller/SnapshotGeneratorTest.java | 112 ----- .../image/publisher/SnapshotEmitterTest.java | 206 ++++++++ .../publisher/SnapshotGeneratorTest.java | 185 +++++++ .../apache/kafka/metalog/LocalLogManager.java | 19 +- .../kafka/metalog/LocalLogManagerTestEnv.java | 7 + 40 files changed, 1869 insertions(+), 1360 deletions(-) delete mode 100644 metadata/src/main/java/org/apache/kafka/controller/SnapshotGenerator.java create mode 100644 metadata/src/main/java/org/apache/kafka/image/loader/LogDeltaManifest.java create mode 100644 metadata/src/main/java/org/apache/kafka/image/loader/MetadataLoader.java create mode 100644 metadata/src/main/java/org/apache/kafka/image/loader/MetadataLoaderMetrics.java create mode 100644 metadata/src/main/java/org/apache/kafka/image/loader/SnapshotManifest.java create mode 100644 metadata/src/main/java/org/apache/kafka/image/publisher/MetadataPublisher.java create mode 100644 metadata/src/main/java/org/apache/kafka/image/publisher/SnapshotEmitter.java create mode 100644 metadata/src/main/java/org/apache/kafka/image/publisher/SnapshotGenerator.java delete mode 100644 metadata/src/test/java/org/apache/kafka/controller/SnapshotGeneratorTest.java create mode 100644 metadata/src/test/java/org/apache/kafka/image/publisher/SnapshotEmitterTest.java create mode 100644 metadata/src/test/java/org/apache/kafka/image/publisher/SnapshotGeneratorTest.java diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index df9a2e9adfdf4..32f13d0e34004 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -225,6 +225,7 @@ + @@ -248,15 +249,18 @@ - + + + + diff --git a/core/src/main/scala/kafka/server/ControllerServer.scala b/core/src/main/scala/kafka/server/ControllerServer.scala index 03b86b7067ee0..77f9ba546fdd6 100644 --- a/core/src/main/scala/kafka/server/ControllerServer.scala +++ b/core/src/main/scala/kafka/server/ControllerServer.scala @@ -112,6 +112,7 @@ class ControllerServer( maybeChangeStatus(STARTING, STARTED) this.logIdent = new LogContext(s"[ControllerServer id=${config.nodeId}] ").logPrefix() + newGauge("ClusterId", () => clusterId) newGauge("yammer-metrics-count", () => KafkaYammerMetrics.defaultRegistry.allMetrics.size) @@ -194,8 +195,6 @@ class ControllerServer( setDefaultNumPartitions(config.numPartitions.intValue()). setSessionTimeoutNs(TimeUnit.NANOSECONDS.convert(config.brokerSessionTimeoutMs.longValue(), TimeUnit.MILLISECONDS)). - setSnapshotMaxNewRecordBytes(config.metadataSnapshotMaxNewRecordBytes). - setSnapshotMaxIntervalMs(config.metadataSnapshotMaxIntervalMs). setLeaderImbalanceCheckIntervalNs(leaderImbalanceCheckIntervalNs). setMaxIdleIntervalNs(maxIdleIntervalNs). setMetrics(sharedServer.controllerMetrics). diff --git a/core/src/main/scala/kafka/server/SharedServer.scala b/core/src/main/scala/kafka/server/SharedServer.scala index 8b647e7464fc3..826598ed2801d 100644 --- a/core/src/main/scala/kafka/server/SharedServer.scala +++ b/core/src/main/scala/kafka/server/SharedServer.scala @@ -25,6 +25,8 @@ import kafka.utils.{CoreUtils, Logging} import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.utils.{AppInfoParser, LogContext, Time} import org.apache.kafka.controller.QuorumControllerMetrics +import org.apache.kafka.image.loader.MetadataLoader +import org.apache.kafka.image.publisher.{SnapshotEmitter, SnapshotGenerator} import org.apache.kafka.metadata.MetadataRecordSerde import org.apache.kafka.raft.RaftConfig.AddressSpec import org.apache.kafka.server.common.ApiMessageAndVersion @@ -32,7 +34,9 @@ import org.apache.kafka.server.fault.{FaultHandler, LoggingFaultHandler, Process import org.apache.kafka.server.metrics.KafkaYammerMetrics import java.util +import java.util.Collections import java.util.concurrent.CompletableFuture +import java.util.concurrent.atomic.AtomicReference /** @@ -97,6 +101,10 @@ class SharedServer( @volatile var raftManager: KafkaRaftManager[ApiMessageAndVersion] = _ @volatile var brokerMetrics: BrokerServerMetrics = _ @volatile var controllerMetrics: QuorumControllerMetrics = _ + @volatile var loader: MetadataLoader = _ + val snapshotsDiabledReason = new AtomicReference[String](null) + @volatile var snapshotEmitter: SnapshotEmitter = _ + @volatile var snapshotGenerator: SnapshotGenerator = _ def isUsed(): Boolean = synchronized { usedByController || usedByBroker @@ -145,39 +153,48 @@ class SharedServer( /** * The fault handler to use when metadata loading fails. */ - def metadataLoaderFaultHandler: FaultHandler = faultHandlerFactory.build("metadata loading", + def metadataLoaderFaultHandler: FaultHandler = faultHandlerFactory.build( + name = "metadata loading", fatal = sharedServerConfig.processRoles.contains(ControllerRole), action = () => SharedServer.this.synchronized { if (brokerMetrics != null) brokerMetrics.metadataLoadErrorCount.getAndIncrement() if (controllerMetrics != null) controllerMetrics.incrementMetadataErrorCount() + snapshotsDiabledReason.compareAndSet(null, "metadata loading fault") }) /** * The fault handler to use when the initial broker metadata load fails. */ - def initialBrokerMetadataLoadFaultHandler: FaultHandler = faultHandlerFactory.build("initial metadata loading", + def initialBrokerMetadataLoadFaultHandler: FaultHandler = faultHandlerFactory.build( + name = "initial broker metadata loading", fatal = true, action = () => SharedServer.this.synchronized { if (brokerMetrics != null) brokerMetrics.metadataApplyErrorCount.getAndIncrement() if (controllerMetrics != null) controllerMetrics.incrementMetadataErrorCount() + snapshotsDiabledReason.compareAndSet(null, "initial broker metadata loading fault") }) /** * The fault handler to use when the QuorumController experiences a fault. */ - def quorumControllerFaultHandler: FaultHandler = faultHandlerFactory.build("quorum controller", + def quorumControllerFaultHandler: FaultHandler = faultHandlerFactory.build( + name = "quorum controller", fatal = true, - action = () => {} - ) + action = () => SharedServer.this.synchronized { + if (controllerMetrics != null) controllerMetrics.incrementMetadataErrorCount() + snapshotsDiabledReason.compareAndSet(null, "quorum controller fault") + }) /** * The fault handler to use when metadata cannot be published. */ - def metadataPublishingFaultHandler: FaultHandler = faultHandlerFactory.build("metadata publishing", + def metadataPublishingFaultHandler: FaultHandler = faultHandlerFactory.build( + name = "metadata publishing", fatal = false, action = () => SharedServer.this.synchronized { if (brokerMetrics != null) brokerMetrics.metadataApplyErrorCount.getAndIncrement() if (controllerMetrics != null) controllerMetrics.incrementMetadataErrorCount() + // Note: snapshot generation does not need to be disabled for a publishing fault. }) private def start(): Unit = synchronized { @@ -210,6 +227,40 @@ class SharedServer( threadNamePrefix, controllerQuorumVotersFuture) raftManager.startup() + + if (sharedServerConfig.processRoles.contains(ControllerRole) && + !sharedServerConfig.processRoles.contains(BrokerRole)) { + val loaderBuilder = new MetadataLoader.Builder(). + setNodeId(metaProps.nodeId). + setTime(time). + setThreadNamePrefix(threadNamePrefix.getOrElse("")). + setFaultHandler(metadataLoaderFaultHandler) + if (brokerMetrics != null) { + loaderBuilder.setMetadataLoaderMetrics(brokerMetrics) + } + loader = loaderBuilder.build() + snapshotEmitter = new SnapshotEmitter.Builder(). + setNodeId(metaProps.nodeId). + setRaftClient(raftManager.client). + build() + snapshotGenerator = new SnapshotGenerator.Builder(snapshotEmitter). + setNodeId(metaProps.nodeId). + setTime(time). + setFaultHandler(metadataPublishingFaultHandler). + setMaxBytesSinceLastSnapshot(sharedServerConfig.metadataSnapshotMaxNewRecordBytes). + setMaxTimeSinceLastSnapshotNs(sharedServerConfig.metadataSnapshotMaxIntervalMs). + setDisabledReason(snapshotsDiabledReason). + build() + raftManager.register(loader) + try { + loader.installPublishers(Collections.singletonList(snapshotGenerator)) + } catch { + case t: Throwable => { + error("Unable to install metadata publishers", t) + throw new RuntimeException("Unable to install metadata publishers.", t) + } + } + } debug("Completed SharedServer startup.") started = true } catch { @@ -235,6 +286,20 @@ class SharedServer( debug("SharedServer is not running.") } else { info("Stopping SharedServer") + if (loader != null) { + CoreUtils.swallow(loader.beginShutdown(), this) + } + if (snapshotGenerator != null) { + CoreUtils.swallow(snapshotGenerator.beginShutdown(), this) + } + if (loader != null) { + CoreUtils.swallow(loader.close(), this) + loader = null + } + if (snapshotGenerator != null) { + CoreUtils.swallow(snapshotGenerator.close(), this) + snapshotGenerator = null + } if (raftManager != null) { CoreUtils.swallow(raftManager.shutdown(), this) raftManager = null diff --git a/core/src/main/scala/kafka/server/metadata/BrokerMetadataListener.scala b/core/src/main/scala/kafka/server/metadata/BrokerMetadataListener.scala index 21fc126691f0f..789ae89f049ab 100644 --- a/core/src/main/scala/kafka/server/metadata/BrokerMetadataListener.scala +++ b/core/src/main/scala/kafka/server/metadata/BrokerMetadataListener.scala @@ -18,7 +18,7 @@ package kafka.server.metadata import java.util import java.util.concurrent.atomic.AtomicBoolean -import java.util.concurrent.{CompletableFuture, TimeUnit} +import java.util.concurrent.CompletableFuture import kafka.metrics.KafkaMetricsGroup import org.apache.kafka.common.utils.{LogContext, Time} import org.apache.kafka.image.writer.{ImageWriterOptions, RecordListWriter} @@ -30,14 +30,10 @@ import org.apache.kafka.server.common.ApiMessageAndVersion import org.apache.kafka.server.fault.FaultHandler import org.apache.kafka.snapshot.SnapshotReader +import java.util.concurrent.TimeUnit.NANOSECONDS import scala.compat.java8.OptionConverters._ -object BrokerMetadataListener { - val MetadataBatchProcessingTimeUs = "MetadataBatchProcessingTimeUs" - val MetadataBatchSizes = "MetadataBatchSizes" -} - class BrokerMetadataListener( val brokerId: Int, time: Time, @@ -65,16 +61,6 @@ class BrokerMetadataListener( private val log = logContext.logger(classOf[BrokerMetadataListener]) logIdent = logContext.logPrefix() - /** - * A histogram tracking the time in microseconds it took to process batches of events. - */ - private val batchProcessingTimeHist = newHistogram(BrokerMetadataListener.MetadataBatchProcessingTimeUs) - - /** - * A histogram tracking the sizes of batches that we have processed. - */ - private val metadataBatchSizeHist = newHistogram(BrokerMetadataListener.MetadataBatchSizes) - /** * The highest metadata offset that we've seen. Written only from the event queue thread. */ @@ -293,14 +279,14 @@ class BrokerMetadataListener( } } numBytes = numBytes + batch.sizeInBytes() - metadataBatchSizeHist.update(batch.records().size()) + brokerMetrics.updateBatchSize(batch.records().size()) numBatches = numBatches + 1 } val endTimeNs = time.nanoseconds() - val elapsedUs = TimeUnit.MICROSECONDS.convert(endTimeNs - startTimeNs, TimeUnit.NANOSECONDS) - batchProcessingTimeHist.update(elapsedUs) - BatchLoadResults(numBatches, numRecords, elapsedUs, numBytes) + val elapsedNs = endTimeNs - startTimeNs + brokerMetrics.updateBatchProcessingTime(elapsedNs) + BatchLoadResults(numBatches, numRecords, NANOSECONDS.toMicros(elapsedNs), numBytes) } def startPublishing(publisher: MetadataPublisher): CompletableFuture[Void] = { @@ -367,8 +353,7 @@ class BrokerMetadataListener( publisher.publish(delta, _image) // Update the metrics since the publisher handled the lastest image - brokerMetrics.lastAppliedRecordOffset.set(_highestOffset) - brokerMetrics.lastAppliedRecordTimestamp.set(_highestTimestamp) + brokerMetrics.updateLastAppliedImageProvenance(_image.provenance()) } override def handleLeaderChange(leaderAndEpoch: LeaderAndEpoch): Unit = { @@ -376,15 +361,7 @@ class BrokerMetadataListener( } override def beginShutdown(): Unit = { - eventQueue.beginShutdown("beginShutdown", new ShutdownEvent()) - } - - class ShutdownEvent extends EventQueue.FailureLoggingEvent(log) { - override def run(): Unit = { - brokerMetrics.close() - removeMetric(BrokerMetadataListener.MetadataBatchProcessingTimeUs) - removeMetric(BrokerMetadataListener.MetadataBatchSizes) - } + eventQueue.beginShutdown("beginShutdown") } def close(): Unit = { diff --git a/core/src/main/scala/kafka/server/metadata/BrokerServerMetrics.scala b/core/src/main/scala/kafka/server/metadata/BrokerServerMetrics.scala index 3e68ae85f9232..465b10f1e54b8 100644 --- a/core/src/main/scala/kafka/server/metadata/BrokerServerMetrics.scala +++ b/core/src/main/scala/kafka/server/metadata/BrokerServerMetrics.scala @@ -17,17 +17,48 @@ package kafka.server.metadata -import java.util.concurrent.atomic.AtomicLong +import kafka.metrics.KafkaMetricsGroup + +import java.util.concurrent.atomic.{AtomicLong, AtomicReference} import org.apache.kafka.common.MetricName import org.apache.kafka.common.metrics.Gauge import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.metrics.MetricConfig +import org.apache.kafka.image.MetadataProvenance +import org.apache.kafka.image.loader.MetadataLoaderMetrics +import org.apache.kafka.server.metrics.KafkaYammerMetrics + +import java.util.concurrent.TimeUnit.NANOSECONDS -final class BrokerServerMetrics private (metrics: Metrics) extends AutoCloseable { +final class BrokerServerMetrics private ( + metrics: Metrics +) extends MetadataLoaderMetrics with KafkaMetricsGroup { import BrokerServerMetrics._ - val lastAppliedRecordOffset: AtomicLong = new AtomicLong(0) - val lastAppliedRecordTimestamp: AtomicLong = new AtomicLong(0) + private val batchProcessingTimeHistName = explicitMetricName("kafka.server", + "BrokerMetadataListener", + "MetadataBatchProcessingTimeUs", + Map.empty) + + /** + * A histogram tracking the time in microseconds it took to process batches of events. + */ + private val batchProcessingTimeHist = + KafkaYammerMetrics.defaultRegistry().newHistogram(batchProcessingTimeHistName, true) + + private val batchSizeHistName = explicitMetricName("kafka.server", + "BrokerMetadataListener", + "MetadataBatchSizes", + Map.empty) + + /** + * A histogram tracking the sizes of batches that we have processed. + */ + private val batchSizeHist = + KafkaYammerMetrics.defaultRegistry().newHistogram(batchSizeHistName, true) + + val lastAppliedImageProvenance: AtomicReference[MetadataProvenance] = + new AtomicReference[MetadataProvenance](MetadataProvenance.EMPTY) val metadataLoadErrorCount: AtomicLong = new AtomicLong(0) val metadataApplyErrorCount: AtomicLong = new AtomicLong(0) @@ -62,15 +93,15 @@ final class BrokerServerMetrics private (metrics: Metrics) extends AutoCloseable ) addMetric(metrics, lastAppliedRecordOffsetName) { _ => - lastAppliedRecordOffset.get + lastAppliedImageProvenance.get.offset() } addMetric(metrics, lastAppliedRecordTimestampName) { _ => - lastAppliedRecordTimestamp.get + lastAppliedImageProvenance.get.lastContainedLogTimeMs() } addMetric(metrics, lastAppliedRecordLagMsName) { now => - now - lastAppliedRecordTimestamp.get + now - lastAppliedImageProvenance.get.lastContainedLogTimeMs() } addMetric(metrics, metadataLoadErrorCountName) { _ => @@ -82,6 +113,8 @@ final class BrokerServerMetrics private (metrics: Metrics) extends AutoCloseable } override def close(): Unit = { + KafkaYammerMetrics.defaultRegistry().removeMetric(batchProcessingTimeHistName) + KafkaYammerMetrics.defaultRegistry().removeMetric(batchSizeHistName) List( lastAppliedRecordOffsetName, lastAppliedRecordTimestampName, @@ -90,6 +123,18 @@ final class BrokerServerMetrics private (metrics: Metrics) extends AutoCloseable metadataApplyErrorCountName ).foreach(metrics.removeMetric) } + + override def updateBatchProcessingTime(elapsedNs: Long): Unit = + batchProcessingTimeHist.update(NANOSECONDS.toMicros(elapsedNs)) + + override def updateBatchSize(size: Int): Unit = batchSizeHist.update(size) + + override def updateLastAppliedImageProvenance(provenance: MetadataProvenance): Unit = + lastAppliedImageProvenance.set(provenance) + + override def lastAppliedOffset(): Long = lastAppliedImageProvenance.get().offset() + + def lastAppliedTimestamp(): Long = lastAppliedImageProvenance.get().lastContainedLogTimeMs() } diff --git a/core/src/test/java/kafka/test/MockController.java b/core/src/test/java/kafka/test/MockController.java index ff1154d2119cd..061e19213e397 100644 --- a/core/src/test/java/kafka/test/MockController.java +++ b/core/src/test/java/kafka/test/MockController.java @@ -436,11 +436,6 @@ synchronized public CompletableFuture> createP return CompletableFuture.completedFuture(results); } - @Override - public CompletableFuture beginWritingSnapshot() { - throw new UnsupportedOperationException(); - } - @Override public void beginShutdown() { this.active = false; diff --git a/core/src/test/scala/kafka/server/metadata/BrokerServerMetricsTest.scala b/core/src/test/scala/kafka/server/metadata/BrokerServerMetricsTest.scala index ea2b439c166ae..200deed42704a 100644 --- a/core/src/test/scala/kafka/server/metadata/BrokerServerMetricsTest.scala +++ b/core/src/test/scala/kafka/server/metadata/BrokerServerMetricsTest.scala @@ -22,9 +22,11 @@ import kafka.utils.TestUtils import org.apache.kafka.common.MetricName import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.utils.MockTime +import org.apache.kafka.image.MetadataProvenance import org.junit.jupiter.api.Assertions.assertEquals import org.junit.jupiter.api.Assertions.assertTrue import org.junit.jupiter.api.Test + import scala.jdk.CollectionConverters._ final class BrokerServerMetricsTest { @@ -59,11 +61,14 @@ final class BrokerServerMetricsTest { val metrics = new Metrics() TestUtils.resource(BrokerServerMetrics(metrics)) { brokerMetrics => val offsetMetric = metrics.metrics().get(brokerMetrics.lastAppliedRecordOffsetName) - assertEquals(0, offsetMetric.metricValue.asInstanceOf[Long]) + assertEquals(-1L, offsetMetric.metricValue.asInstanceOf[Long]) // Update metric value and check val expectedValue = 1000 - brokerMetrics.lastAppliedRecordOffset.set(expectedValue) + brokerMetrics.updateLastAppliedImageProvenance(new MetadataProvenance( + expectedValue, + brokerMetrics.lastAppliedImageProvenance.get().epoch(), + brokerMetrics.lastAppliedTimestamp())); assertEquals(expectedValue, offsetMetric.metricValue.asInstanceOf[Long]) } } @@ -77,12 +82,16 @@ final class BrokerServerMetricsTest { val timestampMetric = metrics.metrics().get(brokerMetrics.lastAppliedRecordTimestampName) val lagMetric = metrics.metrics().get(brokerMetrics.lastAppliedRecordLagMsName) - assertEquals(0, timestampMetric.metricValue.asInstanceOf[Long]) - assertEquals(time.milliseconds, lagMetric.metricValue.asInstanceOf[Long]) + assertEquals(-1L, timestampMetric.metricValue.asInstanceOf[Long]) + assertEquals(time.milliseconds + 1, lagMetric.metricValue.asInstanceOf[Long]) // Update metric value and check - val timestamp = 500 - brokerMetrics.lastAppliedRecordTimestamp.set(timestamp) + val timestamp = 500L + + brokerMetrics.updateLastAppliedImageProvenance(new MetadataProvenance( + brokerMetrics.lastAppliedOffset(), + brokerMetrics.lastAppliedImageProvenance.get().epoch(), + timestamp)) assertEquals(timestamp, timestampMetric.metricValue.asInstanceOf[Long]) assertEquals(time.milliseconds - timestamp, lagMetric.metricValue.asInstanceOf[Long]) } diff --git a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala index 84d6f5a2ef93d..bd1b506e27c17 100644 --- a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala +++ b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala @@ -22,7 +22,6 @@ import java.util import java.util.Collections.{singletonList, singletonMap} import java.util.{Collections, Properties} import java.util.concurrent.ExecutionException - import kafka.integration.KafkaServerTestHarness import kafka.log.LogConfig._ import kafka.utils._ @@ -43,7 +42,7 @@ import org.apache.kafka.common.record.{CompressionType, RecordVersion} import org.apache.kafka.common.security.auth.KafkaPrincipal import org.apache.kafka.server.common.MetadataVersion.IBP_3_0_IV1 import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.api.Test +import org.junit.jupiter.api.{Test, Timeout} import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.ValueSource import org.mockito.ArgumentMatchers.{any, anyString} @@ -53,6 +52,7 @@ import scala.annotation.nowarn import scala.collection.{Map, Seq} import scala.jdk.CollectionConverters._ +@Timeout(100) class DynamicConfigChangeTest extends KafkaServerTestHarness { def generateConfigs = List(KafkaConfig.fromProps(TestUtils.createBrokerConfig(0, zkConnectOrNull))) diff --git a/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataListenerTest.scala b/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataListenerTest.scala index a589c0572ddd8..e559a6b753d18 100644 --- a/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataListenerTest.scala +++ b/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataListenerTest.scala @@ -86,8 +86,8 @@ class BrokerMetadataListenerTest { val imageRecords = listener.getImageRecords().get() assertEquals(0, imageRecords.size()) assertEquals(100L, listener.highestMetadataOffset) - assertEquals(0L, metrics.lastAppliedRecordOffset.get) - assertEquals(0L, metrics.lastAppliedRecordTimestamp.get) + assertEquals(-1L, metrics.lastAppliedOffset()) + assertEquals(-1L, metrics.lastAppliedTimestamp()) assertEquals(0L, metrics.metadataLoadErrorCount.get) assertEquals(0L, metrics.metadataApplyErrorCount.get) @@ -121,8 +121,8 @@ class BrokerMetadataListenerTest { override def publishedOffset: Long = -1 }).get() - assertEquals(fencedLastOffset, metrics.lastAppliedRecordOffset.get) - assertEquals(fencedTimestamp, metrics.lastAppliedRecordTimestamp.get) + assertEquals(fencedLastOffset, metrics.lastAppliedOffset()) + assertEquals(fencedTimestamp, metrics.lastAppliedTimestamp()) assertEquals(0L, metrics.metadataLoadErrorCount.get) assertEquals(0L, metrics.metadataApplyErrorCount.get) } finally { diff --git a/metadata/src/main/java/org/apache/kafka/controller/AclControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/AclControlManager.java index d3fc0fe76edd8..313927fc5370b 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/AclControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/AclControlManager.java @@ -28,7 +28,6 @@ import org.apache.kafka.common.requests.ApiError; import org.apache.kafka.metadata.authorizer.ClusterMetadataAuthorizer; import org.apache.kafka.metadata.authorizer.StandardAcl; -import org.apache.kafka.metadata.authorizer.StandardAclRecordIterator; import org.apache.kafka.metadata.authorizer.StandardAclWithId; import org.apache.kafka.raft.OffsetAndEpoch; import org.apache.kafka.server.authorizer.AclCreateResult; @@ -42,7 +41,6 @@ import java.util.ArrayList; import java.util.Collections; import java.util.HashSet; -import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Map.Entry; @@ -216,20 +214,4 @@ public void replay(RemoveAccessControlEntryRecord record, Map idToAcl() { return Collections.unmodifiableMap(idToAcl); } - - Iterator> iterator(long epoch) { - Iterator> iterator = idToAcl.entrySet(epoch).iterator(); - return new StandardAclRecordIterator(new Iterator() { - @Override - public boolean hasNext() { - return iterator.hasNext(); - } - - @Override - public StandardAclWithId next() { - Entry entry = iterator.next(); - return new StandardAclWithId(entry.getKey(), entry.getValue()); - } - }); - } } diff --git a/metadata/src/main/java/org/apache/kafka/controller/ClientQuotaControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ClientQuotaControlManager.java index 504994b3a5b5c..b859bbfd65d20 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ClientQuotaControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ClientQuotaControlManager.java @@ -36,11 +36,9 @@ import java.util.Collection; import java.util.Collections; import java.util.HashMap; -import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Map.Entry; -import java.util.NoSuchElementException; import java.util.Objects; import java.util.function.Supplier; import java.util.stream.Collectors; @@ -288,44 +286,4 @@ private ApiError validateEntity(ClientQuotaEntity entity, Map va return ApiError.NONE; } - - class ClientQuotaControlIterator implements Iterator> { - private final long epoch; - private final Iterator>> iterator; - - ClientQuotaControlIterator(long epoch) { - this.epoch = epoch; - this.iterator = clientQuotaData.entrySet(epoch).iterator(); - } - - @Override - public boolean hasNext() { - return iterator.hasNext(); - } - - @Override - public List next() { - if (!hasNext()) throw new NoSuchElementException(); - Entry> entry = iterator.next(); - ClientQuotaEntity entity = entry.getKey(); - List records = new ArrayList<>(); - for (Entry quotaEntry : entry.getValue().entrySet(epoch)) { - ClientQuotaRecord record = new ClientQuotaRecord(); - for (Entry entityEntry : entity.entries().entrySet()) { - record.entity().add(new EntityData(). - setEntityType(entityEntry.getKey()). - setEntityName(entityEntry.getValue())); - } - record.setKey(quotaEntry.getKey()); - record.setValue(quotaEntry.getValue()); - record.setRemove(false); - records.add(new ApiMessageAndVersion(record, (short) 0)); - } - return records; - } - } - - ClientQuotaControlIterator iterator(long epoch) { - return new ClientQuotaControlIterator(epoch); - } } diff --git a/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java index 95aae773fb445..ba6c0e1d1c215 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java @@ -29,9 +29,7 @@ import org.apache.kafka.common.metadata.FenceBrokerRecord; import org.apache.kafka.common.metadata.RegisterBrokerRecord; import org.apache.kafka.common.metadata.RegisterBrokerRecord.BrokerEndpoint; -import org.apache.kafka.common.metadata.RegisterBrokerRecord.BrokerEndpointCollection; import org.apache.kafka.common.metadata.RegisterBrokerRecord.BrokerFeature; -import org.apache.kafka.common.metadata.RegisterBrokerRecord.BrokerFeatureCollection; import org.apache.kafka.common.metadata.UnfenceBrokerRecord; import org.apache.kafka.common.metadata.UnregisterBrokerRecord; import org.apache.kafka.common.protocol.ApiMessage; @@ -59,7 +57,6 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; -import java.util.NoSuchElementException; import java.util.Optional; import java.util.OptionalLong; import java.util.Random; @@ -68,7 +65,6 @@ import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; -import static java.util.Collections.singletonList; import static java.util.concurrent.TimeUnit.NANOSECONDS; @@ -654,60 +650,4 @@ public void addReadyBrokersFuture(CompletableFuture future, int minBrokers readyBrokersFuture = Optional.empty(); } } - - class ClusterControlIterator implements Iterator> { - private final Iterator> iterator; - private final MetadataVersion metadataVersion; - - ClusterControlIterator(long epoch) { - this.iterator = brokerRegistrations.entrySet(epoch).iterator(); - this.metadataVersion = featureControl.metadataVersion(); - } - - @Override - public boolean hasNext() { - return iterator.hasNext(); - } - - @Override - public List next() { - if (!hasNext()) throw new NoSuchElementException(); - Entry entry = iterator.next(); - int brokerId = entry.getKey(); - BrokerRegistration registration = entry.getValue(); - BrokerEndpointCollection endpoints = new BrokerEndpointCollection(); - for (Entry endpointEntry : registration.listeners().entrySet()) { - endpoints.add(new BrokerEndpoint().setName(endpointEntry.getKey()). - setHost(endpointEntry.getValue().host()). - setPort(endpointEntry.getValue().port()). - setSecurityProtocol(endpointEntry.getValue().securityProtocol().id)); - } - BrokerFeatureCollection features = new BrokerFeatureCollection(); - for (Entry featureEntry : registration.supportedFeatures().entrySet()) { - features.add(new BrokerFeature().setName(featureEntry.getKey()). - setMaxSupportedVersion(featureEntry.getValue().max()). - setMinSupportedVersion(featureEntry.getValue().min())); - } - RegisterBrokerRecord record = new RegisterBrokerRecord(). - setBrokerId(brokerId). - setIncarnationId(registration.incarnationId()). - setBrokerEpoch(registration.epoch()). - setEndPoints(endpoints). - setFeatures(features). - setRack(registration.rack().orElse(null)). - setFenced(registration.fenced()); - if (metadataVersion.isInControlledShutdownStateSupported()) { - record.setInControlledShutdown(registration.inControlledShutdown()); - } - if (metadataVersion.isMigrationSupported()) { - record.setIsMigratingZkBroker(registration.isMigratingZkBroker()); - } - return singletonList(new ApiMessageAndVersion(record, - metadataVersion.registerBrokerRecordVersion())); - } - } - - ClusterControlIterator iterator(long epoch) { - return new ClusterControlIterator(epoch); - } } diff --git a/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java index d569ae1a6cab7..b5d71230cf473 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java @@ -43,7 +43,6 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; -import java.util.NoSuchElementException; import java.util.Objects; import java.util.Optional; import java.util.function.Consumer; @@ -474,39 +473,4 @@ Map currentControllerConfig() { Map result = configData.get(currentController); return (result == null) ? Collections.emptyMap() : result; } - - class ConfigurationControlIterator implements Iterator> { - private final long epoch; - private final Iterator>> iterator; - - ConfigurationControlIterator(long epoch) { - this.epoch = epoch; - this.iterator = configData.entrySet(epoch).iterator(); - } - - @Override - public boolean hasNext() { - return iterator.hasNext(); - } - - @Override - public List next() { - if (!hasNext()) throw new NoSuchElementException(); - List records = new ArrayList<>(); - Entry> entry = iterator.next(); - ConfigResource resource = entry.getKey(); - for (Entry configEntry : entry.getValue().entrySet(epoch)) { - records.add(new ApiMessageAndVersion(new ConfigRecord(). - setResourceName(resource.name()). - setResourceType(resource.type().id()). - setName(configEntry.getKey()). - setValue(configEntry.getValue()), (short) 0)); - } - return records; - } - } - - ConfigurationControlIterator iterator(long epoch) { - return new ConfigurationControlIterator(epoch); - } } diff --git a/metadata/src/main/java/org/apache/kafka/controller/Controller.java b/metadata/src/main/java/org/apache/kafka/controller/Controller.java index 3622fe225dc8f..ed6c523753304 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/Controller.java +++ b/metadata/src/main/java/org/apache/kafka/controller/Controller.java @@ -316,14 +316,6 @@ CompletableFuture updateFeatures( UpdateFeaturesRequestData request ); - /** - * Begin writing a controller snapshot. If there was already an ongoing snapshot, it - * simply returns information about that snapshot rather than starting a new one. - * - * @return A future yielding the epoch of the snapshot. - */ - CompletableFuture beginWritingSnapshot(); - /** * Create partitions on certain topics. * diff --git a/metadata/src/main/java/org/apache/kafka/controller/FeatureControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/FeatureControlManager.java index b3758586cb88a..7b2a9308e96b7 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/FeatureControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/FeatureControlManager.java @@ -20,11 +20,9 @@ import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; -import java.util.Iterator; import java.util.List; import java.util.Map.Entry; import java.util.Map; -import java.util.NoSuchElementException; import java.util.Optional; import java.util.TreeMap; import java.util.function.Consumer; @@ -306,48 +304,6 @@ public void replay(FeatureLevelRecord record) { } } - class FeatureControlIterator implements Iterator> { - private final Iterator> iterator; - private final MetadataVersion metadataVersion; - private boolean wroteVersion = false; - - FeatureControlIterator(long epoch) { - this.iterator = finalizedVersions.entrySet(epoch).iterator(); - this.metadataVersion = FeatureControlManager.this.metadataVersion.get(epoch); - } - - @Override - public boolean hasNext() { - return needsWriteMetadataVersion() || iterator.hasNext(); - } - - private boolean needsWriteMetadataVersion() { - return !wroteVersion && metadataVersion.isAtLeast(minimumBootstrapVersion); - } - - @Override - public List next() { - // Write the metadata.version first - if (needsWriteMetadataVersion()) { - wroteVersion = true; - return Collections.singletonList(new ApiMessageAndVersion(new FeatureLevelRecord() - .setName(MetadataVersion.FEATURE_NAME) - .setFeatureLevel(metadataVersion.featureLevel()), FEATURE_LEVEL_RECORD.lowestSupportedVersion())); - } - - // Then write the rest of the features - if (!iterator.hasNext()) throw new NoSuchElementException(); - Entry entry = iterator.next(); - return Collections.singletonList(new ApiMessageAndVersion(new FeatureLevelRecord() - .setName(entry.getKey()) - .setFeatureLevel(entry.getValue()), (short) 0)); - } - } - - FeatureControlIterator iterator(long epoch) { - return new FeatureControlIterator(epoch); - } - boolean isControllerId(int nodeId) { return quorumFeatures.isControllerId(nodeId); } diff --git a/metadata/src/main/java/org/apache/kafka/controller/LogReplayTracker.java b/metadata/src/main/java/org/apache/kafka/controller/LogReplayTracker.java index 2e29a2a52f9b0..41f881245fcd1 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/LogReplayTracker.java +++ b/metadata/src/main/java/org/apache/kafka/controller/LogReplayTracker.java @@ -24,8 +24,7 @@ /** * The LogReplayTracker manages state associated with replaying the metadata log, such as whether - * we have seen any records and whether we have seen any metadata version records. It is accessed - * solely from the quorum controller thread. + * we have seen any records. It is accessed solely from the quorum controller thread. */ public class LogReplayTracker { public static class Builder { diff --git a/metadata/src/main/java/org/apache/kafka/controller/ProducerIdControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ProducerIdControlManager.java index 178ef46bdb55e..47e4e1b430fdb 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ProducerIdControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ProducerIdControlManager.java @@ -25,14 +25,10 @@ import org.apache.kafka.timeline.TimelineLong; import org.apache.kafka.timeline.TimelineObject; -import java.util.ArrayList; import java.util.Collections; -import java.util.Iterator; -import java.util.List; public class ProducerIdControlManager { - private final ClusterControlManager clusterControlManager; private final TimelineObject nextProducerBlock; private final TimelineLong brokerEpoch; @@ -62,6 +58,11 @@ ControllerResult generateNextProducerId(int brokerId, long bro return ControllerResult.of(Collections.singletonList(new ApiMessageAndVersion(record, (short) 0)), block); } + // VisibleForTesting + ProducerIdsBlock nextProducerBlock() { + return nextProducerBlock.get(); + } + void replay(ProducerIdsRecord record) { long currentNextProducerId = nextProducerBlock.get().firstProducerId(); if (record.nextProducerId() <= currentNextProducerId) { @@ -72,19 +73,4 @@ void replay(ProducerIdsRecord record) { brokerEpoch.set(record.brokerEpoch()); } } - - Iterator> iterator(long epoch) { - List records = new ArrayList<>(1); - - ProducerIdsBlock producerIdBlock = nextProducerBlock.get(epoch); - if (producerIdBlock.firstProducerId() > 0) { - records.add(new ApiMessageAndVersion( - new ProducerIdsRecord() - .setNextProducerId(producerIdBlock.firstProducerId()) - .setBrokerId(producerIdBlock.assignedBrokerId()) - .setBrokerEpoch(brokerEpoch.get(epoch)), - (short) 0)); - } - return Collections.singleton(records).iterator(); - } } diff --git a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java index 6716044bef5d1..642d6976dba8c 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java +++ b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java @@ -72,7 +72,6 @@ import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; -import org.apache.kafka.controller.SnapshotGenerator.Section; import org.apache.kafka.metadata.BrokerHeartbeatReply; import org.apache.kafka.metadata.BrokerRegistrationReply; import org.apache.kafka.metadata.FinalizedControllerFeatures; @@ -89,7 +88,6 @@ import org.apache.kafka.raft.LeaderAndEpoch; import org.apache.kafka.raft.OffsetAndEpoch; import org.apache.kafka.raft.RaftClient; -import org.apache.kafka.metadata.util.SnapshotReason; import org.apache.kafka.server.authorizer.AclCreateResult; import org.apache.kafka.server.authorizer.AclDeleteResult; import org.apache.kafka.server.common.ApiMessageAndVersion; @@ -98,7 +96,6 @@ import org.apache.kafka.server.policy.AlterConfigPolicy; import org.apache.kafka.server.policy.CreateTopicPolicy; import org.apache.kafka.snapshot.SnapshotReader; -import org.apache.kafka.snapshot.SnapshotWriter; import org.apache.kafka.timeline.SnapshotRegistry; import org.slf4j.Logger; @@ -107,7 +104,6 @@ import java.util.Collection; import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map.Entry; import java.util.Map; @@ -168,8 +164,6 @@ static public class Builder { private short defaultReplicationFactor = 3; private int defaultNumPartitions = 1; private ReplicaPlacer replicaPlacer = new StripedReplicaPlacer(new Random()); - private long snapshotMaxNewRecordBytes = Long.MAX_VALUE; - private long snapshotMaxIntervalMs = 0; private OptionalLong leaderImbalanceCheckIntervalNs = OptionalLong.empty(); private OptionalLong maxIdleIntervalNs = OptionalLong.empty(); private long sessionTimeoutNs = ClusterControlManager.DEFAULT_SESSION_TIMEOUT_NS; @@ -242,16 +236,6 @@ public Builder setReplicaPlacer(ReplicaPlacer replicaPlacer) { return this; } - public Builder setSnapshotMaxNewRecordBytes(long value) { - this.snapshotMaxNewRecordBytes = value; - return this; - } - - public Builder setSnapshotMaxIntervalMs(long value) { - this.snapshotMaxIntervalMs = value; - return this; - } - public Builder setLeaderImbalanceCheckIntervalNs(OptionalLong value) { this.leaderImbalanceCheckIntervalNs = value; return this; @@ -351,8 +335,6 @@ public QuorumController build() throws Exception { defaultReplicationFactor, defaultNumPartitions, replicaPlacer, - snapshotMaxNewRecordBytes, - snapshotMaxIntervalMs, leaderImbalanceCheckIntervalNs, maxIdleIntervalNs, sessionTimeoutNs, @@ -521,132 +503,6 @@ private void appendControlEvent(String name, Runnable handler) { queue.append(event); } - private static final String GENERATE_SNAPSHOT = "generateSnapshot"; - - private static final int MAX_BATCHES_PER_GENERATE_CALL = 10; - - class SnapshotGeneratorManager implements Runnable { - private SnapshotGenerator generator = null; - - void createSnapshotGenerator(long committedOffset, int committedEpoch, long committedTimestamp) { - if (snapshotInProgress()) { - throw new IllegalStateException("Snapshot generator already exists"); - } - if (!snapshotRegistry.hasSnapshot(committedOffset)) { - throw new IllegalStateException( - String.format( - "Cannot generate a snapshot at committed offset %d because it does not exists in the snapshot registry.", - committedOffset - ) - ); - } - - OffsetAndEpoch snapshotId = new OffsetAndEpoch( - committedOffset + 1, - committedEpoch - ); - - Optional> writer = raftClient.createSnapshot( - snapshotId, - committedTimestamp - ); - if (writer.isPresent()) { - generator = new SnapshotGenerator( - logContext, - writer.get(), - MAX_BATCHES_PER_GENERATE_CALL, - Arrays.asList( - new Section("features", featureControl.iterator(committedOffset)), - new Section("cluster", clusterControl.iterator(committedOffset)), - new Section("replication", replicationControl.iterator(committedOffset)), - new Section("configuration", configurationControl.iterator(committedOffset)), - new Section("clientQuotas", clientQuotaControlManager.iterator(committedOffset)), - new Section("producerIds", producerIdControlManager.iterator(committedOffset)), - new Section("acls", aclControlManager.iterator(committedOffset)) - ) - ); - reschedule(0); - } else { - log.info( - "Skipping generation of snapshot for committed offset {} and epoch {} since it already exists", - committedOffset, - committedEpoch - ); - } - } - - void cancel() { - if (!snapshotInProgress()) return; - log.error("Cancelling snapshot {}", generator.lastContainedLogOffset()); - generator.writer().close(); - generator = null; - - // Delete every in-memory snapshot up to the committed offset. They are not needed since this - // snapshot generation was canceled. - snapshotRegistry.deleteSnapshotsUpTo(lastCommittedOffset); - - queue.cancelDeferred(GENERATE_SNAPSHOT); - } - - void reschedule(long delayNs) { - ControllerEvent event = new ControllerEvent(GENERATE_SNAPSHOT, this); - queue.scheduleDeferred(event.name, - new EarliestDeadlineFunction(time.nanoseconds() + delayNs), event); - } - - void handleSnapshotFinished(Optional exception) { - if (exception.isPresent()) { - log.error("Error while generating snapshot {}", generator.lastContainedLogOffset(), exception.get()); - } else { - log.info("Finished generating snapshot {}.", generator.lastContainedLogOffset()); - } - - generator.writer().close(); - generator = null; - - // Delete every in-memory snapshot up to the committed offset. They are not needed since this - // snapshot generation finished. - snapshotRegistry.deleteSnapshotsUpTo(lastCommittedOffset); - - // The snapshot counters for size-based and time-based snapshots could have changed to cause a new - // snapshot to get generated. - maybeGenerateSnapshot(); - } - - @Override - public void run() { - if (!snapshotInProgress()) { - log.debug("No snapshot is in progress because it was previously canceled"); - return; - } - - OptionalLong nextDelay; - try { - nextDelay = generator.generateBatches(); - } catch (Exception e) { - handleSnapshotFinished(Optional.of(e)); - return; - } - - if (nextDelay.isPresent()) { - reschedule(nextDelay.getAsLong()); - } else { - handleSnapshotFinished(Optional.empty()); - } - } - - OptionalLong snapshotLastOffsetFromLog() { - if (!snapshotInProgress()) { - return OptionalLong.empty(); - } - return OptionalLong.of(generator.lastContainedLogOffset()); - } - - public boolean snapshotInProgress() { - return generator != null; - } - } - /** * A controller event that reads the committed internal state in order to expose it * to an API. @@ -985,12 +841,8 @@ public void handleCommit(BatchReader reader) { // Complete any events in the purgatory that were waiting for this offset. purgatory.completeUpTo(offset); - // Delete all the in-memory snapshots that are no longer needed. - // - // If the active controller has a snapshot in progress, it needs to keep that in-memory - // snapshot. Otherwise, the active controller can delete up to the current committed offset. - snapshotRegistry.deleteSnapshotsUpTo( - snapshotGeneratorManager.snapshotLastOffsetFromLog().orElse(offset)); + // The active controller can delete up to the current committed offset. + snapshotRegistry.deleteSnapshotsUpTo(offset); } else { // If the controller is a standby, replay the records that were // created by the active controller. @@ -1022,8 +874,7 @@ public void handleCommit(BatchReader reader) { updateLastCommittedState( offset, epoch, - batch.appendTimestamp(), - committedBytesSinceLastSnapshot + batch.sizeInBytes() + batch.appendTimestamp() ); if (offset >= raftClient.latestSnapshotId().map(OffsetAndEpoch::offset).orElse(0L)) { @@ -1033,8 +884,6 @@ public void handleCommit(BatchReader reader) { ); } } - - maybeGenerateSnapshot(); } finally { reader.close(); } @@ -1086,12 +935,9 @@ public void handleSnapshot(SnapshotReader reader) { i++; } } - updateLastCommittedState( - reader.lastContainedLogOffset(), + updateLastCommittedState(reader.lastContainedLogOffset(), reader.lastContainedLogEpoch(), - reader.lastContainedLogTimestamp(), - 0 - ); + reader.lastContainedLogTimestamp()); snapshotRegistry.getOrCreateSnapshot(lastCommittedOffset); authorizer.ifPresent(a -> a.loadSnapshot(aclControlManager.idToAcl())); } finally { @@ -1247,13 +1093,11 @@ public void processBatchEndOffset(long offset) { private void updateLastCommittedState( long offset, int epoch, - long timestamp, - long bytesSinceLastSnapshot + long timestamp ) { lastCommittedOffset = offset; lastCommittedEpoch = epoch; lastCommittedTimestamp = timestamp; - committedBytesSinceLastSnapshot = bytesSinceLastSnapshot; controllerMetrics.setLastCommittedRecordOffset(offset); if (!isActiveController()) { @@ -1426,38 +1270,6 @@ private void cancelNextWriteNoOpRecord() { queue.cancelDeferred(WRITE_NO_OP_RECORD); } - private static final String MAYBE_GENERATE_SNAPSHOT = "maybeGenerateSnapshot"; - - private void maybeScheduleNextGenerateSnapshot() { - if (!generateSnapshotScheduled) { - long now = time.milliseconds(); - long delayMs = Math.min( - 0, - snapshotMaxIntervalMs + oldestNonSnapshottedTimestamp - now - ); - - log.debug( - "Scheduling write event for {} because snapshotMaxIntervalMs ({}), " + - "oldestNonSnapshottedTimestamp ({}) and now ({})", - MAYBE_GENERATE_SNAPSHOT, - snapshotMaxIntervalMs, - oldestNonSnapshottedTimestamp, - now - ); - - ControllerEvent event = new ControllerEvent(MAYBE_GENERATE_SNAPSHOT, this::maybeGenerateSnapshot); - - long scheduleNs = time.nanoseconds() + TimeUnit.MILLISECONDS.toNanos(delayMs); - queue.scheduleDeferred(MAYBE_GENERATE_SNAPSHOT, new EarliestDeadlineFunction(scheduleNs), event); - generateSnapshotScheduled = true; - } - } - - private void cancelNextGenerateSnapshot() { - queue.cancelDeferred(MAYBE_GENERATE_SNAPSHOT); - generateSnapshotScheduled = false; - } - private void handleFeatureControlChange() { // The feature control maybe have changed. On the active controller cancel or schedule noop // record writes accordingly. @@ -1536,72 +1348,13 @@ private void replay(ApiMessage message, Optional snapshotId, lon } } - private void maybeGenerateSnapshot() { - if (snapshotGeneratorManager.snapshotInProgress()) { - /* Skip snapshot generation if there is a snaphshot in progress. - * - * When the in-progress snapshot completes it will call this method to check if the controller should - * generate another snapshot due to any of the reasons supported by this method. - */ - } else { - Set snapshotReasons = new HashSet<>(); - // Check if a snapshot should be generated because of committed bytes - if (committedBytesSinceLastSnapshot >= snapshotMaxNewRecordBytes) { - snapshotReasons.add( - SnapshotReason.maxBytesExceeded(committedBytesSinceLastSnapshot, snapshotMaxNewRecordBytes) - ); - } - - // Check if a snapshot should be generated because of committed append times - if (snapshotMaxIntervalMs > 0) { - // Time base snasphots are enabled - long snapshotIntervalMs = time.milliseconds() - oldestNonSnapshottedTimestamp; - if (snapshotIntervalMs >= snapshotMaxIntervalMs) { - snapshotReasons.add(SnapshotReason.maxIntervalExceeded(snapshotIntervalMs, snapshotMaxIntervalMs)); - } else { - maybeScheduleNextGenerateSnapshot(); - } - } - - if (!snapshotReasons.isEmpty()) { - if (!isActiveController()) { - // The inactive controllers only create an in-memory snapshot when generating a snapshot. This is - // unlike the active controller which creates in-memory snapshots every time an uncommitted batch - // gets replayed. - snapshotRegistry.getOrCreateSnapshot(lastCommittedOffset); - } - - log.info( - "Generating a snapshot that includes (epoch={}, offset={}) because: {}", - lastCommittedEpoch, - lastCommittedOffset, - SnapshotReason.stringFromReasons(snapshotReasons) - ); - - snapshotGeneratorManager.createSnapshotGenerator( - lastCommittedOffset, - lastCommittedEpoch, - lastCommittedTimestamp - ); - - // Reset all of the snapshot counters - committedBytesSinceLastSnapshot = 0; - oldestNonSnapshottedTimestamp = Long.MAX_VALUE; - - // Starting a snapshot invalidates any scheduled snapshot generation - cancelNextGenerateSnapshot(); - } - } - } - /** * Clear all data structures and reset all KRaft state. */ private void resetToEmptyState() { - snapshotGeneratorManager.cancel(); snapshotRegistry.reset(); - updateLastCommittedState(-1, -1, -1, 0); + updateLastCommittedState(-1, -1, -1); } /** @@ -1715,11 +1468,6 @@ private void resetToEmptyState() { */ private final LogReplayTracker logReplayTracker; - /** - * Manages generating controller snapshots. - */ - private final SnapshotGeneratorManager snapshotGeneratorManager = new SnapshotGeneratorManager(); - /** * The interface that we use to mutate the Raft log. */ @@ -1766,21 +1514,6 @@ private void resetToEmptyState() { */ private long writeOffset; - /** - * Maximum number of bytes processed through handling commits before generating a snapshot. - */ - private final long snapshotMaxNewRecordBytes; - - /** - * Number of bytes processed through handling commits since the last snapshot was generated. - */ - private long committedBytesSinceLastSnapshot = 0; - - /** - * Maximum amount of to wait for a record in the log to get included in a snapshot. - */ - private final long snapshotMaxIntervalMs; - /** * Timestamp for the oldest record that was committed but not included in a snapshot. */ @@ -1843,8 +1576,6 @@ private QuorumController( short defaultReplicationFactor, int defaultNumPartitions, ReplicaPlacer replicaPlacer, - long snapshotMaxNewRecordBytes, - long snapshotMaxIntervalMs, OptionalLong leaderImbalanceCheckIntervalNs, OptionalLong maxIdleIntervalNs, long sessionTimeoutNs, @@ -1903,8 +1634,6 @@ private QuorumController( setZkMigrationEnabled(zkMigrationEnabled). build(); this.producerIdControlManager = new ProducerIdControlManager(clusterControl, snapshotRegistry); - this.snapshotMaxNewRecordBytes = snapshotMaxNewRecordBytes; - this.snapshotMaxIntervalMs = snapshotMaxIntervalMs; this.leaderImbalanceCheckIntervalNs = leaderImbalanceCheckIntervalNs; this.maxIdleIntervalNs = maxIdleIntervalNs; this.replicationControl = new ReplicationControlManager.Builder(). @@ -2243,28 +1972,6 @@ public CompletableFuture> createPartitions( }); } - @Override - public CompletableFuture beginWritingSnapshot() { - CompletableFuture future = new CompletableFuture<>(); - appendControlEvent("beginWritingSnapshot", () -> { - if (!snapshotGeneratorManager.snapshotInProgress()) { - log.info( - "Generating a snapshot that includes (epoch={}, offset={}) because, {}.", - lastCommittedEpoch, - lastCommittedOffset, - SnapshotReason.UNKNOWN - ); - snapshotGeneratorManager.createSnapshotGenerator( - lastCommittedOffset, - lastCommittedEpoch, - lastCommittedTimestamp - ); - } - future.complete(snapshotGeneratorManager.generator.lastContainedLogOffset()); - }); - return future; - } - @Override public CompletableFuture> createAcls( ControllerRequestContext context, diff --git a/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java index dbe23218d48c5..5fd9af85e06ac 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java @@ -108,7 +108,6 @@ import java.util.ListIterator; import java.util.Map.Entry; import java.util.Map; -import java.util.NoSuchElementException; import java.util.Optional; import java.util.OptionalInt; import java.util.Set; @@ -1911,39 +1910,6 @@ private void listReassigningTopic(ListPartitionReassignmentsResponseData respons setReplicas(Replicas.toList(partition.replicas))); } - class ReplicationControlIterator implements Iterator> { - private final long epoch; - private final Iterator iterator; - - ReplicationControlIterator(long epoch) { - this.epoch = epoch; - this.iterator = topics.values(epoch).iterator(); - } - - @Override - public boolean hasNext() { - return iterator.hasNext(); - } - - @Override - public List next() { - if (!hasNext()) throw new NoSuchElementException(); - TopicControlInfo topic = iterator.next(); - List records = new ArrayList<>(); - records.add(new ApiMessageAndVersion(new TopicRecord(). - setName(topic.name). - setTopicId(topic.id), (short) 0)); - for (Entry entry : topic.parts.entrySet(epoch)) { - records.add(entry.getValue().toRecord(topic.id, entry.getKey())); - } - return records; - } - } - - ReplicationControlIterator iterator(long epoch) { - return new ReplicationControlIterator(epoch); - } - private static final class IneligibleReplica { private final int replicaId; private final String reason; diff --git a/metadata/src/main/java/org/apache/kafka/controller/SnapshotGenerator.java b/metadata/src/main/java/org/apache/kafka/controller/SnapshotGenerator.java deleted file mode 100644 index d34696ef44f11..0000000000000 --- a/metadata/src/main/java/org/apache/kafka/controller/SnapshotGenerator.java +++ /dev/null @@ -1,132 +0,0 @@ -/* - * 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 java.util.Collections; -import java.util.Iterator; -import java.util.List; -import java.util.OptionalLong; - -import org.apache.kafka.common.utils.LogContext; -import org.apache.kafka.server.common.ApiMessageAndVersion; -import org.apache.kafka.snapshot.SnapshotWriter; -import org.slf4j.Logger; - - -final class SnapshotGenerator { - static class Section { - private final String name; - private final Iterator> iterator; - - Section(String name, Iterator> iterator) { - this.name = name; - this.iterator = iterator; - } - - String name() { - return name; - } - - Iterator> iterator() { - return iterator; - } - } - - private final Logger log; - private final SnapshotWriter writer; - private final int maxBatchesPerGenerateCall; - private final List
sections; - private final Iterator
sectionIterator; - private Iterator> batchIterator; - private List batch; - private Section section; - private long numRecords; - - SnapshotGenerator(LogContext logContext, - SnapshotWriter writer, - int maxBatchesPerGenerateCall, - List
sections) { - this.log = logContext.logger(SnapshotGenerator.class); - this.writer = writer; - this.maxBatchesPerGenerateCall = maxBatchesPerGenerateCall; - this.sections = sections; - this.sectionIterator = this.sections.iterator(); - this.batchIterator = Collections.emptyIterator(); - this.batch = null; - this.section = null; - this.numRecords = 0; - } - - /** - * Returns the last offset from the log that will be included in the snapshot. - */ - long lastContainedLogOffset() { - return writer.lastContainedLogOffset(); - } - - SnapshotWriter writer() { - return writer; - } - - /** - * Generate and write the next batch of records. - * - * @return true if the last batch was generated, otherwise false - */ - private boolean generateBatch() throws Exception { - if (batch == null) { - while (!batchIterator.hasNext()) { - if (section != null) { - log.info("Generated {} record(s) for the {} section of snapshot {}.", - numRecords, section.name(), writer.snapshotId()); - section = null; - numRecords = 0; - } - if (!sectionIterator.hasNext()) { - writer.freeze(); - return true; - } - section = sectionIterator.next(); - log.info("Generating records for the {} section of snapshot {}.", - section.name(), writer.snapshotId()); - batchIterator = section.iterator(); - } - batch = batchIterator.next(); - } - - writer.append(batch); - numRecords += batch.size(); - batch = null; - return false; - } - - /** - * Generate the next few batches of records. - * - * @return The number of nanoseconds to delay before rescheduling the - * generateBatches event, or empty if the snapshot is done. - */ - OptionalLong generateBatches() throws Exception { - for (int numBatches = 0; numBatches < maxBatchesPerGenerateCall; numBatches++) { - if (generateBatch()) { - return OptionalLong.empty(); - } - } - return OptionalLong.of(0); - } -} diff --git a/metadata/src/main/java/org/apache/kafka/image/loader/LogDeltaManifest.java b/metadata/src/main/java/org/apache/kafka/image/loader/LogDeltaManifest.java new file mode 100644 index 0000000000000..c3ab72b3b79f2 --- /dev/null +++ b/metadata/src/main/java/org/apache/kafka/image/loader/LogDeltaManifest.java @@ -0,0 +1,106 @@ +/* + * 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.image.loader; + +import org.apache.kafka.image.MetadataProvenance; + +import java.util.Objects; + + +/** + * Contains information about a set of changes that were loaded from the metadata log. + */ +public class LogDeltaManifest { + /** + * The highest offset and epoch included in this delta, inclusive. + */ + private final MetadataProvenance provenance; + + /** + * The number of batches that were loaded. + */ + private final int numBatches; + + /** + * The time in nanoseconds that it took to load the changes. + */ + private final long elapsedNs; + + /** + * The total size of the records in bytes that we read while creating the delta. + */ + private final long numBytes; + + public LogDeltaManifest( + MetadataProvenance provenance, + int numBatches, + long elapsedNs, + long numBytes + ) { + this.provenance = provenance; + this.numBatches = numBatches; + this.elapsedNs = elapsedNs; + this.numBytes = numBytes; + } + + + public MetadataProvenance provenance() { + return provenance; + } + + public int numBatches() { + return numBatches; + } + + public long elapsedNs() { + return elapsedNs; + } + + public long numBytes() { + return numBytes; + } + + @Override + public int hashCode() { + return Objects.hash( + provenance, + numBatches, + elapsedNs, + numBytes); + } + + @Override + public boolean equals(Object o) { + if (o == null || !o.getClass().equals(this.getClass())) return false; + LogDeltaManifest other = (LogDeltaManifest) o; + return provenance.equals(other.provenance) && + numBatches == other.numBatches && + elapsedNs == other.elapsedNs && + numBytes == other.numBytes; + } + + @Override + public String toString() { + return "LogDeltaManifest(" + + "provenance=" + provenance + + ", numBatches=" + numBatches + + ", elapsedNs=" + elapsedNs + + ", numBytes=" + numBytes + + ")"; + } +} diff --git a/metadata/src/main/java/org/apache/kafka/image/loader/MetadataLoader.java b/metadata/src/main/java/org/apache/kafka/image/loader/MetadataLoader.java new file mode 100644 index 0000000000000..8672c98f2112d --- /dev/null +++ b/metadata/src/main/java/org/apache/kafka/image/loader/MetadataLoader.java @@ -0,0 +1,473 @@ +/* + * 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.image.loader; + +import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.image.MetadataDelta; +import org.apache.kafka.image.MetadataImage; +import org.apache.kafka.image.MetadataProvenance; +import org.apache.kafka.image.publisher.MetadataPublisher; +import org.apache.kafka.image.writer.ImageReWriter; +import org.apache.kafka.image.writer.ImageWriterOptions; +import org.apache.kafka.queue.KafkaEventQueue; +import org.apache.kafka.raft.Batch; +import org.apache.kafka.raft.BatchReader; +import org.apache.kafka.raft.LeaderAndEpoch; +import org.apache.kafka.raft.RaftClient; +import org.apache.kafka.server.common.ApiMessageAndVersion; +import org.apache.kafka.server.fault.FaultHandler; +import org.apache.kafka.server.fault.FaultHandlerException; +import org.apache.kafka.snapshot.SnapshotReader; +import org.slf4j.Logger; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.CompletableFuture; + +import static java.util.concurrent.TimeUnit.NANOSECONDS; + + +/** + * The MetadataLoader follows changes provided by a RaftClient, and packages them into metadata + * deltas and images that can be consumed by publishers. + * + * The Loader maintains its own thread, which is used to make all callbacks into publishers. If a + * publisher A is installed before B, A will receive all callbacks before B. This is also true if + * A and B are installed as part of a list [A, B]. + * + * Publishers should not modify any data structures passed to them. + * + * It is possible to change the list of publishers dynamically over time. Whenever a new publisher is + * added, it receives a catch-up delta which contains the full state. Any publisher installed when the + * loader is closed will itself be closed. + */ +public class MetadataLoader implements RaftClient.Listener, AutoCloseable { + public static class Builder { + private int nodeId = -1; + private Time time = Time.SYSTEM; + private LogContext logContext = null; + private String threadNamePrefix = ""; + private FaultHandler faultHandler = (m, e) -> new FaultHandlerException(m, e); + private MetadataLoaderMetrics metrics = new MetadataLoaderMetrics() { + private volatile long lastAppliedOffset = -1L; + + @Override + public void updateBatchProcessingTime(long elapsedNs) { } + + @Override + public void updateBatchSize(int size) { } + + @Override + public void updateLastAppliedImageProvenance(MetadataProvenance provenance) { + this.lastAppliedOffset = provenance.offset(); + } + + @Override + public long lastAppliedOffset() { + return lastAppliedOffset; + } + + @Override + public void close() throws Exception { } + }; + + public Builder setNodeId(int nodeId) { + this.nodeId = nodeId; + return this; + } + + public Builder setTime(Time time) { + this.time = time; + return this; + } + + public Builder setThreadNamePrefix(String threadNamePrefix) { + this.threadNamePrefix = threadNamePrefix; + return this; + } + + public Builder setFaultHandler(FaultHandler faultHandler) { + this.faultHandler = faultHandler; + return this; + } + + public Builder setMetadataLoaderMetrics(MetadataLoaderMetrics metrics) { + this.metrics = metrics; + return this; + } + + public MetadataLoader build() { + if (logContext == null) { + logContext = new LogContext("[MetadataLoader " + nodeId + "] "); + } + return new MetadataLoader( + time, + logContext, + threadNamePrefix, + faultHandler, + metrics); + } + } + + /** + * The log4j logger for this loader. + */ + private final Logger log; + + /** + * The clock used by this loader. + */ + private final Time time; + + /** + * The fault handler to use if metadata loading fails. + */ + private final FaultHandler faultHandler; + + /** + * Callbacks for updating metrics. + */ + private final MetadataLoaderMetrics metrics; + + /** + * The publishers which should receive cluster metadata updates. + */ + private final List publishers; + + /** + * The current leader and epoch. + */ + private LeaderAndEpoch currentLeaderAndEpoch = LeaderAndEpoch.UNKNOWN; + + /** + * The current metadata image. Accessed only from the event queue thread. + */ + private MetadataImage image; + + /** + * The event queue which runs this loader. + */ + private final KafkaEventQueue eventQueue; + + private MetadataLoader( + Time time, + LogContext logContext, + String threadNamePrefix, + FaultHandler faultHandler, + MetadataLoaderMetrics metrics + ) { + this.log = logContext.logger(MetadataLoader.class); + this.time = time; + this.faultHandler = faultHandler; + this.metrics = metrics; + this.publishers = new ArrayList<>(); + this.image = MetadataImage.EMPTY; + this.eventQueue = new KafkaEventQueue(time, logContext, threadNamePrefix); + } + + @Override + public void handleCommit(BatchReader reader) { + eventQueue.append(() -> { + try { + MetadataDelta delta = new MetadataDelta.Builder(). + setImage(image). + build(); + LogDeltaManifest manifest = loadLogDelta(delta, reader); + if (log.isDebugEnabled()) { + log.debug("Generated a metadata delta between {} and {} from {} batch(es) " + + "in {} us.", image.offset(), manifest.provenance().offset(), + manifest.numBatches(), NANOSECONDS.toMicros(manifest.elapsedNs())); + } + try { + image = delta.apply(manifest.provenance()); + } catch (Throwable e) { + faultHandler.handleFault("Error generating new metadata image from " + + "metadata delta between offset " + image.offset() + + " and " + manifest.provenance().offset(), e); + return; + } + log.debug("Publishing new image with provenance {}.", image.provenance()); + for (MetadataPublisher publisher : publishers) { + try { + publisher.publishLogDelta(delta, image, manifest); + } catch (Throwable e) { + faultHandler.handleFault("Unhandled error publishing the new metadata " + + "image ending at " + manifest.provenance().offset() + + " with publisher " + publisher.name(), e); + } + } + metrics.updateLastAppliedImageProvenance(image.provenance()); + } catch (Throwable e) { + // This is a general catch-all block where we don't expect to end up; + // failure-prone operations should have individual try/catch blocks around them. + faultHandler.handleFault("Unhandled fault in MetadataLoader#handleCommit. " + + "Last image offset was " + image.offset(), e); + } finally { + reader.close(); + } + }); + } + + /** + * Load some batches of records from the log. We have to do some bookkeeping here to + * translate between batch offsets and record offsets, and track the number of bytes we + * have read. Additionally, there is the chance that one of the records is a metadata + * version change which needs to be handled differently. + * + * @param delta The metadata delta we are preparing. + * @param reader The reader which yields the batches. + * @return A manifest of what was loaded. + */ + LogDeltaManifest loadLogDelta( + MetadataDelta delta, + BatchReader reader + ) { + long startNs = time.nanoseconds(); + int numBatches = 0; + long numBytes = 0L; + long lastOffset = image.provenance().offset(); + int lastEpoch = image.provenance().epoch(); + long lastContainedLogTimeMs = image.provenance().lastContainedLogTimeMs(); + + while (reader.hasNext()) { + Batch batch = reader.next(); + int indexWithinBatch = 0; + for (ApiMessageAndVersion record : batch.records()) { + try { + delta.replay(record.message()); + } catch (Throwable e) { + faultHandler.handleFault("Error loading metadata log record from offset " + + batch.baseOffset() + indexWithinBatch, e); + } + indexWithinBatch++; + } + metrics.updateBatchSize(batch.records().size()); + lastOffset = batch.lastOffset(); + lastEpoch = batch.epoch(); + lastContainedLogTimeMs = batch.appendTimestamp(); + numBytes += batch.sizeInBytes(); + numBatches++; + } + MetadataProvenance provenance = + new MetadataProvenance(lastOffset, lastEpoch, lastContainedLogTimeMs); + long elapsedNs = time.nanoseconds() - startNs; + metrics.updateBatchProcessingTime(elapsedNs); + return new LogDeltaManifest(provenance, + numBatches, + elapsedNs, + numBytes); + } + + @Override + public void handleSnapshot(SnapshotReader reader) { + eventQueue.append(() -> { + try { + MetadataDelta delta = new MetadataDelta.Builder(). + setImage(image). + build(); + SnapshotManifest manifest = loadSnapshot(delta, reader); + if (log.isDebugEnabled()) { + log.debug("Generated a metadata delta from a snapshot at offset {} " + + "in {} us.", manifest.provenance().offset(), + NANOSECONDS.toMicros(manifest.elapsedNs())); + } + try { + image = delta.apply(manifest.provenance()); + } catch (Throwable e) { + faultHandler.handleFault("Error generating new metadata image from " + + "snapshot at offset " + reader.lastContainedLogOffset(), e); + return; + } + log.debug("Publishing new snapshot image with provenance {}.", image.provenance()); + for (MetadataPublisher publisher : publishers) { + try { + publisher.publishSnapshot(delta, image, manifest); + } catch (Throwable e) { + faultHandler.handleFault("Unhandled error publishing the new metadata " + + "image from snapshot at offset " + reader.lastContainedLogOffset() + + " with publisher " + publisher.name(), e); + } + } + metrics.updateLastAppliedImageProvenance(image.provenance()); + } catch (Throwable e) { + // This is a general catch-all block where we don't expect to end up; + // failure-prone operations should have individual try/catch blocks around them. + faultHandler.handleFault("Unhandled fault in MetadataLoader#handleSnapshot. " + + "Snapshot offset was " + reader.lastContainedLogOffset(), e); + } finally { + reader.close(); + } + }); + } + + /** + * Load a snapshot. This is relatively straightforward since we don't track as many things as + * we do in loadLogDelta. The main complication here is that we have to maintain an index + * of what record we are processing so that we can give useful error messages. + * + * @param delta The metadata delta we are preparing. + * @param reader The reader which yields the snapshot batches. + * @return A manifest of what was loaded. + */ + SnapshotManifest loadSnapshot( + MetadataDelta delta, + SnapshotReader reader + ) { + long startNs = time.nanoseconds(); + int snapshotIndex = 0; + while (reader.hasNext()) { + Batch batch = reader.next(); + for (ApiMessageAndVersion record : batch.records()) { + try { + delta.replay(record.message()); + } catch (Throwable e) { + faultHandler.handleFault("Error loading metadata log record " + snapshotIndex + + " in snapshot at offset " + reader.lastContainedLogOffset(), e); + } + snapshotIndex++; + } + } + MetadataProvenance provenance = new MetadataProvenance(reader.lastContainedLogOffset(), + reader.lastContainedLogEpoch(), reader.lastContainedLogTimestamp()); + return new SnapshotManifest(provenance, + time.nanoseconds() - startNs); + } + + @Override + public void handleLeaderChange(LeaderAndEpoch leaderAndEpoch) { + eventQueue.append(() -> { + currentLeaderAndEpoch = leaderAndEpoch; + }); + } + + /** + * Install a list of publishers. When a publisher is installed, we will publish a MetadataDelta + * to it which contains the entire current image. + * + * @param newPublishers The publishers to install. + * + * @return A future which yields null when the publishers have been added, or + * an exception if the installation failed. + */ + public CompletableFuture installPublishers(List newPublishers) { + if (newPublishers.isEmpty()) return CompletableFuture.completedFuture(null); + CompletableFuture future = new CompletableFuture<>(); + eventQueue.append(() -> { + try { + installNewPublishers(newPublishers); + future.complete(null); + } catch (Throwable e) { + future.completeExceptionally(faultHandler.handleFault("Unhandled fault in " + + "MetadataLoader#installPublishers", e)); + } + }); + return future; + } + + void installNewPublishers( + List newPublishers + ) { + long startNs = time.nanoseconds(); + // Publishers can't be re-installed if they're already present. + for (MetadataPublisher publisher : newPublishers) { + if (publishers.contains(publisher)) { + throw faultHandler.handleFault("Attempted to install publisher " + publisher.name() + + ", which is already installed."); + } + } + MetadataDelta delta = new MetadataDelta.Builder(). + setImage(image). + build(); + ImageReWriter writer = new ImageReWriter(delta); + image.write(writer, new ImageWriterOptions.Builder(). + setMetadataVersion(image.features().metadataVersion()). + build()); + SnapshotManifest manifest = new SnapshotManifest( + image.provenance(), + time.nanoseconds() - startNs); + for (MetadataPublisher publisher : newPublishers) { + try { + log.info("Publishing initial snapshot at offset {} to {}", + image.highestOffsetAndEpoch().offset(), publisher.name()); + publisher.publishSnapshot(delta, image, manifest); + publishers.add(publisher); + } catch (Throwable e) { + faultHandler.handleFault("Unhandled error publishing the initial metadata " + + "image from snapshot at offset " + image.highestOffsetAndEpoch().offset() + + " with publisher " + publisher.name(), e); + } + } + } + + // VisibleForTesting + void waitForAllEventsToBeHandled() throws Exception { + CompletableFuture future = new CompletableFuture<>(); + eventQueue.append(() -> future.complete(null)); + future.get(); + } + + /** + * Remove a publisher and close it. + * + * @param publisher The publisher to remove and close. + * + * @return A future which yields null when the publisher has been removed + * and closed, or an exception if the removal failed. + */ + public CompletableFuture removeAndClosePublisher(MetadataPublisher publisher) { + CompletableFuture future = new CompletableFuture<>(); + eventQueue.append(() -> { + try { + if (!publishers.remove(publisher)) { + throw faultHandler.handleFault("Attempted to remove publisher " + publisher.name() + + ", which is not installed."); + } + publisher.close(); + future.complete(null); + } catch (Throwable e) { + future.completeExceptionally(e); + } + }); + return future; + } + + public long lastAppliedOffset() { + return metrics.lastAppliedOffset(); + } + + @Override + public void beginShutdown() { + eventQueue.beginShutdown("beginShutdown", () -> { + for (MetadataPublisher publisher : publishers) { + try { + publisher.close(); + } catch (Throwable e) { + faultHandler.handleFault("Got unexpected exception while closing " + + "publisher " + publisher.name(), e); + } + } + }); + } + + @Override + public void close() throws Exception { + beginShutdown(); + eventQueue.close(); + } +} diff --git a/metadata/src/main/java/org/apache/kafka/image/loader/MetadataLoaderMetrics.java b/metadata/src/main/java/org/apache/kafka/image/loader/MetadataLoaderMetrics.java new file mode 100644 index 0000000000000..654bc9dd505d9 --- /dev/null +++ b/metadata/src/main/java/org/apache/kafka/image/loader/MetadataLoaderMetrics.java @@ -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.image.loader; + +import org.apache.kafka.image.MetadataProvenance; + + +/** + * An interface for the metadata loader metrics. + */ +public interface MetadataLoaderMetrics extends AutoCloseable { + /** + * Update the batch processing time histogram. + */ + void updateBatchProcessingTime(long elapsedNs); + + /** + * Update the batch size histogram. + */ + void updateBatchSize(int size); + + /** + * Set the provenance of the last image which has been processed by all publishers. + */ + void updateLastAppliedImageProvenance(MetadataProvenance provenance); + + /** + * Retrieve the last offset which has been processed by all publishers. + */ + long lastAppliedOffset(); +} diff --git a/metadata/src/main/java/org/apache/kafka/image/loader/SnapshotManifest.java b/metadata/src/main/java/org/apache/kafka/image/loader/SnapshotManifest.java new file mode 100644 index 0000000000000..b6c6dcce4d5ea --- /dev/null +++ b/metadata/src/main/java/org/apache/kafka/image/loader/SnapshotManifest.java @@ -0,0 +1,77 @@ +/* + * 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.image.loader; + +import org.apache.kafka.image.MetadataProvenance; + +import java.util.Objects; + + +/** + * Contains information about a snapshot that was loaded. + */ +public class SnapshotManifest { + /** + * The source of this snapshot. + */ + private final MetadataProvenance provenance; + + /** + * The time in microseconds that it took to load the snapshot. + */ + private final long elapsedNs; + + public SnapshotManifest( + MetadataProvenance provenance, + long elapsedNs + ) { + this.provenance = provenance; + this.elapsedNs = elapsedNs; + } + + public MetadataProvenance provenance() { + return provenance; + } + + public long elapsedNs() { + return elapsedNs; + } + + @Override + public int hashCode() { + return Objects.hash( + provenance, + elapsedNs); + } + + @Override + public boolean equals(Object o) { + if (o == null || !o.getClass().equals(this.getClass())) return false; + SnapshotManifest other = (SnapshotManifest) o; + return provenance.equals(other.provenance) && + elapsedNs == other.elapsedNs; + } + + @Override + public String toString() { + return "SnapshotManifest(" + + "provenance=" + provenance + + ", elapsedNs=" + elapsedNs + + ")"; + } +} diff --git a/metadata/src/main/java/org/apache/kafka/image/publisher/MetadataPublisher.java b/metadata/src/main/java/org/apache/kafka/image/publisher/MetadataPublisher.java new file mode 100644 index 0000000000000..8dfba7a99abd8 --- /dev/null +++ b/metadata/src/main/java/org/apache/kafka/image/publisher/MetadataPublisher.java @@ -0,0 +1,72 @@ +/* + * 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.image.publisher; + +import org.apache.kafka.image.MetadataDelta; +import org.apache.kafka.image.MetadataImage; +import org.apache.kafka.image.loader.LogDeltaManifest; +import org.apache.kafka.image.loader.SnapshotManifest; + + +/** + * Publishes metadata deltas which we have loaded from the log and snapshots. + * + * Publishers receive a stream of callbacks from the metadata loader which keeps them notified + * of the latest cluster metadata. This interface abstracts away some of the complications of + * following the cluster metadata. For example, if the loader needs to read a snapshot, it will + * present the contents of the snapshot in the form of a delta from the previous state. + */ +public interface MetadataPublisher extends AutoCloseable { + /** + * Returns the name of this publisher. + * + * @return The publisher name. + */ + String name(); + + /** + * Publish a new cluster metadata snapshot that we loaded. + * + * @param delta The delta between the previous state and the new one. + * @param newImage The complete new state. + * @param manifest The contents of what was published. + */ + void publishSnapshot( + MetadataDelta delta, + MetadataImage newImage, + SnapshotManifest manifest + ); + + /** + * Publish a change to the cluster metadata. + * + * @param delta The delta between the previous state and the new one. + * @param newImage The complete new state. + * @param manifest The contents of what was published. + */ + void publishLogDelta( + MetadataDelta delta, + MetadataImage newImage, + LogDeltaManifest manifest + ); + + /** + * Close this metadata publisher. + */ + void close() throws Exception; +} diff --git a/metadata/src/main/java/org/apache/kafka/image/publisher/SnapshotEmitter.java b/metadata/src/main/java/org/apache/kafka/image/publisher/SnapshotEmitter.java new file mode 100644 index 0000000000000..31ac2169be4e4 --- /dev/null +++ b/metadata/src/main/java/org/apache/kafka/image/publisher/SnapshotEmitter.java @@ -0,0 +1,123 @@ +/* + * 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.image.publisher; + +import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.image.MetadataImage; +import org.apache.kafka.image.MetadataProvenance; +import org.apache.kafka.image.writer.ImageWriterOptions; +import org.apache.kafka.image.writer.RaftSnapshotWriter; +import org.apache.kafka.raft.RaftClient; +import org.apache.kafka.server.common.ApiMessageAndVersion; +import org.apache.kafka.snapshot.SnapshotWriter; +import org.slf4j.Logger; + +import java.util.Optional; + + +public class SnapshotEmitter implements SnapshotGenerator.Emitter { + /** + * The maximum number of records we will put in each snapshot batch by default. + * + * From the perspective of the Raft layer, the limit on batch size is specified in terms of + * bytes, not number of records. See MAX_BATCH_SIZE_BYTES in KafkaRaftClient for details. + * However, it's more convenient to limit the batch size here in terms of number of records. + * So we chose a low number that will not cause problems. + */ + private final static int DEFAULT_BATCH_SIZE = 1024; + + public static class Builder { + private int nodeId = 0; + private RaftClient raftClient = null; + private int batchSize = DEFAULT_BATCH_SIZE; + + public Builder setNodeId(int nodeId) { + this.nodeId = nodeId; + return this; + } + + public Builder setRaftClient(RaftClient raftClient) { + this.raftClient = raftClient; + return this; + } + + public Builder setBatchSize(int batchSize) { + this.batchSize = batchSize; + return this; + } + + public SnapshotEmitter build() { + if (raftClient == null) throw new RuntimeException("You must set the raftClient."); + return new SnapshotEmitter(nodeId, + raftClient, + batchSize); + } + } + + /** + * The slf4j logger to use. + */ + private final Logger log; + + /** + * The RaftClient to use. + */ + private final RaftClient raftClient; + + /** + * The maximum number of records to put in each batch. + */ + private final int batchSize; + + private SnapshotEmitter( + int nodeId, + RaftClient raftClient, + int batchSize + ) { + this.log = new LogContext("[SnapshotEmitter id=" + nodeId + "] ").logger(SnapshotEmitter.class); + this.raftClient = raftClient; + this.batchSize = batchSize; + } + + @Override + public void maybeEmit(MetadataImage image) { + MetadataProvenance provenance = image.provenance(); + Optional> snapshotWriter = + raftClient.createSnapshot(provenance.offsetAndEpoch(), + provenance.lastContainedLogTimeMs()); + if (!snapshotWriter.isPresent()) { + log.error("Not generating {} because it already exists.", provenance.snapshotName()); + return; + } + RaftSnapshotWriter writer = new RaftSnapshotWriter(snapshotWriter.get(), batchSize); + try { + image.write(writer, new ImageWriterOptions.Builder(). + setMetadataVersion(image.features().metadataVersion()). + build()); + writer.close(true); + } catch (Throwable e) { + log.error("Encountered error while writing {}", provenance.snapshotName(), e); + throw e; + } finally { + Utils.closeQuietly(writer, "RaftSnapshotWriter"); + Utils.closeQuietly(snapshotWriter.get(), "SnapshotWriter"); + } + log.info("Successfully wrote {}", provenance.snapshotName()); + } +} diff --git a/metadata/src/main/java/org/apache/kafka/image/publisher/SnapshotGenerator.java b/metadata/src/main/java/org/apache/kafka/image/publisher/SnapshotGenerator.java new file mode 100644 index 0000000000000..43809de3898fe --- /dev/null +++ b/metadata/src/main/java/org/apache/kafka/image/publisher/SnapshotGenerator.java @@ -0,0 +1,275 @@ +/* + * 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.image.publisher; + +import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.image.MetadataDelta; +import org.apache.kafka.image.MetadataImage; +import org.apache.kafka.image.loader.LogDeltaManifest; +import org.apache.kafka.image.loader.SnapshotManifest; +import org.apache.kafka.queue.EventQueue; +import org.apache.kafka.queue.KafkaEventQueue; +import org.apache.kafka.server.fault.FaultHandler; +import org.slf4j.Logger; + +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; + + +/** + * A metadata publisher that generates snapshots when appropriate. + */ +public class SnapshotGenerator implements MetadataPublisher { + public static class Builder { + private final Emitter emitter; + private int nodeId = 0; + private Time time = Time.SYSTEM; + private FaultHandler faultHandler = (m, e) -> null; + private long maxBytesSinceLastSnapshot = 100 * 1024L * 1024L; + private long maxTimeSinceLastSnapshotNs = TimeUnit.DAYS.toNanos(1); + private AtomicReference disabledReason = null; + + public Builder(Emitter emitter) { + this.emitter = emitter; + } + + public Builder setNodeId(int nodeId) { + this.nodeId = nodeId; + return this; + } + + public Builder setTime(Time time) { + this.time = time; + return this; + } + + public Builder setFaultHandler(FaultHandler faultHandler) { + this.faultHandler = faultHandler; + return this; + } + + public Builder setMaxBytesSinceLastSnapshot(long maxBytesSinceLastSnapshot) { + this.maxBytesSinceLastSnapshot = maxBytesSinceLastSnapshot; + return this; + } + + public Builder setMaxTimeSinceLastSnapshotNs(long maxTimeSinceLastSnapshotNs) { + this.maxTimeSinceLastSnapshotNs = maxTimeSinceLastSnapshotNs; + return this; + } + + public Builder setDisabledReason(AtomicReference disabledReason) { + this.disabledReason = disabledReason; + return this; + } + + public SnapshotGenerator build() { + if (disabledReason == null) { + disabledReason = new AtomicReference<>(); + } + return new SnapshotGenerator( + nodeId, + time, + emitter, + faultHandler, + maxBytesSinceLastSnapshot, + maxTimeSinceLastSnapshotNs, + disabledReason + ); + } + } + + /** + * The callback which actually generates the snapshot. + */ + public interface Emitter { + /** + * Emit a snapshot for the given image. + * + * Note: if a snapshot has already been emitted for the given offset and epoch pair, this + * function will not recreate it. + * + * @param image The metadata image to emit. + */ + void maybeEmit(MetadataImage image); + } + + /** + * The node ID. + */ + private final int nodeId; + + /** + * The clock to use. + */ + private final Time time; + + /** + * The emitter callback, which actually generates the snapshot. + */ + private final Emitter emitter; + + /** + * The slf4j logger to use. + */ + private final Logger log; + + /** + * The fault handler to use. + */ + private final FaultHandler faultHandler; + + /** + * The maximum number of bytes we will wait to see before triggering a new snapshot. + */ + private final long maxBytesSinceLastSnapshot; + + /** + * The maximum amount of time we will wait before triggering a snapshot, or 0 to disable + * time-based snapshotting. + */ + private final long maxTimeSinceLastSnapshotNs; + + /** + * If non-null, the reason why snapshots have been disabled. + */ + private final AtomicReference disabledReason; + + /** + * The event queue used to schedule emitting snapshots. + */ + private final EventQueue eventQueue; + + /** + * The log bytes that we have read since the last snapshot. + */ + private long bytesSinceLastSnapshot; + + /** + * The time at which we created the last snapshot. + */ + private long lastSnapshotTimeNs; + + private SnapshotGenerator( + int nodeId, + Time time, + Emitter emitter, + FaultHandler faultHandler, + long maxBytesSinceLastSnapshot, + long maxTimeSinceLastSnapshotNs, + AtomicReference disabledReason + ) { + this.nodeId = nodeId; + this.time = time; + this.emitter = emitter; + this.faultHandler = faultHandler; + this.maxBytesSinceLastSnapshot = maxBytesSinceLastSnapshot; + this.maxTimeSinceLastSnapshotNs = maxTimeSinceLastSnapshotNs; + LogContext logContext = new LogContext("[SnapshotGenerator " + nodeId + "] "); + this.log = logContext.logger(SnapshotGenerator.class); + this.disabledReason = disabledReason; + this.eventQueue = new KafkaEventQueue(time, logContext, "SnapshotGenerator" + nodeId); + resetSnapshotCounters(); + log.debug("Starting SnapshotGenerator."); + } + + @Override + public String name() { + return "SnapshotGenerator"; + } + + void resetSnapshotCounters() { + this.bytesSinceLastSnapshot = 0L; + this.lastSnapshotTimeNs = time.nanoseconds(); + } + + @Override + public void publishSnapshot( + MetadataDelta delta, + MetadataImage newImage, + SnapshotManifest manifest + ) { + log.debug("Resetting the snapshot counters because we just read a snapshot at offset {}.", + newImage.provenance().offset()); + resetSnapshotCounters(); + } + + @Override + public void publishLogDelta( + MetadataDelta delta, + MetadataImage newImage, + LogDeltaManifest manifest + ) { + bytesSinceLastSnapshot += manifest.numBytes(); + if (bytesSinceLastSnapshot >= maxBytesSinceLastSnapshot) { + if (eventQueue.isEmpty()) { + scheduleEmit("we have replayed at least " + maxBytesSinceLastSnapshot + + " bytes", newImage); + } else if (log.isTraceEnabled()) { + log.trace("Not scheduling bytes-based snapshot because event queue is not empty yet."); + } + } else if (maxTimeSinceLastSnapshotNs != 0 && + (time.nanoseconds() - lastSnapshotTimeNs >= maxTimeSinceLastSnapshotNs)) { + if (eventQueue.isEmpty()) { + scheduleEmit("we have waited at least " + + TimeUnit.NANOSECONDS.toMinutes(maxTimeSinceLastSnapshotNs) + " minute(s)", newImage); + } else if (log.isTraceEnabled()) { + log.trace("Not scheduling time-based snapshot because event queue is not empty yet."); + } + } else if (log.isTraceEnabled()) { + log.trace("Neither time-based nor bytes-based criteria are met; not scheduling snapshot."); + } + } + + void scheduleEmit( + String reason, + MetadataImage image + ) { + resetSnapshotCounters(); + eventQueue.append(() -> { + String currentDisabledReason = disabledReason.get(); + if (currentDisabledReason != null) { + log.error("Not emitting {} despite the fact that {} because snapshots are " + + "disabled; {}", image.provenance().snapshotName(), reason, + currentDisabledReason); + } else { + log.info("Creating new KRaft snapshot file {} because {}.", + image.provenance().snapshotName(), reason); + try { + emitter.maybeEmit(image); + } catch (Throwable e) { + faultHandler.handleFault("KRaft snapshot file generation error", e); + } + } + }); + } + + public void beginShutdown() { + log.debug("Beginning shutdown of SnapshotGenerator."); + this.disabledReason.compareAndSet(null, "we are shutting down"); + eventQueue.beginShutdown("beginShutdown"); + } + + @Override + public void close() throws InterruptedException { + eventQueue.beginShutdown("close"); + log.debug("Closing SnapshotGenerator."); + eventQueue.close(); + } +} diff --git a/metadata/src/test/java/org/apache/kafka/controller/AclControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/AclControlManagerTest.java index fdc0327645176..566fa4acd5471 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/AclControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/AclControlManagerTest.java @@ -215,11 +215,9 @@ public void testLoadSnapshot() { // Verify that the ACLs stored in the AclControlManager match the ones we expect. Set foundAcls = new HashSet<>(); - for (Iterator> iterator = manager.iterator(Long.MAX_VALUE); - iterator.hasNext(); ) { - for (ApiMessageAndVersion apiMessageAndVersion : iterator.next()) { - assertTrue(foundAcls.add(apiMessageAndVersion)); - } + for (Map.Entry entry : manager.idToAcl().entrySet()) { + foundAcls.add(new ApiMessageAndVersion( + new StandardAclWithId(entry.getKey(), entry.getValue()).toRecord(), (short) 0)); } assertEquals(loadedAcls, foundAcls); @@ -233,7 +231,7 @@ public void testLoadSnapshot() { // a cluster metadata authorizer. snapshotRegistry.revertToSnapshot(0); authorizer.loadSnapshot(manager.idToAcl()); - assertFalse(manager.iterator(Long.MAX_VALUE).hasNext()); + assertTrue(manager.idToAcl().isEmpty()); } @Test @@ -243,11 +241,9 @@ public void testAddAndDelete() { MockClusterMetadataAuthorizer authorizer = new MockClusterMetadataAuthorizer(); authorizer.loadSnapshot(manager.idToAcl()); manager.replay(StandardAclWithIdTest.TEST_ACLS.get(0).toRecord(), Optional.empty()); - assertEquals(new ApiMessageAndVersion(TEST_ACLS.get(0).toRecord(), (short) 0), - manager.iterator(Long.MAX_VALUE).next().get(0)); manager.replay(new RemoveAccessControlEntryRecord(). setId(TEST_ACLS.get(0).id()), Optional.empty()); - assertFalse(manager.iterator(Long.MAX_VALUE).hasNext()); + assertTrue(manager.idToAcl().isEmpty()); } @Test @@ -284,7 +280,7 @@ public void testCreateAclDeleteAcl() { } } RecordTestUtils.replayAll(manager, createResult.records()); - assertTrue(manager.iterator(Long.MAX_VALUE).hasNext()); + assertFalse(manager.idToAcl().isEmpty()); ControllerResult> deleteResult = manager.deleteAcls(Arrays.asList( @@ -308,12 +304,8 @@ public void testCreateAclDeleteAcl() { deleteResult.response().get(1).exception().get().getClass()); RecordTestUtils.replayAll(manager, deleteResult.records()); - Iterator> iterator = manager.iterator(Long.MAX_VALUE); - assertTrue(iterator.hasNext()); - List list = iterator.next(); - assertEquals(1, list.size()); - assertEquals(TEST_ACLS.get(1).toBinding(), StandardAcl.fromRecord( - (AccessControlEntryRecord) list.get(0).message()).toBinding()); + Iterator> iterator = manager.idToAcl().entrySet().iterator(); + assertEquals(TEST_ACLS.get(1).acl(), iterator.next().getValue()); assertFalse(iterator.hasNext()); } diff --git a/metadata/src/test/java/org/apache/kafka/controller/ClientQuotaControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/ClientQuotaControlManagerTest.java index b915db3fe0d49..1fb81cbf7a473 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/ClientQuotaControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/ClientQuotaControlManagerTest.java @@ -205,49 +205,50 @@ public void testEntityTypes() throws Exception { List alters = new ArrayList<>(); quotasToTest.forEach((entity, quota) -> entityQuotaToAlterations(entity, quota, alters::add)); - alterQuotas(alters, manager); - - RecordTestUtils.assertBatchIteratorContains(Arrays.asList( - Arrays.asList(new ApiMessageAndVersion(new ClientQuotaRecord().setEntity(Arrays.asList( + List records = alterQuotas(alters, manager); + List expectedRecords = Arrays.asList( + new ApiMessageAndVersion(new ClientQuotaRecord().setEntity(Arrays.asList( new EntityData().setEntityType("user").setEntityName("user-1"), new EntityData().setEntityType("client-id").setEntityName("client-id-1"))). - setKey("request_percentage").setValue(50.5).setRemove(false), (short) 0)), - Arrays.asList(new ApiMessageAndVersion(new ClientQuotaRecord().setEntity(Arrays.asList( + setKey("request_percentage").setValue(50.5).setRemove(false), (short) 0), + new ApiMessageAndVersion(new ClientQuotaRecord().setEntity(Arrays.asList( new EntityData().setEntityType("user").setEntityName("user-2"), new EntityData().setEntityType("client-id").setEntityName("client-id-1"))). - setKey("request_percentage").setValue(51.51).setRemove(false), (short) 0)), - Arrays.asList(new ApiMessageAndVersion(new ClientQuotaRecord().setEntity(Arrays.asList( + setKey("request_percentage").setValue(51.51).setRemove(false), (short) 0), + new ApiMessageAndVersion(new ClientQuotaRecord().setEntity(Arrays.asList( new EntityData().setEntityType("user").setEntityName("user-3"), new EntityData().setEntityType("client-id").setEntityName("client-id-2"))). - setKey("request_percentage").setValue(52.52).setRemove(false), (short) 0)), - Arrays.asList(new ApiMessageAndVersion(new ClientQuotaRecord().setEntity(Arrays.asList( + setKey("request_percentage").setValue(52.52).setRemove(false), (short) 0), + new ApiMessageAndVersion(new ClientQuotaRecord().setEntity(Arrays.asList( new EntityData().setEntityType("user").setEntityName(null), new EntityData().setEntityType("client-id").setEntityName("client-id-1"))). - setKey("request_percentage").setValue(53.53).setRemove(false), (short) 0)), - Arrays.asList(new ApiMessageAndVersion(new ClientQuotaRecord().setEntity(Arrays.asList( + setKey("request_percentage").setValue(53.53).setRemove(false), (short) 0), + new ApiMessageAndVersion(new ClientQuotaRecord().setEntity(Arrays.asList( new EntityData().setEntityType("user").setEntityName("user-1"), new EntityData().setEntityType("client-id").setEntityName(null))). - setKey("request_percentage").setValue(54.54).setRemove(false), (short) 0)), - Arrays.asList(new ApiMessageAndVersion(new ClientQuotaRecord().setEntity(Arrays.asList( + setKey("request_percentage").setValue(54.54).setRemove(false), (short) 0), + new ApiMessageAndVersion(new ClientQuotaRecord().setEntity(Arrays.asList( new EntityData().setEntityType("user").setEntityName("user-3"), new EntityData().setEntityType("client-id").setEntityName(null))). - setKey("request_percentage").setValue(55.55).setRemove(false), (short) 0)), - Arrays.asList(new ApiMessageAndVersion(new ClientQuotaRecord().setEntity(Arrays.asList( + setKey("request_percentage").setValue(55.55).setRemove(false), (short) 0), + new ApiMessageAndVersion(new ClientQuotaRecord().setEntity(Arrays.asList( new EntityData().setEntityType("user").setEntityName("user-1"))). - setKey("request_percentage").setValue(56.56).setRemove(false), (short) 0)), - Arrays.asList(new ApiMessageAndVersion(new ClientQuotaRecord().setEntity(Arrays.asList( + setKey("request_percentage").setValue(56.56).setRemove(false), (short) 0), + new ApiMessageAndVersion(new ClientQuotaRecord().setEntity(Arrays.asList( new EntityData().setEntityType("user").setEntityName("user-2"))). - setKey("request_percentage").setValue(57.57).setRemove(false), (short) 0)), - Arrays.asList(new ApiMessageAndVersion(new ClientQuotaRecord().setEntity(Arrays.asList( + setKey("request_percentage").setValue(57.57).setRemove(false), (short) 0), + new ApiMessageAndVersion(new ClientQuotaRecord().setEntity(Arrays.asList( new EntityData().setEntityType("user").setEntityName("user-3"))). - setKey("request_percentage").setValue(58.58).setRemove(false), (short) 0)), - Arrays.asList(new ApiMessageAndVersion(new ClientQuotaRecord().setEntity(Arrays.asList( + setKey("request_percentage").setValue(58.58).setRemove(false), (short) 0), + new ApiMessageAndVersion(new ClientQuotaRecord().setEntity(Arrays.asList( new EntityData().setEntityType("user").setEntityName(null))). - setKey("request_percentage").setValue(59.59).setRemove(false), (short) 0)), - Arrays.asList(new ApiMessageAndVersion(new ClientQuotaRecord().setEntity(Arrays.asList( + setKey("request_percentage").setValue(59.59).setRemove(false), (short) 0), + new ApiMessageAndVersion(new ClientQuotaRecord().setEntity(Arrays.asList( new EntityData().setEntityType("client-id").setEntityName("client-id-2"))). - setKey("request_percentage").setValue(60.60).setRemove(false), (short) 0))), - manager.iterator(Long.MAX_VALUE)); + setKey("request_percentage").setValue(60.60).setRemove(false), (short) 0)); + RecordTestUtils.deepSortRecords(records); + RecordTestUtils.deepSortRecords(expectedRecords); + assertEquals(expectedRecords, records); } static void entityQuotaToAlterations(ClientQuotaEntity entity, Map quota, @@ -258,11 +259,15 @@ static void entityQuotaToAlterations(ClientQuotaEntity entity, Map alterations, ClientQuotaControlManager manager) { + static List alterQuotas( + List alterations, + ClientQuotaControlManager manager + ) { ControllerResult> result = manager.alterClientQuotas(alterations); assertTrue(result.response().values().stream().allMatch(ApiError::isSuccess)); result.records().forEach(apiMessageAndVersion -> manager.replay((ClientQuotaRecord) apiMessageAndVersion.message())); + return result.records(); } static Map quotas(String key, Double value) { diff --git a/metadata/src/test/java/org/apache/kafka/controller/ClusterControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/ClusterControlManagerTest.java index 72eea5427e898..86918d5513c28 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/ClusterControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/ClusterControlManagerTest.java @@ -34,12 +34,12 @@ import org.apache.kafka.common.security.auth.SecurityProtocol; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.image.writer.ImageWriterOptions; import org.apache.kafka.metadata.BrokerRegistration; import org.apache.kafka.metadata.BrokerRegistrationFencingChange; import org.apache.kafka.metadata.BrokerRegistrationInControlledShutdownChange; import org.apache.kafka.metadata.BrokerRegistrationReply; import org.apache.kafka.metadata.FinalizedControllerFeatures; -import org.apache.kafka.metadata.RecordTestUtils; import org.apache.kafka.metadata.VersionRange; import org.apache.kafka.metadata.placement.PartitionAssignment; import org.apache.kafka.metadata.placement.PlacementSpec; @@ -418,7 +418,7 @@ public void testPlaceReplicas(int numUsableBrokers) throws Exception { @ParameterizedTest @EnumSource(value = MetadataVersion.class, names = {"IBP_3_3_IV2", "IBP_3_3_IV3"}) - public void testIterator(MetadataVersion metadataVersion) throws Exception { + public void testRegistrationsToRecords(MetadataVersion metadataVersion) throws Exception { MockTime time = new MockTime(0, 0, 0); SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext()); FeatureControlManager featureControl = new FeatureControlManager.Builder(). @@ -460,8 +460,12 @@ public void testIterator(MetadataVersion metadataVersion) throws Exception { IN_CONTROLLED_SHUTDOWN.value()); clusterControl.replay(registrationChangeRecord); short expectedVersion = metadataVersion.registerBrokerRecordVersion(); - RecordTestUtils.assertBatchIteratorContains(Arrays.asList( - Arrays.asList(new ApiMessageAndVersion(new RegisterBrokerRecord(). + + ImageWriterOptions options = new ImageWriterOptions.Builder(). + setMetadataVersion(metadataVersion). + setLossHandler(__ -> { }). + build(); + assertEquals(new ApiMessageAndVersion(new RegisterBrokerRecord(). setBrokerEpoch(100).setBrokerId(0).setRack(null). setEndPoints(new BrokerEndpointCollection(Collections.singleton( new BrokerEndpoint().setSecurityProtocol(SecurityProtocol.PLAINTEXT.id). @@ -469,27 +473,28 @@ public void testIterator(MetadataVersion metadataVersion) throws Exception { setName("PLAINTEXT"). setHost("example.com")).iterator())). setInControlledShutdown(metadataVersion.isInControlledShutdownStateSupported()). - setFenced(false), expectedVersion)), - Arrays.asList(new ApiMessageAndVersion(new RegisterBrokerRecord(). + setFenced(false), expectedVersion), + clusterControl.brokerRegistrations().get(0).toRecord(options)); + assertEquals(new ApiMessageAndVersion(new RegisterBrokerRecord(). setBrokerEpoch(100).setBrokerId(1).setRack(null). setEndPoints(new BrokerEndpointCollection(Collections.singleton( new BrokerEndpoint().setSecurityProtocol(SecurityProtocol.PLAINTEXT.id). setPort((short) 9093). setName("PLAINTEXT"). setHost("example.com")).iterator())). - setFenced(false), expectedVersion)), - Arrays.asList(new ApiMessageAndVersion(new RegisterBrokerRecord(). + setFenced(false), expectedVersion), + clusterControl.brokerRegistrations().get(1).toRecord(options)); + assertEquals(new ApiMessageAndVersion(new RegisterBrokerRecord(). setBrokerEpoch(100).setBrokerId(2).setRack(null). setEndPoints(new BrokerEndpointCollection(Collections.singleton( new BrokerEndpoint().setSecurityProtocol(SecurityProtocol.PLAINTEXT.id). setPort((short) 9094). setName("PLAINTEXT"). setHost("example.com")).iterator())). - setFenced(true), expectedVersion))), - clusterControl.iterator(Long.MAX_VALUE)); + setFenced(true), expectedVersion), + clusterControl.brokerRegistrations().get(2).toRecord(options)); } - @Test public void testRegistrationWithUnsupportedMetadataVersion() { SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext()); diff --git a/metadata/src/test/java/org/apache/kafka/controller/ConfigurationControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/ConfigurationControlManagerTest.java index 02bd6e4d7e969..bbca7bfbf6b67 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/ConfigurationControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/ConfigurationControlManagerTest.java @@ -135,14 +135,6 @@ public void testReplay() throws Exception { setName("def").setValue("blah")); assertEquals(toMap(entry("abc", "x,y,z"), entry("def", "blah")), manager.getConfigs(MYTOPIC)); - RecordTestUtils.assertBatchIteratorContains(asList( - asList(new ApiMessageAndVersion(new ConfigRecord(). - setResourceType(TOPIC.id()).setResourceName("mytopic"). - setName("abc").setValue("x,y,z"), CONFIG_RECORD.highestSupportedVersion()), - new ApiMessageAndVersion(new ConfigRecord(). - setResourceType(TOPIC.id()).setResourceName("mytopic"). - setName("def").setValue("blah"), CONFIG_RECORD.highestSupportedVersion()))), - manager.iterator(Long.MAX_VALUE)); } @Test diff --git a/metadata/src/test/java/org/apache/kafka/controller/FeatureControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/FeatureControlManagerTest.java index 2478f4ce164a4..8345b4e314955 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/FeatureControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/FeatureControlManagerTest.java @@ -21,6 +21,7 @@ import java.util.Arrays; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Optional; @@ -202,25 +203,7 @@ public void testUpdateFeaturesErrorCases() { } @Test - public void testFeatureControlIteratorWithOldMetadataVersion() throws Exception { - // We require minimum of IBP_3_3_IV0 to write metadata version in the snapshot. - - LogContext logContext = new LogContext(); - SnapshotRegistry snapshotRegistry = new SnapshotRegistry(logContext); - FeatureControlManager manager = new FeatureControlManager.Builder() - .setLogContext(logContext) - .setSnapshotRegistry(snapshotRegistry) - .setMetadataVersion(MetadataVersion.IBP_3_2_IV0) - .build(); - - RecordTestUtils.assertBatchIteratorContains( - Collections.emptyList(), - manager.iterator(Long.MAX_VALUE) - ); - } - - @Test - public void testFeatureControlIterator() throws Exception { + public void testReplayRecords() throws Exception { LogContext logContext = new LogContext(); SnapshotRegistry snapshotRegistry = new SnapshotRegistry(logContext); FeatureControlManager manager = new FeatureControlManager.Builder(). @@ -233,17 +216,12 @@ public void testFeatureControlIterator() throws Exception { updateFeatures(updateMap("foo", 5, "bar", 1), Collections.emptyMap(), Collections.emptyMap(), false); RecordTestUtils.replayAll(manager, result.records()); - RecordTestUtils.assertBatchIteratorContains(Arrays.asList( - Arrays.asList(new ApiMessageAndVersion(new FeatureLevelRecord(). - setName("metadata.version"). - setFeatureLevel((short) 4), (short) 0)), - Arrays.asList(new ApiMessageAndVersion(new FeatureLevelRecord(). - setName("foo"). - setFeatureLevel((short) 5), (short) 0)), - Arrays.asList(new ApiMessageAndVersion(new FeatureLevelRecord(). - setName("bar"). - setFeatureLevel((short) 1), (short) 0))), - manager.iterator(Long.MAX_VALUE)); + assertEquals(MetadataVersion.IBP_3_3_IV0, manager.metadataVersion()); + assertEquals(Optional.of((short) 5), manager.finalizedFeatures(Long.MAX_VALUE).get("foo")); + assertEquals(Optional.of((short) 1), manager.finalizedFeatures(Long.MAX_VALUE).get("bar")); + assertEquals(new HashSet<>(Arrays.asList( + MetadataVersion.FEATURE_NAME, "foo", "bar")), + manager.finalizedFeatures(Long.MAX_VALUE).featureNames()); } private static final FeatureControlManager.Builder TEST_MANAGER_BUILDER1 = diff --git a/metadata/src/test/java/org/apache/kafka/controller/ProducerIdControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/ProducerIdControlManagerTest.java index 80c5c505ae0eb..d0c16d1e3b0d7 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/ProducerIdControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/ProducerIdControlManagerTest.java @@ -26,20 +26,14 @@ import org.apache.kafka.common.security.auth.SecurityProtocol; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; -import org.apache.kafka.server.common.ApiMessageAndVersion; import org.apache.kafka.server.common.MetadataVersion; import org.apache.kafka.server.common.ProducerIdsBlock; import org.apache.kafka.timeline.SnapshotRegistry; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; -import java.util.Iterator; -import java.util.List; - import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertThrows; -import static org.junit.jupiter.api.Assertions.assertTrue; public class ProducerIdControlManagerTest { @@ -149,29 +143,11 @@ public void testMaxValue() { } @Test - public void testSnapshotIterator() { - ProducerIdsBlock range = null; + public void testGenerateProducerIds() { for (int i = 0; i < 100; i++) { - range = generateProducerIds(producerIdControlManager, i % 4, 100); - } - - Iterator> snapshotIterator = producerIdControlManager.iterator(Long.MAX_VALUE); - assertTrue(snapshotIterator.hasNext()); - List batch = snapshotIterator.next(); - assertEquals(1, batch.size(), "Producer IDs record batch should only contain a single record"); - assertEquals(range.firstProducerId() + range.size(), ((ProducerIdsRecord) batch.get(0).message()).nextProducerId()); - assertFalse(snapshotIterator.hasNext(), "Producer IDs iterator should only contain a single batch"); - - ProducerIdControlManager newProducerIdManager = new ProducerIdControlManager(clusterControl, snapshotRegistry); - snapshotIterator = producerIdControlManager.iterator(Long.MAX_VALUE); - while (snapshotIterator.hasNext()) { - snapshotIterator.next().forEach(message -> newProducerIdManager.replay((ProducerIdsRecord) message.message())); + generateProducerIds(producerIdControlManager, i % 4, 100); } - - // Verify that after reloading state from this "snapshot", we don't produce any overlapping IDs - long lastProducerID = range.firstProducerId() + range.size() - 1; - range = generateProducerIds(producerIdControlManager, 1, 100); - assertTrue(range.firstProducerId() > lastProducerID); + assertEquals(new ProducerIdsBlock(3, 100000, 1000), producerIdControlManager.nextProducerBlock()); } static ProducerIdsBlock generateProducerIds( diff --git a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java index 10921c77fbd7d..9d0508d876ac9 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java @@ -31,8 +31,6 @@ import java.util.OptionalInt; import java.util.OptionalLong; import java.util.Set; -import java.util.Spliterator; -import java.util.Spliterators; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; @@ -42,7 +40,6 @@ import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.IntStream; -import java.util.stream.StreamSupport; import org.apache.kafka.common.Uuid; import org.apache.kafka.common.errors.BrokerIdNotRegisteredException; @@ -85,27 +82,22 @@ import org.apache.kafka.common.metadata.TopicRecord; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.requests.ApiError; -import org.apache.kafka.common.utils.BufferSupplier; import org.apache.kafka.controller.QuorumController.ConfigResourceExistenceChecker; import org.apache.kafka.metadata.BrokerHeartbeatReply; +import org.apache.kafka.metadata.BrokerRegistrationFencingChange; import org.apache.kafka.metadata.BrokerRegistrationReply; import org.apache.kafka.metadata.FinalizedControllerFeatures; -import org.apache.kafka.metadata.MetadataRecordSerde; import org.apache.kafka.metadata.PartitionRegistration; -import org.apache.kafka.metadata.RecordTestUtils; import org.apache.kafka.metadata.authorizer.StandardAuthorizer; import org.apache.kafka.metadata.bootstrap.BootstrapMetadata; import org.apache.kafka.metadata.util.BatchFileWriter; import org.apache.kafka.metalog.LocalLogManager; import org.apache.kafka.metalog.LocalLogManagerTestEnv; -import org.apache.kafka.raft.Batch; import org.apache.kafka.raft.OffsetAndEpoch; import org.apache.kafka.server.common.ApiMessageAndVersion; import org.apache.kafka.server.common.MetadataVersion; import org.apache.kafka.snapshot.FileRawSnapshotReader; -import org.apache.kafka.snapshot.SnapshotReader; import org.apache.kafka.snapshot.RawSnapshotReader; -import org.apache.kafka.snapshot.RecordsSnapshotReader; import org.apache.kafka.snapshot.Snapshots; import org.apache.kafka.test.TestUtils; import org.junit.jupiter.api.Disabled; @@ -128,7 +120,6 @@ import static org.junit.jupiter.api.Assertions.assertSame; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; -import static org.junit.jupiter.api.Assertions.fail; @Timeout(value = 40) @@ -669,324 +660,43 @@ public void testSnapshotSaveAndLoad() throws Throwable { fooId = fooData.topics().find("foo").topicId(); active.allocateProducerIds(ANONYMOUS_CONTEXT, new AllocateProducerIdsRequestData().setBrokerId(0).setBrokerEpoch(brokerEpochs.get(0))).get(); - long snapshotLogOffset = active.beginWritingSnapshot().get(); - reader = logEnv.waitForSnapshot(snapshotLogOffset); - SnapshotReader snapshot = createSnapshotReader(reader); - assertEquals(snapshotLogOffset, snapshot.lastContainedLogOffset()); - checkSnapshotContent(expectedSnapshotContent(fooId, brokerEpochs), snapshot); - } - - try ( - LocalLogManagerTestEnv logEnv = new LocalLogManagerTestEnv.Builder(3). - setSnapshotReader(reader). - build(); - QuorumControllerTestEnv controlEnv = new QuorumControllerTestEnv.Builder(logEnv). - setControllerBuilderInitializer(controllerBuilder -> { - controllerBuilder.setConfigSchema(SCHEMA); - }). - build(); - ) { - QuorumController active = controlEnv.activeController(); - long snapshotLogOffset = active.beginWritingSnapshot().get(); - SnapshotReader snapshot = createSnapshotReader( - logEnv.waitForSnapshot(snapshotLogOffset) - ); - assertEquals(snapshotLogOffset, snapshot.lastContainedLogOffset()); - checkSnapshotContent(expectedSnapshotContent(fooId, brokerEpochs), snapshot); - } - } - - @Test - public void testSnapshotConfiguration() throws Throwable { - final int numBrokers = 4; - final int maxNewRecordBytes = 4; - Map brokerEpochs = new HashMap<>(); - Uuid fooId; - try ( - LocalLogManagerTestEnv logEnv = new LocalLogManagerTestEnv.Builder(3). - build(); - QuorumControllerTestEnv controlEnv = new QuorumControllerTestEnv.Builder(logEnv). - setControllerBuilderInitializer(controllerBuilder -> { - controllerBuilder.setConfigSchema(SCHEMA); - controllerBuilder.setSnapshotMaxNewRecordBytes(maxNewRecordBytes); - controllerBuilder.setBootstrapMetadata(SIMPLE_BOOTSTRAP); - }). - build(); - ) { - QuorumController active = controlEnv.activeController(); - for (int i = 0; i < numBrokers; i++) { - BrokerRegistrationReply reply = active.registerBroker(ANONYMOUS_CONTEXT, - new BrokerRegistrationRequestData(). - setBrokerId(i). - setRack(null). - setClusterId(active.clusterId()). - setFeatures(brokerFeatures(MetadataVersion.IBP_3_0_IV1, MetadataVersion.IBP_3_3_IV3)). - setIncarnationId(Uuid.fromString("kxAT73dKQsitIedpiPtwB" + i)). - setListeners(new ListenerCollection(Arrays.asList(new Listener(). - setName("PLAINTEXT").setHost("localhost"). - setPort(9092 + i)).iterator()))).get(); - brokerEpochs.put(i, reply.epoch()); - } - for (int i = 0; i < numBrokers - 1; i++) { - assertEquals(new BrokerHeartbeatReply(true, false, false, false), - active.processBrokerHeartbeat(ANONYMOUS_CONTEXT, new BrokerHeartbeatRequestData(). - setWantFence(false).setBrokerEpoch(brokerEpochs.get(i)). - setBrokerId(i).setCurrentMetadataOffset(100000L)).get()); - } - CreateTopicsResponseData fooData = active.createTopics(ANONYMOUS_CONTEXT, - new CreateTopicsRequestData().setTopics( - new CreatableTopicCollection(Collections.singleton( - new CreatableTopic().setName("foo").setNumPartitions(-1). - setReplicationFactor((short) -1). - setAssignments(new CreatableReplicaAssignmentCollection( - Arrays.asList(new CreatableReplicaAssignment(). - setPartitionIndex(0). - setBrokerIds(Arrays.asList(0, 1, 2)), - new CreatableReplicaAssignment(). - setPartitionIndex(1). - setBrokerIds(Arrays.asList(1, 2, 0))). - iterator()))).iterator())), - Collections.singleton("foo")).get(); - fooId = fooData.topics().find("foo").topicId(); - active.allocateProducerIds(ANONYMOUS_CONTEXT, - new AllocateProducerIdsRequestData().setBrokerId(0).setBrokerEpoch(brokerEpochs.get(0))).get(); - - SnapshotReader snapshot = createSnapshotReader(logEnv.waitForLatestSnapshot()); - checkSnapshotSubcontent( - expectedSnapshotContent(fooId, brokerEpochs), - snapshot - ); + controlEnv.close(); + assertEquals(generateTestRecords(fooId, brokerEpochs), logEnv.allRecords()); } } - @Test - public void testSnapshotOnlyAfterConfiguredMinBytes() throws Throwable { - final int numBrokers = 4; - final int maxNewRecordBytes = 1000; - Map brokerEpochs = new HashMap<>(); - try ( - LocalLogManagerTestEnv logEnv = new LocalLogManagerTestEnv.Builder(3). - build(); - QuorumControllerTestEnv controlEnv = new QuorumControllerTestEnv.Builder(logEnv). - setControllerBuilderInitializer(controllerBuilder -> { - controllerBuilder.setConfigSchema(SCHEMA); - controllerBuilder.setSnapshotMaxNewRecordBytes(maxNewRecordBytes); - }). - build(); - ) { - QuorumController active = controlEnv.activeController(); - for (int i = 0; i < numBrokers; i++) { - BrokerRegistrationReply reply = active.registerBroker(ANONYMOUS_CONTEXT, - new BrokerRegistrationRequestData(). - setBrokerId(i). - setRack(null). - setClusterId(active.clusterId()). - setFeatures(brokerFeatures(MetadataVersion.IBP_3_0_IV1, MetadataVersion.IBP_3_4_IV0)). - setIncarnationId(Uuid.fromString("kxAT73dKQsitIedpiPtwB" + i)). - setListeners(new ListenerCollection(Arrays.asList(new Listener(). - setName("PLAINTEXT").setHost("localhost"). - setPort(9092 + i)).iterator()))).get(); - brokerEpochs.put(i, reply.epoch()); - assertEquals(new BrokerHeartbeatReply(true, false, false, false), - active.processBrokerHeartbeat(ANONYMOUS_CONTEXT, new BrokerHeartbeatRequestData(). - setWantFence(false).setBrokerEpoch(brokerEpochs.get(i)). - setBrokerId(i).setCurrentMetadataOffset(100000L)).get()); - } - - assertTrue(logEnv.appendedBytes() < maxNewRecordBytes, - String.format("%s appended bytes is not less than %s max new record bytes", - logEnv.appendedBytes(), - maxNewRecordBytes)); - - // Keep creating topic until we reached the max bytes limit - int counter = 0; - while (logEnv.appendedBytes() < maxNewRecordBytes) { - counter += 1; - String topicName = String.format("foo-%s", counter); - active.createTopics(ANONYMOUS_CONTEXT, new CreateTopicsRequestData().setTopics( - new CreatableTopicCollection(Collections.singleton( - new CreatableTopic().setName(topicName).setNumPartitions(-1). - setReplicationFactor((short) -1). - setAssignments(new CreatableReplicaAssignmentCollection( - Arrays.asList(new CreatableReplicaAssignment(). - setPartitionIndex(0). - setBrokerIds(Arrays.asList(0, 1, 2)), - new CreatableReplicaAssignment(). - setPartitionIndex(1). - setBrokerIds(Arrays.asList(1, 2, 0))). - iterator()))).iterator())), - Collections.singleton(topicName)).get(60, TimeUnit.SECONDS); - } - logEnv.waitForLatestSnapshot(); - } - } - - @Test - public void testSnapshotAfterConfiguredMaxInterval() throws Throwable { - final int numBrokers = 4; - Map brokerEpochs = new HashMap<>(); - try ( - LocalLogManagerTestEnv logEnv = new LocalLogManagerTestEnv.Builder(3). - build(); - QuorumControllerTestEnv controlEnv = new QuorumControllerTestEnv.Builder(logEnv). - setControllerBuilderInitializer(controllerBuilder -> { - controllerBuilder.setConfigSchema(SCHEMA); - controllerBuilder.setSnapshotMaxIntervalMs(100); - // Disable snapshot generation due to bytes committed - controllerBuilder.setSnapshotMaxNewRecordBytes(Long.MAX_VALUE); - }). - build(); - ) { - QuorumController active = controlEnv.activeController(); - for (int i = 0; i < numBrokers; i++) { - BrokerRegistrationReply reply = active.registerBroker(ANONYMOUS_CONTEXT, - new BrokerRegistrationRequestData(). - setBrokerId(i). - setRack(null). - setClusterId(active.clusterId()). - setFeatures(brokerFeatures(MetadataVersion.IBP_3_0_IV1, MetadataVersion.IBP_3_4_IV0)). - setIncarnationId(Uuid.fromString("kxAT73dKQsitIedpiPtwB" + i)). - setListeners(new ListenerCollection(Arrays.asList(new Listener(). - setName("PLAINTEXT").setHost("localhost"). - setPort(9092 + i)).iterator()))).get(); - brokerEpochs.put(i, reply.epoch()); - assertEquals(new BrokerHeartbeatReply(true, false, false, false), - active.processBrokerHeartbeat(ANONYMOUS_CONTEXT, new BrokerHeartbeatRequestData(). - setWantFence(false).setBrokerEpoch(brokerEpochs.get(i)). - setBrokerId(i).setCurrentMetadataOffset(100000L)).get()); - } - - logEnv.waitForLatestSnapshot(); - } - } - - @Test - public void testSnapshotAfterRepeatedResign() throws Throwable { - final int numBrokers = 4; - final int maxNewRecordBytes = 1000; - Map brokerEpochs = new HashMap<>(); - try ( - LocalLogManagerTestEnv logEnv = new LocalLogManagerTestEnv.Builder(1). - build(); - QuorumControllerTestEnv controlEnv = new QuorumControllerTestEnv.Builder(logEnv). - setControllerBuilderInitializer(controllerBuilder -> { - controllerBuilder.setConfigSchema(SCHEMA); - controllerBuilder.setSnapshotMaxNewRecordBytes(maxNewRecordBytes); - }). - build(); - ) { - QuorumController active = controlEnv.activeController(); - for (int i = 0; i < numBrokers; i++) { - BrokerRegistrationReply reply = active.registerBroker(ANONYMOUS_CONTEXT, - new BrokerRegistrationRequestData(). - setBrokerId(i). - setRack(null). - setClusterId(active.clusterId()). - setFeatures(brokerFeatures(MetadataVersion.IBP_3_0_IV1, MetadataVersion.IBP_3_4_IV0)). - setIncarnationId(Uuid.fromString("kxAT73dKQsitIedpiPtwB" + i)). - setListeners(new ListenerCollection(Arrays.asList(new Listener(). - setName("PLAINTEXT").setHost("localhost"). - setPort(9092 + i)).iterator()))).get(); - brokerEpochs.put(i, reply.epoch()); - assertEquals(new BrokerHeartbeatReply(true, false, false, false), - active.processBrokerHeartbeat(ANONYMOUS_CONTEXT, new BrokerHeartbeatRequestData(). - setWantFence(false).setBrokerEpoch(brokerEpochs.get(i)). - setBrokerId(i).setCurrentMetadataOffset(100000L)).get()); - } - - assertTrue(logEnv.appendedBytes() < maxNewRecordBytes, - String.format("%s appended bytes is not less than %s max new record bytes", - logEnv.appendedBytes(), - maxNewRecordBytes)); - - // Keep creating topic and resign leader until we reached the max bytes limit - int counter = 0; - while (logEnv.appendedBytes() < maxNewRecordBytes) { - active = controlEnv.activeController(); - - counter += 1; - String topicName = String.format("foo-%s", counter); - active.createTopics(ANONYMOUS_CONTEXT, new CreateTopicsRequestData().setTopics( - new CreatableTopicCollection(Collections.singleton( - new CreatableTopic().setName(topicName).setNumPartitions(-1). - setReplicationFactor((short) -1). - setAssignments(new CreatableReplicaAssignmentCollection( - Arrays.asList(new CreatableReplicaAssignment(). - setPartitionIndex(0). - setBrokerIds(Arrays.asList(0, 1, 2)), - new CreatableReplicaAssignment(). - setPartitionIndex(1). - setBrokerIds(Arrays.asList(1, 2, 0))). - iterator()))).iterator())), - Collections.singleton(topicName)).get(60, TimeUnit.SECONDS); - - LocalLogManager activeLocalLogManager = logEnv.logManagers().get(active.nodeId()); - activeLocalLogManager.resign(activeLocalLogManager.leaderAndEpoch().epoch()); - } - logEnv.waitForLatestSnapshot(); - } - } - - private SnapshotReader createSnapshotReader(RawSnapshotReader reader) { - return RecordsSnapshotReader.of( - reader, - new MetadataRecordSerde(), - BufferSupplier.create(), - Integer.MAX_VALUE, - true - ); - } - - private List expectedSnapshotContent(Uuid fooId, Map brokerEpochs) { + private List generateTestRecords(Uuid fooId, Map brokerEpochs) { return Arrays.asList( new ApiMessageAndVersion(new FeatureLevelRecord(). setName(MetadataVersion.FEATURE_NAME). setFeatureLevel(MetadataVersion.IBP_3_3_IV3.featureLevel()), (short) 0), - new ApiMessageAndVersion(new TopicRecord(). - setName("foo").setTopicId(fooId), (short) 0), - new ApiMessageAndVersion(new PartitionRecord().setPartitionId(0). - setTopicId(fooId).setReplicas(Arrays.asList(0, 1, 2)). - setIsr(Arrays.asList(0, 1, 2)).setRemovingReplicas(Collections.emptyList()). - setAddingReplicas(Collections.emptyList()).setLeader(0).setLeaderEpoch(0). - setPartitionEpoch(0), (short) 0), - new ApiMessageAndVersion(new PartitionRecord().setPartitionId(1). - setTopicId(fooId).setReplicas(Arrays.asList(1, 2, 0)). - setIsr(Arrays.asList(1, 2, 0)).setRemovingReplicas(Collections.emptyList()). - setAddingReplicas(Collections.emptyList()).setLeader(1).setLeaderEpoch(0). - setPartitionEpoch(0), (short) 0), new ApiMessageAndVersion(new RegisterBrokerRecord(). setBrokerId(0).setBrokerEpoch(brokerEpochs.get(0)). setIncarnationId(Uuid.fromString("kxAT73dKQsitIedpiPtwB0")). - setEndPoints( - new BrokerEndpointCollection( - Arrays.asList( - new BrokerEndpoint().setName("PLAINTEXT").setHost("localhost"). - setPort(9092).setSecurityProtocol((short) 0)).iterator())). + setEndPoints(new BrokerEndpointCollection( + Arrays.asList(new BrokerEndpoint().setName("PLAINTEXT").setHost("localhost"). + setPort(9092).setSecurityProtocol((short) 0)).iterator())). setFeatures(registrationFeatures(MetadataVersion.IBP_3_0_IV1, MetadataVersion.IBP_3_3_IV3)). setRack(null). - setFenced(false), (short) 1), + setFenced(true), (short) 1), new ApiMessageAndVersion(new RegisterBrokerRecord(). setBrokerId(1).setBrokerEpoch(brokerEpochs.get(1)). setIncarnationId(Uuid.fromString("kxAT73dKQsitIedpiPtwB1")). - setEndPoints( - new BrokerEndpointCollection( - Arrays.asList( - new BrokerEndpoint().setName("PLAINTEXT").setHost("localhost"). - setPort(9093).setSecurityProtocol((short) 0)).iterator())). + setEndPoints(new BrokerEndpointCollection(Arrays.asList( + new BrokerEndpoint().setName("PLAINTEXT").setHost("localhost"). + setPort(9093).setSecurityProtocol((short) 0)).iterator())). setFeatures(registrationFeatures(MetadataVersion.IBP_3_0_IV1, MetadataVersion.IBP_3_3_IV3)). setRack(null). - setFenced(false), (short) 1), + setFenced(true), (short) 1), new ApiMessageAndVersion(new RegisterBrokerRecord(). setBrokerId(2).setBrokerEpoch(brokerEpochs.get(2)). setIncarnationId(Uuid.fromString("kxAT73dKQsitIedpiPtwB2")). - setEndPoints( - new BrokerEndpointCollection( - Arrays.asList( - new BrokerEndpoint().setName("PLAINTEXT").setHost("localhost"). - setPort(9094).setSecurityProtocol((short) 0)).iterator())). + setEndPoints(new BrokerEndpointCollection( + Arrays.asList(new BrokerEndpoint().setName("PLAINTEXT").setHost("localhost"). + setPort(9094).setSecurityProtocol((short) 0)).iterator())). setFeatures(registrationFeatures(MetadataVersion.IBP_3_0_IV1, MetadataVersion.IBP_3_3_IV3)). setRack(null). - setFenced(false), (short) 1), + setFenced(true), (short) 1), new ApiMessageAndVersion(new RegisterBrokerRecord(). setBrokerId(3).setBrokerEpoch(brokerEpochs.get(3)). setIncarnationId(Uuid.fromString("kxAT73dKQsitIedpiPtwB3")). @@ -994,59 +704,36 @@ private List expectedSnapshotContent(Uuid fooId, Map expected, - Iterator> iterator - ) throws Exception { - RecordTestUtils.assertBatchIteratorContains( - Arrays.asList(expected), - Arrays.asList( - StreamSupport.stream(Spliterators.spliteratorUnknownSize(iterator, Spliterator.ORDERED), false) - .flatMap(batch -> batch.records().stream()) - .collect(Collectors.toList()) - ).iterator() - ); - } - - /** - * This function checks that the iterator is a subset of the expected list. - * - * This is needed because when generating snapshots through configuration is difficult to control exactly when a - * snapshot will be generated and which committed offset will be included in the snapshot. - */ - private void checkSnapshotSubcontent( - List expected, - Iterator> iterator - ) throws Exception { - RecordTestUtils.deepSortRecords(expected); - - List actual = StreamSupport - .stream(Spliterators.spliteratorUnknownSize(iterator, Spliterator.ORDERED), false) - .flatMap(batch -> batch.records().stream()) - .collect(Collectors.toList()); - - RecordTestUtils.deepSortRecords(actual); - - int expectedIndex = 0; - for (ApiMessageAndVersion current : actual) { - while (expectedIndex < expected.size() && !expected.get(expectedIndex).equals(current)) { - expectedIndex += 1; - } - - if (expectedIndex >= expected.size()) { - fail("Failed to find record " + current + " in the expected record set: " + expected); - } - - expectedIndex += 1; - } + setNextProducerId(1000), (short) 0)); } /** @@ -1380,7 +1067,7 @@ public void testQuorumControllerCompletesAuthorizerInitialLoad() throws Throwabl }). build() ) { - logEnv.appendInitialRecords(expectedSnapshotContent(FOO_ID, ALL_ZERO_BROKER_EPOCHS)); + logEnv.appendInitialRecords(generateTestRecords(FOO_ID, ALL_ZERO_BROKER_EPOCHS)); logEnv.logManagers().forEach(m -> m.setMaxReadOffset(2)); try ( QuorumControllerTestEnv controlEnv = new QuorumControllerTestEnv.Builder(logEnv). diff --git a/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java index c81f25e4c000a..0997792cd90f6 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java @@ -58,7 +58,6 @@ import org.apache.kafka.common.metadata.BrokerRegistrationChangeRecord; import org.apache.kafka.common.metadata.ConfigRecord; import org.apache.kafka.common.metadata.PartitionChangeRecord; -import org.apache.kafka.common.metadata.PartitionRecord; import org.apache.kafka.common.metadata.RegisterBrokerRecord; import org.apache.kafka.common.metadata.RemoveTopicRecord; import org.apache.kafka.common.metadata.TopicRecord; @@ -519,16 +518,6 @@ public void testCreateTopics() throws Exception { setErrorCode(Errors.TOPIC_ALREADY_EXISTS.code()). setErrorMessage("Topic 'foo' already exists.")); assertEquals(expectedResponse4, result4.response()); - Uuid fooId = result3.response().topics().find("foo").topicId(); - RecordTestUtils.assertBatchIteratorContains(asList( - asList(new ApiMessageAndVersion(new PartitionRecord(). - setPartitionId(0).setTopicId(fooId). - setReplicas(asList(1, 2, 0)).setIsr(asList(1, 2, 0)). - setRemovingReplicas(Collections.emptyList()).setAddingReplicas(Collections.emptyList()).setLeader(1). - setLeaderEpoch(0).setPartitionEpoch(0), (short) 0), - new ApiMessageAndVersion(new TopicRecord(). - setTopicId(fooId).setName("foo"), (short) 0))), - ctx.replicationControl.iterator(Long.MAX_VALUE)); } @Test diff --git a/metadata/src/test/java/org/apache/kafka/controller/SnapshotGeneratorTest.java b/metadata/src/test/java/org/apache/kafka/controller/SnapshotGeneratorTest.java deleted file mode 100644 index f7fa18f20a4cf..0000000000000 --- a/metadata/src/test/java/org/apache/kafka/controller/SnapshotGeneratorTest.java +++ /dev/null @@ -1,112 +0,0 @@ -/* - * 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.Uuid; -import org.apache.kafka.common.config.ConfigResource; -import org.apache.kafka.common.memory.MemoryPool; -import org.apache.kafka.common.metadata.ConfigRecord; -import org.apache.kafka.common.metadata.TopicRecord; -import org.apache.kafka.common.record.CompressionType; -import org.apache.kafka.common.utils.LogContext; -import org.apache.kafka.common.utils.MockTime; -import org.apache.kafka.controller.SnapshotGenerator.Section; -import org.apache.kafka.metadata.MetadataRecordSerde; -import org.apache.kafka.raft.OffsetAndEpoch; -import org.apache.kafka.server.common.ApiMessageAndVersion; -import org.apache.kafka.snapshot.SnapshotWriter; -import org.apache.kafka.snapshot.MockRawSnapshotWriter; -import org.apache.kafka.snapshot.RawSnapshotWriter; -import org.apache.kafka.snapshot.RecordsSnapshotWriter; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.Timeout; - -import java.util.Arrays; -import java.util.List; -import java.util.OptionalLong; -import java.util.Optional; - -import static org.apache.kafka.raft.KafkaRaftClient.MAX_BATCH_SIZE_BYTES; -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertFalse; -import static org.junit.jupiter.api.Assertions.assertTrue; - - -@Timeout(40) -public class SnapshotGeneratorTest { - private static final List> BATCHES; - - static { - BATCHES = Arrays.asList( - Arrays.asList(new ApiMessageAndVersion(new TopicRecord(). - setName("foo").setTopicId(Uuid.randomUuid()), (short) 0)), - Arrays.asList(new ApiMessageAndVersion(new TopicRecord(). - setName("bar").setTopicId(Uuid.randomUuid()), (short) 0)), - Arrays.asList(new ApiMessageAndVersion(new TopicRecord(). - setName("baz").setTopicId(Uuid.randomUuid()), (short) 0)), - Arrays.asList(new ApiMessageAndVersion(new ConfigRecord(). - setResourceName("foo").setResourceType(ConfigResource.Type.TOPIC.id()). - setName("retention.ms").setValue("10000000"), (short) 0), - new ApiMessageAndVersion(new ConfigRecord(). - setResourceName("foo").setResourceType(ConfigResource.Type.TOPIC.id()). - setName("max.message.bytes").setValue("100000000"), (short) 0)), - Arrays.asList(new ApiMessageAndVersion(new ConfigRecord(). - setResourceName("bar").setResourceType(ConfigResource.Type.TOPIC.id()). - setName("retention.ms").setValue("5000000"), (short) 0))); - } - - @Test - public void testGenerateBatches() throws Exception { - SnapshotWriter writer = createSnapshotWriter(123, 0); - List
sections = Arrays.asList(new Section("replication", - Arrays.asList(BATCHES.get(0), BATCHES.get(1), BATCHES.get(2)).iterator()), - new Section("configuration", - Arrays.asList(BATCHES.get(3), BATCHES.get(4)).iterator())); - SnapshotGenerator generator = new SnapshotGenerator(new LogContext(), - writer, 2, sections); - assertFalse(writer.isFrozen()); - assertEquals(123L, generator.lastContainedLogOffset()); - assertEquals(writer, generator.writer()); - assertEquals(OptionalLong.of(0L), generator.generateBatches()); - assertEquals(OptionalLong.of(0L), generator.generateBatches()); - assertFalse(writer.isFrozen()); - assertEquals(OptionalLong.empty(), generator.generateBatches()); - assertTrue(writer.isFrozen()); - } - - private SnapshotWriter createSnapshotWriter( - long committedOffset, - long lastContainedLogTime - ) { - return RecordsSnapshotWriter.createWithHeader( - () -> createNewSnapshot(new OffsetAndEpoch(committedOffset + 1, 1)), - MAX_BATCH_SIZE_BYTES, - MemoryPool.NONE, - new MockTime(), - lastContainedLogTime, - CompressionType.NONE, - new MetadataRecordSerde() - ).get(); - } - - private Optional createNewSnapshot( - OffsetAndEpoch snapshotId - ) { - return Optional.of(new MockRawSnapshotWriter(snapshotId, buffer -> { })); - } -} diff --git a/metadata/src/test/java/org/apache/kafka/image/publisher/SnapshotEmitterTest.java b/metadata/src/test/java/org/apache/kafka/image/publisher/SnapshotEmitterTest.java new file mode 100644 index 0000000000000..ef40d71460438 --- /dev/null +++ b/metadata/src/test/java/org/apache/kafka/image/publisher/SnapshotEmitterTest.java @@ -0,0 +1,206 @@ +/* + * 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.image.publisher; + +import org.apache.kafka.image.MetadataImageTest; +import org.apache.kafka.raft.LeaderAndEpoch; +import org.apache.kafka.raft.OffsetAndEpoch; +import org.apache.kafka.raft.RaftClient; +import org.apache.kafka.server.common.ApiMessageAndVersion; +import org.apache.kafka.snapshot.SnapshotWriter; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; + +import java.util.ArrayList; +import java.util.List; +import java.util.NavigableSet; +import java.util.Optional; +import java.util.OptionalInt; +import java.util.OptionalLong; +import java.util.TreeMap; +import java.util.concurrent.CompletableFuture; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; + + +@Timeout(value = 40) +public class SnapshotEmitterTest { + static class MockRaftClient implements RaftClient { + TreeMap writers = new TreeMap<>(); + + @Override + public void initialize() { + // nothing to do + } + + @Override + public void register(Listener listener) { + // nothing to do + } + + @Override + public void unregister(Listener listener) { + // nothing to do + } + + @Override + public OptionalLong highWatermark() { + return OptionalLong.empty(); + } + + @Override + public LeaderAndEpoch leaderAndEpoch() { + return LeaderAndEpoch.UNKNOWN; + } + + @Override + public OptionalInt nodeId() { + return OptionalInt.empty(); + } + + @Override + public long scheduleAppend(int epoch, List records) { + return 0; + } + + @Override + public long scheduleAtomicAppend(int epoch, List records) { + return 0; + } + + @Override + public CompletableFuture shutdown(int timeoutMs) { + return CompletableFuture.completedFuture(null); + } + + @Override + public void resign(int epoch) { + // nothing to do + } + + @Override + public Optional> createSnapshot( + OffsetAndEpoch snapshotId, + long lastContainedLogTime + ) { + if (writers.containsKey(snapshotId)) { + return Optional.empty(); + } + MockSnapshotWriter writer = new MockSnapshotWriter(snapshotId); + writers.put(snapshotId, writer); + return Optional.of(writer); + } + + @Override + public Optional latestSnapshotId() { + NavigableSet descendingSet = writers.descendingKeySet(); + if (descendingSet.isEmpty()) { + return Optional.empty(); + } else { + return Optional.of(descendingSet.first()); + } + } + + @Override + public void close() throws Exception { + // nothing to do + } + } + + static class MockSnapshotWriter implements SnapshotWriter { + private final OffsetAndEpoch snapshotId; + private boolean frozen = false; + private boolean closed = false; + private final List> batches; + + MockSnapshotWriter(OffsetAndEpoch snapshotId) { + this.snapshotId = snapshotId; + this.batches = new ArrayList<>(); + } + + @Override + public OffsetAndEpoch snapshotId() { + return snapshotId; + } + + @Override + public long lastContainedLogOffset() { + return snapshotId.offset(); + } + + @Override + public int lastContainedLogEpoch() { + return snapshotId.epoch(); + } + + @Override + public boolean isFrozen() { + return frozen; + } + + @Override + public void append(List records) { + batches.add(records); + } + + List> batches() { + List> results = new ArrayList<>(); + batches.forEach(batch -> results.add(new ArrayList<>(batch))); + return results; + } + + @Override + public void freeze() { + frozen = true; + } + + @Override + public void close() { + closed = true; + } + + boolean isClosed() { + return closed; + } + } + + @Test + public void testEmit() throws Exception { + MockRaftClient mockRaftClient = new MockRaftClient(); + SnapshotEmitter emitter = new SnapshotEmitter.Builder(). + setBatchSize(2). + setRaftClient(mockRaftClient). + build(); + emitter.maybeEmit(MetadataImageTest.IMAGE1); + MockSnapshotWriter writer = mockRaftClient.writers.get( + MetadataImageTest.IMAGE1.highestOffsetAndEpoch()); + assertNotNull(writer); + assertEquals(MetadataImageTest.IMAGE1.highestOffsetAndEpoch().offset(), + writer.lastContainedLogOffset()); + assertEquals(MetadataImageTest.IMAGE1.highestOffsetAndEpoch().epoch(), + writer.lastContainedLogEpoch()); + assertTrue(writer.isFrozen()); + assertTrue(writer.isClosed()); + + // Second call to emit does nothing because we already have a snapshot at that offset and epoch. + emitter.maybeEmit(MetadataImageTest.IMAGE1); + assertEquals(1, mockRaftClient.writers.size()); + } +} diff --git a/metadata/src/test/java/org/apache/kafka/image/publisher/SnapshotGeneratorTest.java b/metadata/src/test/java/org/apache/kafka/image/publisher/SnapshotGeneratorTest.java new file mode 100644 index 0000000000000..42fb8d92f6eba --- /dev/null +++ b/metadata/src/test/java/org/apache/kafka/image/publisher/SnapshotGeneratorTest.java @@ -0,0 +1,185 @@ +/* + * 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.image.publisher; + +import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.image.MetadataDelta; +import org.apache.kafka.image.MetadataImage; +import org.apache.kafka.image.MetadataProvenance; +import org.apache.kafka.image.loader.LogDeltaManifest; +import org.apache.kafka.metadata.RecordTestUtils; +import org.apache.kafka.server.fault.FaultHandlerException; +import org.apache.kafka.server.fault.MockFaultHandler; +import org.apache.kafka.test.TestUtils; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; + + +@Timeout(value = 40) +public class SnapshotGeneratorTest { + static class MockEmitter implements SnapshotGenerator.Emitter { + private final CountDownLatch latch = new CountDownLatch(1); + private final List images = new ArrayList<>(); + private RuntimeException problem = null; + + MockEmitter setReady() { + latch.countDown(); + return this; + } + + synchronized MockEmitter setProblem(RuntimeException problem) { + this.problem = problem; + return this; + } + + @Override + public synchronized void maybeEmit(MetadataImage image) { + RuntimeException currentProblem = problem; + if (currentProblem != null) { + throw currentProblem; + } + try { + latch.await(); + } catch (Throwable e) { + throw new RuntimeException(e); + } + images.add(image); + } + + synchronized List images() { + return new ArrayList<>(images); + } + } + + private final static MetadataDelta TEST_DELTA; + + static { + TEST_DELTA = new MetadataDelta.Builder(). + setImage(MetadataImage.EMPTY). + build(); + TEST_DELTA.replay(RecordTestUtils.testRecord(0).message()); + } + + private final static MetadataImage TEST_IMAGE = TEST_DELTA.apply(MetadataProvenance.EMPTY); + + @Test + public void testCreateSnapshot() throws Exception { + MockFaultHandler faultHandler = new MockFaultHandler("SnapshotGenerator"); + MockEmitter emitter = new MockEmitter(); + try (SnapshotGenerator generator = new SnapshotGenerator.Builder(emitter). + setFaultHandler(faultHandler). + setMaxBytesSinceLastSnapshot(200). + setMaxTimeSinceLastSnapshotNs(TimeUnit.DAYS.toNanos(10)). + build()) { + // Publish a log delta batch. This one will not trigger a snapshot yet. + generator.publishLogDelta(TEST_DELTA, TEST_IMAGE, + new LogDeltaManifest(MetadataProvenance.EMPTY, 1, 100, 100)); + // Publish a log delta batch. This will trigger a snapshot. + generator.publishLogDelta(TEST_DELTA, TEST_IMAGE, + new LogDeltaManifest(MetadataProvenance.EMPTY, 1, 100, 100)); + // Publish a log delta batch. This one will be ignored because there are other images + // queued for writing. + generator.publishLogDelta(TEST_DELTA, TEST_IMAGE, + new LogDeltaManifest(MetadataProvenance.EMPTY, 1, 100, 2000)); + assertEquals(Collections.emptyList(), emitter.images()); + emitter.setReady(); + } + assertEquals(Arrays.asList(TEST_IMAGE), emitter.images()); + faultHandler.maybeRethrowFirstException(); + } + + @Test + public void testSnapshotsDisabled() throws Exception { + MockFaultHandler faultHandler = new MockFaultHandler("SnapshotGenerator"); + MockEmitter emitter = new MockEmitter().setReady(); + AtomicReference disabledReason = new AtomicReference<>(); + try (SnapshotGenerator generator = new SnapshotGenerator.Builder(emitter). + setFaultHandler(faultHandler). + setMaxBytesSinceLastSnapshot(1). + setMaxTimeSinceLastSnapshotNs(0). + setDisabledReason(disabledReason). + build()) { + disabledReason.compareAndSet(null, "we are testing disable()"); + // No snapshots are generated because snapshots are disabled. + generator.publishLogDelta(TEST_DELTA, TEST_IMAGE, + new LogDeltaManifest(MetadataProvenance.EMPTY, 1, 100, 100)); + } + assertEquals(Collections.emptyList(), emitter.images()); + faultHandler.maybeRethrowFirstException(); + } + + @Test + public void testTimeBasedSnapshots() throws Exception { + MockFaultHandler faultHandler = new MockFaultHandler("SnapshotGenerator"); + MockEmitter emitter = new MockEmitter().setReady(); + MockTime mockTime = new MockTime(); + try (SnapshotGenerator generator = new SnapshotGenerator.Builder(emitter). + setTime(mockTime). + setFaultHandler(faultHandler). + setMaxBytesSinceLastSnapshot(200). + setMaxTimeSinceLastSnapshotNs(TimeUnit.MINUTES.toNanos(30)). + build()) { + // This image isn't published yet. + generator.publishLogDelta(TEST_DELTA, TEST_IMAGE, + new LogDeltaManifest(MetadataProvenance.EMPTY, 1, 100, 50)); + assertEquals(Collections.emptyList(), emitter.images()); + mockTime.sleep(TimeUnit.MINUTES.toNanos(40)); + // Next image is published because of the time delay. + generator.publishLogDelta(TEST_DELTA, TEST_IMAGE, + new LogDeltaManifest(MetadataProvenance.EMPTY, 1, 100, 50)); + TestUtils.waitForCondition(() -> emitter.images().size() == 1, "images.size == 1"); + // bytesSinceLastSnapshot was reset to 0 by the previous snapshot, + // so this does not trigger a new snapshot. + generator.publishLogDelta(TEST_DELTA, TEST_IMAGE, + new LogDeltaManifest(MetadataProvenance.EMPTY, 1, 100, 150)); + } + assertEquals(Arrays.asList(TEST_IMAGE), emitter.images()); + faultHandler.maybeRethrowFirstException(); + } + + @Test + public void testEmitterProblem() throws Exception { + MockFaultHandler faultHandler = new MockFaultHandler("SnapshotGenerator"); + MockEmitter emitter = new MockEmitter().setProblem(new RuntimeException("oops")); + try (SnapshotGenerator generator = new SnapshotGenerator.Builder(emitter). + setFaultHandler(faultHandler). + setMaxBytesSinceLastSnapshot(200). + build()) { + for (int i = 0; i < 2; i++) { + generator.publishLogDelta(TEST_DELTA, TEST_IMAGE, + new LogDeltaManifest(MetadataProvenance.EMPTY, 1, 10000, 50000)); + } + } + assertEquals(Collections.emptyList(), emitter.images()); + assertNotNull(faultHandler.firstException()); + assertEquals(FaultHandlerException.class, faultHandler.firstException().getClass()); + assertEquals("SnapshotGenerator: KRaft snapshot file generation error: oops", + faultHandler.firstException().getMessage()); + } +} diff --git a/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java b/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java index 3bc07c06af0e2..a90e2687200ef 100644 --- a/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java +++ b/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java @@ -46,6 +46,7 @@ import org.slf4j.LoggerFactory; import java.util.AbstractMap.SimpleImmutableEntry; +import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; import java.util.IdentityHashMap; @@ -318,9 +319,9 @@ synchronized void addSnapshot(RawSnapshotReader newSnapshot) { } /** - * Returns the snapshot whos last offset is the committed offset. + * Returns the snapshot whose last offset is the committed offset. * - * If such snapshot doesn't exists, it waits until it does. + * If such snapshot doesn't exist, it waits until it does. */ synchronized RawSnapshotReader waitForSnapshot(long committedOffset) throws InterruptedException { while (true) { @@ -380,6 +381,20 @@ public SharedLogData setInitialMaxReadOffset(long initialMaxReadOffset) { public long initialMaxReadOffset() { return initialMaxReadOffset; } + + /** + * Return all records in the log as a list. + */ + public synchronized List allRecords() { + List allRecords = new ArrayList<>(); + for (LocalBatch batch : batches.values()) { + if (batch instanceof LocalRecordBatch) { + LocalRecordBatch recordBatch = (LocalRecordBatch) batch; + allRecords.addAll(recordBatch.records); + } + } + return allRecords; + } } private static class MetaLogListenerData { diff --git a/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManagerTestEnv.java b/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManagerTestEnv.java index 1693b62be1aec..d72b7557b48e0 100644 --- a/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManagerTestEnv.java +++ b/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManagerTestEnv.java @@ -140,6 +140,13 @@ private LocalLogManagerTestEnv( this.logManagers = newLogManagers; } + /** + * Return all records in the log as a list. + */ + public List allRecords() { + return shared.allRecords(); + } + /** * Append some records to the log. This method is meant to be called before the * controllers are started, to simulate a pre-existing metadata log. From 5221e33cfdadd51be07183d6ef457752b5ddfa85 Mon Sep 17 00:00:00 2001 From: David Arthur Date: Thu, 15 Dec 2022 14:46:12 -0500 Subject: [PATCH 04/30] Register the migration driver with the controller --- .../scala/kafka/server/ControllerServer.scala | 13 ++++ .../kafka/image/loader/MetadataLoader.java | 8 +++ .../image/publisher/MetadataPublisher.java | 9 +++ .../migration/KRaftMigrationDriver.java | 65 ++++++++++++++----- 4 files changed, 77 insertions(+), 18 deletions(-) diff --git a/core/src/main/scala/kafka/server/ControllerServer.scala b/core/src/main/scala/kafka/server/ControllerServer.scala index 77f9ba546fdd6..70359864cf7ce 100644 --- a/core/src/main/scala/kafka/server/ControllerServer.scala +++ b/core/src/main/scala/kafka/server/ControllerServer.scala @@ -29,6 +29,7 @@ import kafka.server.KafkaConfig.{AlterConfigPolicyClassNameProp, CreateTopicPoli import kafka.server.KafkaRaftServer.BrokerRole import kafka.server.QuotaFactory.QuotaManagers import kafka.utils.{CoreUtils, Logging} +import kafka.zk.{KafkaZkClient, ZkMigrationClient} import org.apache.kafka.common.message.ApiMessageType.ListenerType import org.apache.kafka.common.security.scram.internals.ScramMechanism import org.apache.kafka.common.security.token.delegation.internals.DelegationTokenCache @@ -42,6 +43,7 @@ import org.apache.kafka.server.common.ApiMessageAndVersion import org.apache.kafka.common.config.ConfigException import org.apache.kafka.metadata.authorizer.ClusterMetadataAuthorizer import org.apache.kafka.metadata.bootstrap.BootstrapMetadata +import org.apache.kafka.metadata.migration.{BrokersRpcClient, KRaftMigrationDriver} import org.apache.kafka.server.metrics.KafkaYammerMetrics import org.apache.kafka.server.policy.{AlterConfigPolicy, CreateTopicPolicy} @@ -81,6 +83,7 @@ class ControllerServer( var quotaManagers: QuotaManagers = _ var controllerApis: ControllerApis = _ var controllerApisHandlerPool: KafkaRequestHandlerPool = _ + var migrationDriver: KRaftMigrationDriver = _ private def maybeChangeStatus(from: ProcessStatus, to: ProcessStatus): Boolean = { lock.lock() @@ -112,6 +115,14 @@ class ControllerServer( maybeChangeStatus(STARTING, STARTED) this.logIdent = new LogContext(s"[ControllerServer id=${config.nodeId}] ").logPrefix() + if (config.migrationEnabled) { + val zkClient = KafkaZkClient.createZkClient("KRaft migration", time, config, KafkaServer.zkClientConfigFromKafkaConfig(config)) + val migrationClient = new ZkMigrationClient(zkClient) + val rpcClient: BrokersRpcClient = null + migrationDriver = new KRaftMigrationDriver(config.nodeId, migrationClient, rpcClient) + sharedServer.loader.installPublishers(java.util.Collections.singletonList(migrationDriver)) + } + newGauge("ClusterId", () => clusterId) newGauge("yammer-metrics-count", () => KafkaYammerMetrics.defaultRegistry.allMetrics.size) @@ -267,6 +278,8 @@ class ControllerServer( sharedServer.ensureNotRaftLeader() if (socketServer != null) CoreUtils.swallow(socketServer.stopProcessingRequests(), this) + if (migrationDriver != null) + CoreUtils.swallow(migrationDriver.close(), this) if (controller != null) controller.beginShutdown() if (socketServer != null) diff --git a/metadata/src/main/java/org/apache/kafka/image/loader/MetadataLoader.java b/metadata/src/main/java/org/apache/kafka/image/loader/MetadataLoader.java index 8672c98f2112d..fc99fe27a8b58 100644 --- a/metadata/src/main/java/org/apache/kafka/image/loader/MetadataLoader.java +++ b/metadata/src/main/java/org/apache/kafka/image/loader/MetadataLoader.java @@ -353,6 +353,14 @@ SnapshotManifest loadSnapshot( public void handleLeaderChange(LeaderAndEpoch leaderAndEpoch) { eventQueue.append(() -> { currentLeaderAndEpoch = leaderAndEpoch; + for (MetadataPublisher publisher : publishers) { + try { + publisher.publishLeaderAndEpoch(leaderAndEpoch); + } catch (Throwable e) { + faultHandler.handleFault("Unhandled error publishing the new leader and epoch" + + leaderAndEpoch + " with publisher " + publisher.name(), e); + } + } }); } diff --git a/metadata/src/main/java/org/apache/kafka/image/publisher/MetadataPublisher.java b/metadata/src/main/java/org/apache/kafka/image/publisher/MetadataPublisher.java index 8dfba7a99abd8..5b42347e544f0 100644 --- a/metadata/src/main/java/org/apache/kafka/image/publisher/MetadataPublisher.java +++ b/metadata/src/main/java/org/apache/kafka/image/publisher/MetadataPublisher.java @@ -21,6 +21,7 @@ import org.apache.kafka.image.MetadataImage; import org.apache.kafka.image.loader.LogDeltaManifest; import org.apache.kafka.image.loader.SnapshotManifest; +import org.apache.kafka.raft.LeaderAndEpoch; /** @@ -65,6 +66,14 @@ void publishLogDelta( LogDeltaManifest manifest ); + + /** + * Publish a change in the KRaft leader and epoch. + * + * @param leaderAndEpoch The new leader and epoch. + */ + default void publishLeaderAndEpoch(LeaderAndEpoch leaderAndEpoch) { } + /** * Close this metadata publisher. */ diff --git a/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java b/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java index da24850ac5009..a72a71c18170c 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java @@ -20,15 +20,22 @@ import org.apache.kafka.common.utils.Time; import org.apache.kafka.image.MetadataDelta; import org.apache.kafka.image.MetadataImage; +import org.apache.kafka.image.loader.LogDeltaManifest; +import org.apache.kafka.image.loader.SnapshotManifest; +import org.apache.kafka.image.publisher.MetadataPublisher; +import org.apache.kafka.metadata.BrokerRegistration; import org.apache.kafka.queue.EventQueue; import org.apache.kafka.queue.KafkaEventQueue; +import org.apache.kafka.raft.LeaderAndEpoch; import org.apache.kafka.raft.OffsetAndEpoch; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.Set; import java.util.function.Function; +import java.util.stream.Collectors; import static java.util.concurrent.TimeUnit.NANOSECONDS; import static java.util.concurrent.TimeUnit.SECONDS; @@ -37,7 +44,7 @@ * This class orchestrates and manages the state related to a ZK to KRaft migration. An event thread is used to * serialize events coming from various threads and listeners. */ -public class KRaftMigrationDriver { +public class KRaftMigrationDriver implements MetadataPublisher { private final Time time; private final Logger log; private final int nodeId; @@ -66,18 +73,6 @@ public void start() { eventQueue.prepend(new PollEvent()); } - public void shutdown() throws InterruptedException { - eventQueue.close(); - } - - public void handleLeaderChange(boolean isActive, int epoch) { - eventQueue.append(new KRaftLeaderEvent(isActive, epoch)); - } - - public void publishMetadata(MetadataDelta delta, MetadataImage image) { - eventQueue.append(new MetadataChangeEvent(delta, image)); - } - private void initializeMigrationState() { log.info("Recovering migration state"); apply("Recovery", zkMigrationClient::getOrCreateMigrationRecoveryState); @@ -88,14 +83,23 @@ private void initializeMigrationState() { } private boolean isControllerQuorumReadyForMigration() { - // TODO - return false; + return true; } private boolean areZkBrokersReadyForMigration() { - // TODO: Check available broker registrations and known topic assignments to confirm of - // all Zk brokers are registered before beginning migration. - return false; + Set kraftRegisteredZkBrokers = image.cluster().brokers().values() + .stream() + .filter(BrokerRegistration::isMigratingZkBroker) + .map(BrokerRegistration::id) + .collect(Collectors.toSet()); + Set zkRegisteredZkBrokers = zkMigrationClient.readBrokerIds(); + zkRegisteredZkBrokers.removeAll(kraftRegisteredZkBrokers); + if (zkRegisteredZkBrokers.isEmpty()) { + return true; + } else { + log.info("Still waiting for ZK brokers {} to register with KRaft.", zkRegisteredZkBrokers); + return false; + } } private void apply(String name, Function stateMutator) { @@ -159,6 +163,31 @@ private void transitionTo(MigrationState newState) { migrationState = newState; } + @Override + public String name() { + return "KRaftMigrationDriver"; + } + + @Override + public void publishSnapshot(MetadataDelta delta, MetadataImage newImage, SnapshotManifest manifest) { + eventQueue.append(new MetadataChangeEvent(delta, newImage)); + } + + @Override + public void publishLogDelta(MetadataDelta delta, MetadataImage newImage, LogDeltaManifest manifest) { + eventQueue.append(new MetadataChangeEvent(delta, newImage)); + } + + @Override + public void publishLeaderAndEpoch(LeaderAndEpoch leaderAndEpoch) { + eventQueue.append(new KRaftLeaderEvent(leaderAndEpoch.isLeader(nodeId), leaderAndEpoch.epoch())); + } + + @Override + public void close() throws Exception { + eventQueue.close(); + } + // Events handled by Migration Driver. class PollEvent implements EventQueue.Event { From 88aa6d7186dd3f8a76482d486256c808b897071f Mon Sep 17 00:00:00 2001 From: Akhilesh Chaganti Date: Thu, 15 Dec 2022 18:06:51 -0800 Subject: [PATCH 05/30] Fix build issue and add helper functions to cluster image and delta --- .../org/apache/kafka/image/ClusterDelta.java | 12 ++++++++++++ .../org/apache/kafka/image/ClusterImage.java | 16 ++++++++++++++++ .../kafka/image/loader/MetadataLoader.java | 2 +- .../metadata/migration/KRaftMigrationDriver.java | 10 ++-------- 4 files changed, 31 insertions(+), 9 deletions(-) diff --git a/metadata/src/main/java/org/apache/kafka/image/ClusterDelta.java b/metadata/src/main/java/org/apache/kafka/image/ClusterDelta.java index 39d6fdb3d744c..e0dcb7cc580be 100644 --- a/metadata/src/main/java/org/apache/kafka/image/ClusterDelta.java +++ b/metadata/src/main/java/org/apache/kafka/image/ClusterDelta.java @@ -31,6 +31,8 @@ import java.util.Map; import java.util.Map.Entry; import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; /** @@ -56,6 +58,16 @@ public BrokerRegistration broker(int nodeId) { return image.broker(nodeId); } + public Set newZkBrokers() { + return changedBrokers + .entrySet() + .stream() + .filter(entry -> !image.containsBroker(entry.getKey()) + && entry.getValue().filter(BrokerRegistration::isMigratingZkBroker).isPresent()) + .map(Entry::getKey) + .collect(Collectors.toSet()); + } + public void finishSnapshot() { for (Integer brokerId : image.brokers().keySet()) { if (!changedBrokers.containsKey(brokerId)) { diff --git a/metadata/src/main/java/org/apache/kafka/image/ClusterImage.java b/metadata/src/main/java/org/apache/kafka/image/ClusterImage.java index 05ba74412ba82..b59fa15cfffa1 100644 --- a/metadata/src/main/java/org/apache/kafka/image/ClusterImage.java +++ b/metadata/src/main/java/org/apache/kafka/image/ClusterImage.java @@ -34,9 +34,15 @@ public final class ClusterImage { public static final ClusterImage EMPTY = new ClusterImage(Collections.emptyMap()); private final Map brokers; + private final Map zkBrokers; public ClusterImage(Map brokers) { this.brokers = Collections.unmodifiableMap(brokers); + this.zkBrokers = Collections.unmodifiableMap(brokers + .entrySet() + .stream() + .filter(x -> x.getValue().isMigratingZkBroker()) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue))); } public boolean isEmpty() { @@ -47,10 +53,20 @@ public Map brokers() { return brokers; } + public Map zkBrokers() { + return zkBrokers; + } + public BrokerRegistration broker(int nodeId) { return brokers.get(nodeId); } + + + public boolean containsBroker(int brokerId) { + return brokers.containsKey(brokerId); + } + public void write(ImageWriter writer, ImageWriterOptions options) { for (BrokerRegistration broker : brokers.values()) { writer.write(broker.toRecord(options)); diff --git a/metadata/src/main/java/org/apache/kafka/image/loader/MetadataLoader.java b/metadata/src/main/java/org/apache/kafka/image/loader/MetadataLoader.java index 51bda2a738711..1e4827c733041 100644 --- a/metadata/src/main/java/org/apache/kafka/image/loader/MetadataLoader.java +++ b/metadata/src/main/java/org/apache/kafka/image/loader/MetadataLoader.java @@ -447,7 +447,7 @@ SnapshotManifest loadSnapshot( public void handleLeaderChange(LeaderAndEpoch leaderAndEpoch) { eventQueue.append(() -> { currentLeaderAndEpoch = leaderAndEpoch; - for (MetadataPublisher publisher : publishers) { + for (MetadataPublisher publisher : publishers.values()) { try { publisher.publishLeaderAndEpoch(leaderAndEpoch); } catch (Throwable e) { diff --git a/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java b/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java index a72a71c18170c..fa12daf6a6489 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java @@ -23,7 +23,6 @@ import org.apache.kafka.image.loader.LogDeltaManifest; import org.apache.kafka.image.loader.SnapshotManifest; import org.apache.kafka.image.publisher.MetadataPublisher; -import org.apache.kafka.metadata.BrokerRegistration; import org.apache.kafka.queue.EventQueue; import org.apache.kafka.queue.KafkaEventQueue; @@ -35,7 +34,6 @@ import java.util.Set; import java.util.function.Function; -import java.util.stream.Collectors; import static java.util.concurrent.TimeUnit.NANOSECONDS; import static java.util.concurrent.TimeUnit.SECONDS; @@ -87,12 +85,8 @@ private boolean isControllerQuorumReadyForMigration() { } private boolean areZkBrokersReadyForMigration() { - Set kraftRegisteredZkBrokers = image.cluster().brokers().values() - .stream() - .filter(BrokerRegistration::isMigratingZkBroker) - .map(BrokerRegistration::id) - .collect(Collectors.toSet()); - Set zkRegisteredZkBrokers = zkMigrationClient.readBrokerIds(); + Set kraftRegisteredZkBrokers = image.cluster().zkBrokers().keySet(); + Set zkRegisteredZkBrokers = zkMigrationClient.readBrokerIdsFromTopicAssignments(); zkRegisteredZkBrokers.removeAll(kraftRegisteredZkBrokers); if (zkRegisteredZkBrokers.isEmpty()) { return true; From d52cd952b26d362e5ea50c4c2f03830ba04171cd Mon Sep 17 00:00:00 2001 From: David Arthur Date: Fri, 16 Dec 2022 17:03:41 -0500 Subject: [PATCH 06/30] WIP --- .../scala/kafka/server/ControllerServer.scala | 43 +++++--- .../scala/kafka/zk/ZkMigrationClient.scala | 3 +- .../controller/MigrationControlManager.java | 22 +++++ .../kafka/controller/QuorumController.java | 87 ++++++++++++---- .../org/apache/kafka/image/MetadataDelta.java | 3 + .../kafka/image/loader/LogDeltaManifest.java | 15 +++ .../kafka/image/loader/MetadataLoader.java | 9 +- .../image/publisher/MetadataPublisher.java | 9 -- .../migration/KRaftMigrationDriver.java | 99 +++++++++++++------ .../metadata/migration/ZkRecordConsumer.java | 14 +++ .../image/loader/MetadataLoaderTest.java | 3 +- .../publisher/SnapshotGeneratorTest.java | 17 ++-- 12 files changed, 236 insertions(+), 88 deletions(-) create mode 100644 metadata/src/main/java/org/apache/kafka/controller/MigrationControlManager.java create mode 100644 metadata/src/main/java/org/apache/kafka/metadata/migration/ZkRecordConsumer.java diff --git a/core/src/main/scala/kafka/server/ControllerServer.scala b/core/src/main/scala/kafka/server/ControllerServer.scala index 70359864cf7ce..e6975c4851023 100644 --- a/core/src/main/scala/kafka/server/ControllerServer.scala +++ b/core/src/main/scala/kafka/server/ControllerServer.scala @@ -50,6 +50,25 @@ import org.apache.kafka.server.policy.{AlterConfigPolicy, CreateTopicPolicy} import scala.jdk.CollectionConverters._ import scala.compat.java8.OptionConverters._ + +case class ControllerMigrationSupport( + zkClient: KafkaZkClient, + migrationDriver: KRaftMigrationDriver, + brokersRpcClient: BrokersRpcClient +) { + def shutdown(logging: Logging): Unit = { + if (migrationDriver != null) { + CoreUtils.swallow(migrationDriver.close(), logging) + } + if (zkClient != null) { + CoreUtils.swallow(zkClient.close(), logging) + } + if (brokersRpcClient != null) { + // TODO + } + } +} + /** * A Kafka controller that runs in KRaft (Kafka Raft) mode. */ @@ -83,7 +102,7 @@ class ControllerServer( var quotaManagers: QuotaManagers = _ var controllerApis: ControllerApis = _ var controllerApisHandlerPool: KafkaRequestHandlerPool = _ - var migrationDriver: KRaftMigrationDriver = _ + var migrationSupport: Option[ControllerMigrationSupport] = None private def maybeChangeStatus(from: ProcessStatus, to: ProcessStatus): Boolean = { lock.lock() @@ -115,15 +134,6 @@ class ControllerServer( maybeChangeStatus(STARTING, STARTED) this.logIdent = new LogContext(s"[ControllerServer id=${config.nodeId}] ").logPrefix() - if (config.migrationEnabled) { - val zkClient = KafkaZkClient.createZkClient("KRaft migration", time, config, KafkaServer.zkClientConfigFromKafkaConfig(config)) - val migrationClient = new ZkMigrationClient(zkClient) - val rpcClient: BrokersRpcClient = null - migrationDriver = new KRaftMigrationDriver(config.nodeId, migrationClient, rpcClient) - sharedServer.loader.installPublishers(java.util.Collections.singletonList(migrationDriver)) - } - - newGauge("ClusterId", () => clusterId) newGauge("yammer-metrics-count", () => KafkaYammerMetrics.defaultRegistry.allMetrics.size) @@ -228,6 +238,16 @@ class ControllerServer( doRemoteKraftSetup() } + if (config.migrationEnabled) { + val zkClient = KafkaZkClient.createZkClient("KRaft Migration", time, config, KafkaServer.zkClientConfigFromKafkaConfig(config)) + val migrationClient = new ZkMigrationClient(zkClient) + val rpcClient: BrokersRpcClient = null + val migrationDriver = new KRaftMigrationDriver(config.nodeId, controller.asInstanceOf[QuorumController].zkRecordConsumer(), migrationClient, rpcClient) + sharedServer.loader.installPublishers(java.util.Collections.singletonList(migrationDriver)) + migrationDriver.start() + migrationSupport = Some(ControllerMigrationSupport(zkClient, migrationDriver, rpcClient)) + } + quotaManagers = QuotaFactory.instantiate(config, metrics, time, @@ -278,8 +298,7 @@ class ControllerServer( sharedServer.ensureNotRaftLeader() if (socketServer != null) CoreUtils.swallow(socketServer.stopProcessingRequests(), this) - if (migrationDriver != null) - CoreUtils.swallow(migrationDriver.close(), this) + migrationSupport.foreach(_.shutdown(this)) if (controller != null) controller.beginShutdown() if (socketServer != null) diff --git a/core/src/main/scala/kafka/zk/ZkMigrationClient.scala b/core/src/main/scala/kafka/zk/ZkMigrationClient.scala index d6c2149846413..3a9490bd7e5e0 100644 --- a/core/src/main/scala/kafka/zk/ZkMigrationClient.scala +++ b/core/src/main/scala/kafka/zk/ZkMigrationClient.scala @@ -58,8 +58,7 @@ class ZkMigrationClient(zkClient: KafkaZkClient) extends MigrationClient with Lo override def claimControllerLeadership(state: ZkMigrationLeadershipState): ZkMigrationLeadershipState = { zkClient.tryRegisterKRaftControllerAsActiveController(state.kraftControllerId(), state.kraftControllerEpoch()) match { - case SuccessfulRegistrationResult(_, controllerEpochZkVersion) => - zkClient.getOrCreateMigrationState(state.withControllerZkVersion(controllerEpochZkVersion)) + case SuccessfulRegistrationResult(_, controllerEpochZkVersion) => state.withControllerZkVersion(controllerEpochZkVersion) case FailedRegistrationResult() => state.withControllerZkVersion(-1) } } diff --git a/metadata/src/main/java/org/apache/kafka/controller/MigrationControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/MigrationControlManager.java new file mode 100644 index 0000000000000..3001decb85c81 --- /dev/null +++ b/metadata/src/main/java/org/apache/kafka/controller/MigrationControlManager.java @@ -0,0 +1,22 @@ +package org.apache.kafka.controller; + +import org.apache.kafka.common.metadata.ZkMigrationStateRecord; +import org.apache.kafka.metadata.migration.ZkMigrationState; +import org.apache.kafka.timeline.SnapshotRegistry; +import org.apache.kafka.timeline.TimelineObject; + +public class MigrationControlManager { + private final TimelineObject zkMigrationState; + + MigrationControlManager(SnapshotRegistry snapshotRegistry) { + zkMigrationState = new TimelineObject<>(snapshotRegistry, ZkMigrationState.NONE); + } + + public ZkMigrationState zkMigrationState() { + return zkMigrationState.get(); + } + + void replay(ZkMigrationStateRecord record) { + zkMigrationState.set(ZkMigrationState.of(record.zkMigrationState())); + } +} diff --git a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java index 642d6976dba8c..4fc4c576887d7 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java +++ b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java @@ -48,23 +48,7 @@ import org.apache.kafka.common.message.ListPartitionReassignmentsResponseData; import org.apache.kafka.common.message.UpdateFeaturesRequestData; import org.apache.kafka.common.message.UpdateFeaturesResponseData; -import org.apache.kafka.common.metadata.AccessControlEntryRecord; -import org.apache.kafka.common.metadata.BrokerRegistrationChangeRecord; -import org.apache.kafka.common.metadata.ConfigRecord; -import org.apache.kafka.common.metadata.ClientQuotaRecord; -import org.apache.kafka.common.metadata.FeatureLevelRecord; -import org.apache.kafka.common.metadata.FenceBrokerRecord; -import org.apache.kafka.common.metadata.MetadataRecordType; -import org.apache.kafka.common.metadata.NoOpRecord; -import org.apache.kafka.common.metadata.PartitionChangeRecord; -import org.apache.kafka.common.metadata.PartitionRecord; -import org.apache.kafka.common.metadata.ProducerIdsRecord; -import org.apache.kafka.common.metadata.RegisterBrokerRecord; -import org.apache.kafka.common.metadata.RemoveAccessControlEntryRecord; -import org.apache.kafka.common.metadata.RemoveTopicRecord; -import org.apache.kafka.common.metadata.TopicRecord; -import org.apache.kafka.common.metadata.UnfenceBrokerRecord; -import org.apache.kafka.common.metadata.UnregisterBrokerRecord; +import org.apache.kafka.common.metadata.*; import org.apache.kafka.common.protocol.ApiMessage; import org.apache.kafka.common.quota.ClientQuotaAlteration; import org.apache.kafka.common.quota.ClientQuotaEntity; @@ -78,6 +62,8 @@ import org.apache.kafka.metadata.KafkaConfigSchema; import org.apache.kafka.metadata.authorizer.ClusterMetadataAuthorizer; import org.apache.kafka.metadata.bootstrap.BootstrapMetadata; +import org.apache.kafka.metadata.migration.ZkMigrationState; +import org.apache.kafka.metadata.migration.ZkRecordConsumer; import org.apache.kafka.metadata.placement.ReplicaPlacer; import org.apache.kafka.metadata.placement.StripedReplicaPlacer; import org.apache.kafka.queue.EventQueue.EarliestDeadlineFunction; @@ -566,6 +552,10 @@ ConfigurationControlManager configurationControl() { return configurationControl; } + public ZkRecordConsumer zkRecordConsumer() { + return zkRecordConsumer; + } + CompletableFuture appendReadEvent( String name, OptionalLong deadlineNs, @@ -819,6 +809,63 @@ CompletableFuture appendWriteEvent(String name, return event.future(); } + class MigrationRecordConsumer implements ZkRecordConsumer { + private volatile OffsetAndEpoch highestMigrationRecordOffset; + + class MigrationWriteOperation implements ControllerWriteOperation { + private final List batch; + + MigrationWriteOperation(List batch) { + this.batch = batch; + } + @Override + public ControllerResult generateRecordsAndResult() { + log.info("Migrating batch {}", batch); + return ControllerResult.atomicOf(batch, null); + } + + public void processBatchEndOffset(long offset) { + highestMigrationRecordOffset = new OffsetAndEpoch(offset, curClaimEpoch); + } + } + @Override + public void beginMigration() { + // TODO use KIP-868 transaction + ControllerWriteEvent event = new ControllerWriteEvent<>("Begin ZK Migration", + new MigrationWriteOperation( + Collections.singletonList( + new ApiMessageAndVersion( + new ZkMigrationStateRecord().setZkMigrationState(ZkMigrationState.PRE_MIGRATION.value()), + ZkMigrationStateRecord.LOWEST_SUPPORTED_VERSION) + ))); + queue.append(event); + } + + @Override + public CompletableFuture acceptBatch(List recordBatch) { + if (queue.size() > 100) { // TODO configure this + CompletableFuture future = new CompletableFuture<>(); + future.completeExceptionally(new NotControllerException("Cannot accept migration record batch. Controller queue is too large")); + return future; + } + ControllerWriteEvent batchEvent = new ControllerWriteEvent<>("ZK Migration Batch", + new MigrationWriteOperation(recordBatch)); + queue.append(batchEvent); + return batchEvent.future; + } + + @Override + public OffsetAndEpoch completeMigration() { + // TODO write migration record, use KIP-868 transaction + return highestMigrationRecordOffset; + } + + @Override + public void abortMigration() { + // TODO use KIP-868 transaction + } + } + class QuorumMetaLogListener implements RaftClient.Listener { @Override public void handleCommit(BatchReader reader) { @@ -1343,6 +1390,9 @@ private void replay(ApiMessage message, Optional snapshotId, lon case NO_OP_RECORD: // NoOpRecord is an empty record and doesn't need to be replayed break; + case ZK_MIGRATION_STATE_RECORD: + // TODO handle this + break; default: throw new RuntimeException("Unhandled record type " + type); } @@ -1558,6 +1608,8 @@ private enum ImbalanceSchedule { */ private final BootstrapMetadata bootstrapMetadata; + private final ZkRecordConsumer zkRecordConsumer; + /** * The maximum number of records per batch to allow. */ @@ -1660,6 +1712,7 @@ private QuorumController( this.metaLogListener = new QuorumMetaLogListener(); this.curClaimEpoch = -1; this.needToCompleteAuthorizerLoad = authorizer.isPresent(); + this.zkRecordConsumer = new MigrationRecordConsumer(); updateWriteOffset(-1); resetToEmptyState(); diff --git a/metadata/src/main/java/org/apache/kafka/image/MetadataDelta.java b/metadata/src/main/java/org/apache/kafka/image/MetadataDelta.java index 3d5ee9821ae01..ab4fd68f41a85 100644 --- a/metadata/src/main/java/org/apache/kafka/image/MetadataDelta.java +++ b/metadata/src/main/java/org/apache/kafka/image/MetadataDelta.java @@ -206,6 +206,9 @@ public void replay(ApiMessage record) { * updating the highest offset and epoch. */ break; + case ZK_MIGRATION_STATE_RECORD: + // TODO handle this + break; default: throw new RuntimeException("Unknown metadata record type " + type); } diff --git a/metadata/src/main/java/org/apache/kafka/image/loader/LogDeltaManifest.java b/metadata/src/main/java/org/apache/kafka/image/loader/LogDeltaManifest.java index c3ab72b3b79f2..982a1f8e27180 100644 --- a/metadata/src/main/java/org/apache/kafka/image/loader/LogDeltaManifest.java +++ b/metadata/src/main/java/org/apache/kafka/image/loader/LogDeltaManifest.java @@ -18,6 +18,7 @@ package org.apache.kafka.image.loader; import org.apache.kafka.image.MetadataProvenance; +import org.apache.kafka.raft.LeaderAndEpoch; import java.util.Objects; @@ -31,6 +32,11 @@ public class LogDeltaManifest { */ private final MetadataProvenance provenance; + /** + * The current leader and epoch at the end of this delta. + */ + private final LeaderAndEpoch leaderAndEpoch; + /** * The number of batches that were loaded. */ @@ -48,11 +54,13 @@ public class LogDeltaManifest { public LogDeltaManifest( MetadataProvenance provenance, + LeaderAndEpoch leaderAndEpoch, int numBatches, long elapsedNs, long numBytes ) { this.provenance = provenance; + this.leaderAndEpoch = leaderAndEpoch; this.numBatches = numBatches; this.elapsedNs = elapsedNs; this.numBytes = numBytes; @@ -63,6 +71,10 @@ public MetadataProvenance provenance() { return provenance; } + public LeaderAndEpoch leaderAndEpoch() { + return leaderAndEpoch; + } + public int numBatches() { return numBatches; } @@ -79,6 +91,7 @@ public long numBytes() { public int hashCode() { return Objects.hash( provenance, + leaderAndEpoch, numBatches, elapsedNs, numBytes); @@ -89,6 +102,7 @@ public boolean equals(Object o) { if (o == null || !o.getClass().equals(this.getClass())) return false; LogDeltaManifest other = (LogDeltaManifest) o; return provenance.equals(other.provenance) && + leaderAndEpoch == other.leaderAndEpoch && numBatches == other.numBatches && elapsedNs == other.elapsedNs && numBytes == other.numBytes; @@ -98,6 +112,7 @@ public boolean equals(Object o) { public String toString() { return "LogDeltaManifest(" + "provenance=" + provenance + + ", leaderAndEpoch=" + leaderAndEpoch + ", numBatches=" + numBatches + ", elapsedNs=" + elapsedNs + ", numBytes=" + numBytes + diff --git a/metadata/src/main/java/org/apache/kafka/image/loader/MetadataLoader.java b/metadata/src/main/java/org/apache/kafka/image/loader/MetadataLoader.java index 1e4827c733041..7fe1930c8797b 100644 --- a/metadata/src/main/java/org/apache/kafka/image/loader/MetadataLoader.java +++ b/metadata/src/main/java/org/apache/kafka/image/loader/MetadataLoader.java @@ -359,6 +359,7 @@ LogDeltaManifest loadLogDelta( long elapsedNs = time.nanoseconds() - startNs; metrics.updateBatchProcessingTime(elapsedNs); return new LogDeltaManifest(provenance, + currentLeaderAndEpoch, numBatches, elapsedNs, numBytes); @@ -447,14 +448,6 @@ SnapshotManifest loadSnapshot( public void handleLeaderChange(LeaderAndEpoch leaderAndEpoch) { eventQueue.append(() -> { currentLeaderAndEpoch = leaderAndEpoch; - for (MetadataPublisher publisher : publishers.values()) { - try { - publisher.publishLeaderAndEpoch(leaderAndEpoch); - } catch (Throwable e) { - faultHandler.handleFault("Unhandled error publishing the new leader and epoch" - + leaderAndEpoch + " with publisher " + publisher.name(), e); - } - } }); } diff --git a/metadata/src/main/java/org/apache/kafka/image/publisher/MetadataPublisher.java b/metadata/src/main/java/org/apache/kafka/image/publisher/MetadataPublisher.java index 5b42347e544f0..8dfba7a99abd8 100644 --- a/metadata/src/main/java/org/apache/kafka/image/publisher/MetadataPublisher.java +++ b/metadata/src/main/java/org/apache/kafka/image/publisher/MetadataPublisher.java @@ -21,7 +21,6 @@ import org.apache.kafka.image.MetadataImage; import org.apache.kafka.image.loader.LogDeltaManifest; import org.apache.kafka.image.loader.SnapshotManifest; -import org.apache.kafka.raft.LeaderAndEpoch; /** @@ -66,14 +65,6 @@ void publishLogDelta( LogDeltaManifest manifest ); - - /** - * Publish a change in the KRaft leader and epoch. - * - * @param leaderAndEpoch The new leader and epoch. - */ - default void publishLeaderAndEpoch(LeaderAndEpoch leaderAndEpoch) { } - /** * Close this metadata publisher. */ diff --git a/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java b/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java index fa12daf6a6489..8cd8b4991be97 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java @@ -18,6 +18,8 @@ import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; +import org.apache.kafka.controller.Controller; +import org.apache.kafka.controller.MigrationControlManager; import org.apache.kafka.image.MetadataDelta; import org.apache.kafka.image.MetadataImage; import org.apache.kafka.image.loader.LogDeltaManifest; @@ -32,7 +34,10 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.HashSet; import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; import java.util.function.Function; import static java.util.concurrent.TimeUnit.NANOSECONDS; @@ -48,23 +53,32 @@ public class KRaftMigrationDriver implements MetadataPublisher { private final int nodeId; private final MigrationClient zkMigrationClient; private final BrokersRpcClient rpcClient; + private final ZkRecordConsumer zkRecordConsumer; private final KafkaEventQueue eventQueue; + private volatile LeaderAndEpoch leaderAndEpoch; private volatile MigrationState migrationState; private volatile ZkMigrationLeadershipState migrationLeadershipState; private volatile MetadataDelta delta; private volatile MetadataImage image; - public KRaftMigrationDriver(int nodeId, MigrationClient zkMigrationClient, BrokersRpcClient rpcClient) { + public KRaftMigrationDriver( + int nodeId, + ZkRecordConsumer zkRecordConsumer, + MigrationClient zkMigrationClient, + BrokersRpcClient rpcClient + ) { this.nodeId = nodeId; + this.zkRecordConsumer = zkRecordConsumer; + this.zkMigrationClient = zkMigrationClient; + this.rpcClient = rpcClient; this.time = Time.SYSTEM; this.log = LoggerFactory.getLogger(KRaftMigrationDriver.class); this.migrationState = MigrationState.UNINITIALIZED; this.migrationLeadershipState = ZkMigrationLeadershipState.EMPTY; - this.zkMigrationClient = zkMigrationClient; - this.rpcClient = rpcClient; this.eventQueue = new KafkaEventQueue(Time.SYSTEM, new LogContext("KRaftMigrationDriver"), "kraft-migration"); this.delta = null; this.image = MetadataImage.EMPTY; + this.leaderAndEpoch = LeaderAndEpoch.UNKNOWN; } public void start() { @@ -81,6 +95,7 @@ private void initializeMigrationState() { } private boolean isControllerQuorumReadyForMigration() { + // TODO implement this return true; } @@ -169,13 +184,13 @@ public void publishSnapshot(MetadataDelta delta, MetadataImage newImage, Snapsho @Override public void publishLogDelta(MetadataDelta delta, MetadataImage newImage, LogDeltaManifest manifest) { + if (!leaderAndEpoch.equals(manifest.leaderAndEpoch())) { + eventQueue.append(new KRaftLeaderEvent(manifest.leaderAndEpoch())); + } + eventQueue.append(new MetadataChangeEvent(delta, newImage)); } - @Override - public void publishLeaderAndEpoch(LeaderAndEpoch leaderAndEpoch) { - eventQueue.append(new KRaftLeaderEvent(leaderAndEpoch.isLeader(nodeId), leaderAndEpoch.epoch())); - } @Override public void close() throws Exception { @@ -231,16 +246,17 @@ public void handleException(Throwable e) { } class KRaftLeaderEvent implements EventQueue.Event { - private final boolean isActive; - private final int kraftControllerEpoch; + private final LeaderAndEpoch leaderAndEpoch; - KRaftLeaderEvent(boolean isActive, int kraftControllerEpoch) { - this.isActive = isActive; - this.kraftControllerEpoch = kraftControllerEpoch; + KRaftLeaderEvent(LeaderAndEpoch leaderAndEpoch) { + this.leaderAndEpoch = leaderAndEpoch; } + @Override public void run() throws Exception { - // We can either the the active controller or just resigned being the controller. + // We can either be the active controller or just resigned from being the controller. + KRaftMigrationDriver.this.leaderAndEpoch = leaderAndEpoch; + boolean isActive = leaderAndEpoch.isLeader(KRaftMigrationDriver.this.nodeId); switch (migrationState) { case UNINITIALIZED: // Poll and retry after initialization @@ -252,12 +268,11 @@ public void run() throws Exception { break; default: if (!isActive) { - apply("KRaftLeaderEvent is active", - state -> state.withControllerZkVersion(ZkMigrationLeadershipState.EMPTY.controllerZkVersion())); + apply("KRaftLeaderEvent is not active", state -> ZkMigrationLeadershipState.EMPTY); transitionTo(MigrationState.INACTIVE); } else { // Apply the new KRaft state - apply("KRaftLeaderEvent not active", state -> state.withNewKRaftController(nodeId, kraftControllerEpoch)); + apply("KRaftLeaderEvent is active", state -> state.withNewKRaftController(nodeId, leaderAndEpoch.epoch())); // Before becoming the controller fo ZkBrokers, we need to make sure the // Controller Quorum can handle migration. transitionTo(MigrationState.WAIT_FOR_CONTROLLER_QUORUM); @@ -304,10 +319,14 @@ public void run() throws Exception { case BECOME_CONTROLLER: // TODO: Handle unhappy path. apply("BecomeZkLeaderEvent", zkMigrationClient::claimControllerLeadership); - if (!migrationLeadershipState.zkMigrationComplete()) { - transitionTo(MigrationState.WAIT_FOR_BROKERS); + if (migrationLeadershipState.controllerZkVersion() == -1) { + // We could not claim leadership, stay in BECOME_CONTROLLER to retry } else { - transitionTo(MigrationState.KRAFT_CONTROLLER_TO_BROKER_COMM); + if (!migrationLeadershipState.zkMigrationComplete()) { + transitionTo(MigrationState.WAIT_FOR_BROKERS); + } else { + transitionTo(MigrationState.KRAFT_CONTROLLER_TO_BROKER_COMM); + } } break; default: @@ -347,16 +366,33 @@ public void handleException(Throwable e) { class MigrateMetadataEvent implements EventQueue.Event { @Override public void run() throws Exception { - // TODO: Do actual zk write. - OffsetAndEpoch offsetAndEpochAfterMigration = new OffsetAndEpoch(-1, -1); - log.debug("Completed migrating metadata from Zookeeper. Current offset is {} and " + - "epoch is {}", offsetAndEpochAfterMigration.offset(), - offsetAndEpochAfterMigration.epoch()); - ZkMigrationLeadershipState newState = migrationLeadershipState.withKRaftMetadataOffsetAndEpoch( - offsetAndEpochAfterMigration.offset(), - offsetAndEpochAfterMigration.epoch()); - apply("Migrate metadata from Zk", state -> zkMigrationClient.setMigrationRecoveryState(newState)); - transitionTo(MigrationState.KRAFT_CONTROLLER_TO_BROKER_COMM); + Set brokersInMetadata = new HashSet<>(); + zkRecordConsumer.beginMigration(); + try { + zkMigrationClient.readAllMetadata(batch -> { + try { + CompletableFuture future = zkRecordConsumer.acceptBatch(batch); + future.get(); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } catch (ExecutionException e) { + throw new RuntimeException(e.getCause()); + } + }, brokersInMetadata::add); + OffsetAndEpoch offsetAndEpochAfterMigration = zkRecordConsumer.completeMigration(); + log.debug("Completed migrating metadata from Zookeeper. Current offset is {} and epoch is {}. Saw brokers {}", + offsetAndEpochAfterMigration.offset(), + offsetAndEpochAfterMigration.epoch(), + brokersInMetadata); + ZkMigrationLeadershipState newState = migrationLeadershipState.withKRaftMetadataOffsetAndEpoch( + offsetAndEpochAfterMigration.offset(), + offsetAndEpochAfterMigration.epoch()); + apply("Migrate metadata from Zk", state -> zkMigrationClient.setMigrationRecoveryState(newState)); + transitionTo(MigrationState.KRAFT_CONTROLLER_TO_BROKER_COMM); + } catch (Throwable t) { + zkRecordConsumer.abortMigration(); + // TODO ??? + } } @Override @@ -372,8 +408,9 @@ public void run() throws Exception { switch (migrationState) { case KRAFT_CONTROLLER_TO_BROKER_COMM: if (image.highestOffsetAndEpoch().compareTo(migrationLeadershipState.offsetAndEpoch()) >= 0) { - rpcClient.sendRPCsToBrokersFromMetadataImage(image, - migrationLeadershipState.kraftControllerEpoch()); + // TODO do this when we have the real client + //rpcClient.sendRPCsToBrokersFromMetadataImage(image, + // migrationLeadershipState.kraftControllerEpoch()); // Migration leadership state doesn't change since we're not doing any Zk // writes. transitionTo(MigrationState.DUAL_WRITE); diff --git a/metadata/src/main/java/org/apache/kafka/metadata/migration/ZkRecordConsumer.java b/metadata/src/main/java/org/apache/kafka/metadata/migration/ZkRecordConsumer.java new file mode 100644 index 0000000000000..70b2f0aaa0a93 --- /dev/null +++ b/metadata/src/main/java/org/apache/kafka/metadata/migration/ZkRecordConsumer.java @@ -0,0 +1,14 @@ +package org.apache.kafka.metadata.migration; + +import org.apache.kafka.raft.OffsetAndEpoch; +import org.apache.kafka.server.common.ApiMessageAndVersion; + +import java.util.List; +import java.util.concurrent.CompletableFuture; + +public interface ZkRecordConsumer { + void beginMigration(); + CompletableFuture acceptBatch(List recordBatch); + OffsetAndEpoch completeMigration(); + void abortMigration(); +} diff --git a/metadata/src/test/java/org/apache/kafka/image/loader/MetadataLoaderTest.java b/metadata/src/test/java/org/apache/kafka/image/loader/MetadataLoaderTest.java index 585f1dc40a7e1..1aa86738f980b 100644 --- a/metadata/src/test/java/org/apache/kafka/image/loader/MetadataLoaderTest.java +++ b/metadata/src/test/java/org/apache/kafka/image/loader/MetadataLoaderTest.java @@ -28,6 +28,7 @@ import org.apache.kafka.image.publisher.MetadataPublisher; import org.apache.kafka.raft.Batch; import org.apache.kafka.raft.BatchReader; +import org.apache.kafka.raft.LeaderAndEpoch; import org.apache.kafka.raft.OffsetAndEpoch; import org.apache.kafka.server.common.ApiMessageAndVersion; import org.apache.kafka.server.common.MetadataVersion; @@ -412,7 +413,7 @@ public void testLoadEmptyBatch() throws Exception { assertEquals(400L, loader.lastAppliedOffset()); } assertTrue(publishers.get(0).closed); - assertEquals(new LogDeltaManifest(new MetadataProvenance(400, 100, 4000), 1, + assertEquals(new LogDeltaManifest(new MetadataProvenance(400, 100, 4000), LeaderAndEpoch.UNKNOWN, 1, 3000000L, 10), publishers.get(0).latestLogDeltaManifest); assertEquals(MetadataVersion.IBP_3_3_IV1, diff --git a/metadata/src/test/java/org/apache/kafka/image/publisher/SnapshotGeneratorTest.java b/metadata/src/test/java/org/apache/kafka/image/publisher/SnapshotGeneratorTest.java index 42fb8d92f6eba..47befabcaba2a 100644 --- a/metadata/src/test/java/org/apache/kafka/image/publisher/SnapshotGeneratorTest.java +++ b/metadata/src/test/java/org/apache/kafka/image/publisher/SnapshotGeneratorTest.java @@ -23,6 +23,7 @@ import org.apache.kafka.image.MetadataProvenance; import org.apache.kafka.image.loader.LogDeltaManifest; import org.apache.kafka.metadata.RecordTestUtils; +import org.apache.kafka.raft.LeaderAndEpoch; import org.apache.kafka.server.fault.FaultHandlerException; import org.apache.kafka.server.fault.MockFaultHandler; import org.apache.kafka.test.TestUtils; @@ -99,14 +100,14 @@ public void testCreateSnapshot() throws Exception { build()) { // Publish a log delta batch. This one will not trigger a snapshot yet. generator.publishLogDelta(TEST_DELTA, TEST_IMAGE, - new LogDeltaManifest(MetadataProvenance.EMPTY, 1, 100, 100)); + new LogDeltaManifest(MetadataProvenance.EMPTY, LeaderAndEpoch.UNKNOWN, 1, 100, 100)); // Publish a log delta batch. This will trigger a snapshot. generator.publishLogDelta(TEST_DELTA, TEST_IMAGE, - new LogDeltaManifest(MetadataProvenance.EMPTY, 1, 100, 100)); + new LogDeltaManifest(MetadataProvenance.EMPTY, LeaderAndEpoch.UNKNOWN, 1, 100, 100)); // Publish a log delta batch. This one will be ignored because there are other images // queued for writing. generator.publishLogDelta(TEST_DELTA, TEST_IMAGE, - new LogDeltaManifest(MetadataProvenance.EMPTY, 1, 100, 2000)); + new LogDeltaManifest(MetadataProvenance.EMPTY, LeaderAndEpoch.UNKNOWN, 1, 100, 2000)); assertEquals(Collections.emptyList(), emitter.images()); emitter.setReady(); } @@ -128,7 +129,7 @@ public void testSnapshotsDisabled() throws Exception { disabledReason.compareAndSet(null, "we are testing disable()"); // No snapshots are generated because snapshots are disabled. generator.publishLogDelta(TEST_DELTA, TEST_IMAGE, - new LogDeltaManifest(MetadataProvenance.EMPTY, 1, 100, 100)); + new LogDeltaManifest(MetadataProvenance.EMPTY, LeaderAndEpoch.UNKNOWN, 1, 100, 100)); } assertEquals(Collections.emptyList(), emitter.images()); faultHandler.maybeRethrowFirstException(); @@ -147,17 +148,17 @@ public void testTimeBasedSnapshots() throws Exception { build()) { // This image isn't published yet. generator.publishLogDelta(TEST_DELTA, TEST_IMAGE, - new LogDeltaManifest(MetadataProvenance.EMPTY, 1, 100, 50)); + new LogDeltaManifest(MetadataProvenance.EMPTY, LeaderAndEpoch.UNKNOWN, 1, 100, 50)); assertEquals(Collections.emptyList(), emitter.images()); mockTime.sleep(TimeUnit.MINUTES.toNanos(40)); // Next image is published because of the time delay. generator.publishLogDelta(TEST_DELTA, TEST_IMAGE, - new LogDeltaManifest(MetadataProvenance.EMPTY, 1, 100, 50)); + new LogDeltaManifest(MetadataProvenance.EMPTY, LeaderAndEpoch.UNKNOWN, 1, 100, 50)); TestUtils.waitForCondition(() -> emitter.images().size() == 1, "images.size == 1"); // bytesSinceLastSnapshot was reset to 0 by the previous snapshot, // so this does not trigger a new snapshot. generator.publishLogDelta(TEST_DELTA, TEST_IMAGE, - new LogDeltaManifest(MetadataProvenance.EMPTY, 1, 100, 150)); + new LogDeltaManifest(MetadataProvenance.EMPTY, LeaderAndEpoch.UNKNOWN, 1, 100, 150)); } assertEquals(Arrays.asList(TEST_IMAGE), emitter.images()); faultHandler.maybeRethrowFirstException(); @@ -173,7 +174,7 @@ public void testEmitterProblem() throws Exception { build()) { for (int i = 0; i < 2; i++) { generator.publishLogDelta(TEST_DELTA, TEST_IMAGE, - new LogDeltaManifest(MetadataProvenance.EMPTY, 1, 10000, 50000)); + new LogDeltaManifest(MetadataProvenance.EMPTY, LeaderAndEpoch.UNKNOWN, 1, 10000, 50000)); } } assertEquals(Collections.emptyList(), emitter.images()); From df216779603f4d1869b52d22107369c3e0545b57 Mon Sep 17 00:00:00 2001 From: Akhilesh Chaganti Date: Sun, 18 Dec 2022 16:56:57 -0800 Subject: [PATCH 07/30] RPC support from KRaft controller to brokers --- .../src/main/scala/kafka/cluster/Broker.scala | 19 +- .../controller/ControllerChannelManager.scala | 202 +++++++---- .../kafka/controller/ControllerContext.scala | 26 +- .../kafka/controller/KafkaController.scala | 12 +- .../KRaftControllerToZkBrokersRpcClient.scala | 331 ++++++++++++++++++ .../scala/kafka/server/ControllerServer.scala | 5 +- .../ControllerChannelManagerTest.scala | 20 +- .../server/BrokerEpochIntegrationTest.scala | 11 +- .../kafka/server/LeaderElectionTest.scala | 11 +- .../kafka/server/ServerShutdownTest.scala | 10 +- .../controller/MigrationControlManager.java | 16 + .../kafka/controller/QuorumController.java | 19 +- .../org/apache/kafka/image/ClusterDelta.java | 9 + .../org/apache/kafka/image/TopicsImage.java | 10 + .../metadata/migration/BrokersRpcClient.java | 12 +- .../migration/KRaftMigrationDriver.java | 6 +- .../metadata/migration/ZkRecordConsumer.java | 16 + 17 files changed, 628 insertions(+), 107 deletions(-) create mode 100644 core/src/main/scala/kafka/migration/KRaftControllerToZkBrokersRpcClient.scala diff --git a/core/src/main/scala/kafka/cluster/Broker.scala b/core/src/main/scala/kafka/cluster/Broker.scala index 9b1d741835c35..ede63cd3c0a60 100755 --- a/core/src/main/scala/kafka/cluster/Broker.scala +++ b/core/src/main/scala/kafka/cluster/Broker.scala @@ -18,7 +18,6 @@ package kafka.cluster import java.util - import kafka.common.BrokerEndPointNotAvailableException import kafka.server.KafkaConfig import org.apache.kafka.common.feature.{Features, SupportedVersionRange} @@ -26,9 +25,11 @@ import org.apache.kafka.common.feature.Features._ import org.apache.kafka.common.{ClusterResource, Endpoint, Node} import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.security.auth.SecurityProtocol +import org.apache.kafka.metadata.{BrokerRegistration, VersionRange} import org.apache.kafka.server.authorizer.AuthorizerServerInfo import scala.collection.Seq +import scala.compat.java8.OptionConverters._ import scala.jdk.CollectionConverters._ object Broker { @@ -41,6 +42,22 @@ object Broker { def apply(id: Int, endPoints: Seq[EndPoint], rack: Option[String]): Broker = { new Broker(id, endPoints, rack, emptySupportedFeatures) } + + private def supportedFeatures(features: java.util.Map[String, VersionRange]): java.util + .Map[String, SupportedVersionRange] = { + features.asScala.map { case (name, range) => + name -> new SupportedVersionRange(range.min(), range.max()) + }.asJava + } + + def fromBrokerRegistration(registration: BrokerRegistration): Broker = { + new Broker( + registration.id(), + registration.listeners().values().asScala.map(EndPoint.fromJava).toSeq, + registration.rack().asScala, + Features.supportedFeatures(supportedFeatures(registration.supportedFeatures())) + ) + } } /** diff --git a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala index 145eca453a130..ec3a9e9b4511f 100755 --- a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala +++ b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala @@ -16,40 +16,38 @@ */ package kafka.controller -import java.net.SocketTimeoutException -import java.util.concurrent.{BlockingQueue, LinkedBlockingQueue, TimeUnit} - import com.yammer.metrics.core.{Gauge, Timer} import kafka.api._ import kafka.cluster.Broker import kafka.metrics.KafkaMetricsGroup import kafka.server.KafkaConfig -import kafka.utils._ import kafka.utils.Implicits._ +import kafka.utils._ import org.apache.kafka.clients._ +import org.apache.kafka.common._ import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState import org.apache.kafka.common.message.StopReplicaRequestData.{StopReplicaPartitionState, StopReplicaTopicState} +import org.apache.kafka.common.message.UpdateMetadataRequestData.{UpdateMetadataBroker, UpdateMetadataEndpoint, UpdateMetadataPartitionState} import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.network._ import org.apache.kafka.common.protocol.{ApiKeys, Errors} -import org.apache.kafka.common.message.UpdateMetadataRequestData.{UpdateMetadataBroker, UpdateMetadataEndpoint, UpdateMetadataPartitionState} import org.apache.kafka.common.requests._ import org.apache.kafka.common.security.JaasContext import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.utils.{LogContext, Time} -import org.apache.kafka.common.{KafkaException, Node, Reconfigurable, TopicPartition, Uuid} import org.apache.kafka.server.common.MetadataVersion._ -import scala.jdk.CollectionConverters._ -import scala.collection.mutable.HashMap +import java.net.SocketTimeoutException +import java.util.concurrent.{BlockingQueue, LinkedBlockingQueue, TimeUnit} import scala.collection.{Seq, Set, mutable} +import scala.jdk.CollectionConverters._ object ControllerChannelManager { val QueueSizeMetricName = "QueueSize" val RequestRateAndQueueTimeMetricName = "RequestRateAndQueueTimeMs" } -class ControllerChannelManager(controllerContext: ControllerContext, +class ControllerChannelManager(controllerEpoch: () => Int, config: KafkaConfig, time: Time, metrics: Metrics, @@ -57,7 +55,7 @@ class ControllerChannelManager(controllerContext: ControllerContext, threadNamePrefix: Option[String] = None) extends Logging with KafkaMetricsGroup { import ControllerChannelManager._ - protected val brokerStateInfo = new HashMap[Int, ControllerBrokerStateInfo] + protected val brokerStateInfo = new mutable.HashMap[Int, ControllerBrokerStateInfo] private val brokerLock = new Object this.logIdent = "[Channel manager on controller " + config.brokerId + "]: " @@ -67,15 +65,15 @@ class ControllerChannelManager(controllerContext: ControllerContext, } ) - def startup() = { - controllerContext.liveOrShuttingDownBrokers.foreach(addNewBroker) + def startup(initialBrokers: Set[Broker]):Unit = { + initialBrokers.foreach(addNewBroker) brokerLock synchronized { brokerStateInfo.foreach(brokerState => startRequestSendThread(brokerState._1)) } } - def shutdown() = { + def shutdown():Unit = { brokerLock synchronized { brokerStateInfo.values.toList.foreach(removeExistingBroker) } @@ -173,7 +171,7 @@ class ControllerChannelManager(controllerContext: ControllerContext, RequestRateAndQueueTimeMetricName, TimeUnit.MILLISECONDS, TimeUnit.SECONDS, brokerMetricTags(broker.id) ) - val requestThread = new RequestSendThread(config.brokerId, controllerContext, messageQueue, networkClient, + val requestThread = new RequestSendThread(config.brokerId, controllerEpoch, messageQueue, networkClient, brokerNode, config, time, requestRateAndQueueTimeMetrics, stateChangeLogger, threadName) requestThread.setDaemon(false) @@ -214,7 +212,7 @@ case class QueueItem(apiKey: ApiKeys, request: AbstractControlRequest.Builder[_ callback: AbstractResponse => Unit, enqueueTimeMs: Long) class RequestSendThread(val controllerId: Int, - val controllerContext: ControllerContext, + controllerEpoch: () => Int, val queue: BlockingQueue[QueueItem], val networkClient: NetworkClient, val brokerNode: Node, @@ -255,7 +253,8 @@ class RequestSendThread(val controllerId: Int, } } catch { case e: Throwable => // if the send was not successful, reconnect to broker and resend the message - warn(s"Controller $controllerId epoch ${controllerContext.epoch} fails to send request $requestBuilder " + + warn(s"Controller $controllerId epoch ${controllerEpoch()} fails to send request " + + s"$requestBuilder " + s"to broker $brokerNode. Reconnecting to broker.", e) networkClient.close(brokerNode.idString) isSendSuccessful = false @@ -270,7 +269,7 @@ class RequestSendThread(val controllerId: Int, val response = clientResponse.responseBody - stateChangeLogger.withControllerEpoch(controllerContext.epoch).trace(s"Received response " + + stateChangeLogger.withControllerEpoch(controllerEpoch()).trace(s"Received response " + s"$response for request $api with correlation id " + s"${requestHeader.correlationId} sent to broker $brokerNode") @@ -313,35 +312,74 @@ class RequestSendThread(val controllerId: Int, } } -class ControllerBrokerRequestBatch(config: KafkaConfig, - controllerChannelManager: ControllerChannelManager, - controllerEventManager: ControllerEventManager, - controllerContext: ControllerContext, - stateChangeLogger: StateChangeLogger) - extends AbstractControllerBrokerRequestBatch(config, controllerContext, stateChangeLogger) { +class ControllerBrokerRequestBatch( + config: KafkaConfig, + controllerChannelManager: ControllerChannelManager, + controllerEventManager: ControllerEventManager, + controllerContext: ControllerContext, + stateChangeLogger: StateChangeLogger +) extends AbstractControllerBrokerRequestBatch( + config, + () => controllerContext, + stateChangeLogger +) { def sendEvent(event: ControllerEvent): Unit = { controllerEventManager.put(event) } - def sendRequest(brokerId: Int, request: AbstractControlRequest.Builder[_ <: AbstractControlRequest], callback: AbstractResponse => Unit = null): Unit = { controllerChannelManager.sendRequest(brokerId, request, callback) } + override def handleLeaderAndIsrResponse(response: LeaderAndIsrResponse, broker: Int): Unit = { + sendEvent(LeaderAndIsrResponseReceived(response, broker)) + } + + override def handleUpdateMetadataResponse(response: UpdateMetadataResponse, broker: Int): Unit = { + sendEvent(UpdateMetadataResponseReceived(response, broker)) + } + + override def handleStopReplicaResponse(stopReplicaResponse: StopReplicaResponse, brokerId: Int, + partitionErrorsForDeletingTopics: Map[TopicPartition, Errors]): Unit = { + if (partitionErrorsForDeletingTopics.nonEmpty) + sendEvent(TopicDeletionStopReplicaResponseReceived(brokerId, stopReplicaResponse.error, partitionErrorsForDeletingTopics)) + } +} + +abstract class ControllerBrokerRequestMetadata { + def isTopicDeletionInProgress(topicName: String): Boolean + + def topicIds: collection.Map[String, Uuid] + + def liveBrokerIdAndEpochs: collection.Map[Int, Long] + + def liveOrShuttingDownBrokers: collection.Set[Broker] + + def isTopicQueuedUpForDeletion(topic: String): Boolean + + def isReplicaOnline(brokerId: Int, partition: TopicPartition): Boolean + + def partitionReplicaAssignment(partition: TopicPartition): collection.Seq[Int] + + def leaderEpoch(topicPartition: TopicPartition): Int + + def liveOrShuttingDownBrokerIds: collection.Set[Int] + + def partitionLeadershipInfo(topicPartition: TopicPartition): Option[LeaderIsrAndControllerEpoch] } abstract class AbstractControllerBrokerRequestBatch(config: KafkaConfig, - controllerContext: ControllerContext, - stateChangeLogger: StateChangeLogger) extends Logging { + metadataProvider: () => ControllerBrokerRequestMetadata, + stateChangeLogger: StateChangeLogger, + kraftController: Boolean = false) extends Logging { val controllerId: Int = config.brokerId val leaderAndIsrRequestMap = mutable.Map.empty[Int, mutable.Map[TopicPartition, LeaderAndIsrPartitionState]] val stopReplicaRequestMap = mutable.Map.empty[Int, mutable.Map[TopicPartition, StopReplicaPartitionState]] val updateMetadataRequestBrokerSet = mutable.Set.empty[Int] val updateMetadataRequestPartitionInfoMap = mutable.Map.empty[TopicPartition, UpdateMetadataPartitionState] - - def sendEvent(event: ControllerEvent): Unit + private var metadataInstance: ControllerBrokerRequestMetadata = _ def sendRequest(brokerId: Int, request: AbstractControlRequest.Builder[_ <: AbstractControlRequest], @@ -359,6 +397,7 @@ abstract class AbstractControllerBrokerRequestBatch(config: KafkaConfig, throw new IllegalStateException("Controller to broker state change requests batch is not empty while creating a " + s"new one. Some UpdateMetadata state changes to brokers $updateMetadataRequestBrokerSet with partition info " + s"$updateMetadataRequestPartitionInfoMap might be lost ") + metadataInstance = metadataProvider() } def clear(): Unit = { @@ -366,6 +405,7 @@ abstract class AbstractControllerBrokerRequestBatch(config: KafkaConfig, stopReplicaRequestMap.clear() updateMetadataRequestBrokerSet.clear() updateMetadataRequestPartitionInfoMap.clear() + metadataInstance = null } def addLeaderAndIsrRequestForBrokers(brokerIds: Seq[Int], @@ -398,7 +438,7 @@ abstract class AbstractControllerBrokerRequestBatch(config: KafkaConfig, result.put(topicPartition, partitionState) } - addUpdateMetadataRequestForBrokers(controllerContext.liveOrShuttingDownBrokerIds.toSeq, Set(topicPartition)) + addUpdateMetadataRequestForBrokers(metadataInstance.liveOrShuttingDownBrokerIds.toSeq, Set(topicPartition)) } def addStopReplicaRequestForBrokers(brokerIds: Seq[Int], @@ -406,13 +446,7 @@ abstract class AbstractControllerBrokerRequestBatch(config: KafkaConfig, deletePartition: Boolean): Unit = { // A sentinel (-2) is used as an epoch if the topic is queued for deletion. It overrides // any existing epoch. - val leaderEpoch = if (controllerContext.isTopicQueuedUpForDeletion(topicPartition.topic)) { - LeaderAndIsr.EpochDuringDelete - } else { - controllerContext.partitionLeadershipInfo(topicPartition) - .map(_.leaderAndIsr.leaderEpoch) - .getOrElse(LeaderAndIsr.NoEpoch) - } + val leaderEpoch = metadataInstance.leaderEpoch(topicPartition) brokerIds.filter(_ >= 0).foreach { brokerId => val result = stopReplicaRequestMap.getOrElseUpdate(brokerId, mutable.Map.empty) @@ -427,36 +461,48 @@ abstract class AbstractControllerBrokerRequestBatch(config: KafkaConfig, /** Send UpdateMetadataRequest to the given brokers for the given partitions and partitions that are being deleted */ def addUpdateMetadataRequestForBrokers(brokerIds: Seq[Int], partitions: collection.Set[TopicPartition]): Unit = { - - def updateMetadataRequestPartitionInfo(partition: TopicPartition, beingDeleted: Boolean): Unit = { - controllerContext.partitionLeadershipInfo(partition) match { + updateMetadataRequestBrokerSet ++= brokerIds.filter(_ >= 0) + partitions.foreach { partition => + val beingDeleted = metadataInstance.isTopicQueuedUpForDeletion(partition.topic()) + metadataInstance.partitionLeadershipInfo(partition) match { case Some(LeaderIsrAndControllerEpoch(leaderAndIsr, controllerEpoch)) => - val replicas = controllerContext.partitionReplicaAssignment(partition) - val offlineReplicas = replicas.filterNot(controllerContext.isReplicaOnline(_, partition)) - val updatedLeaderAndIsr = - if (beingDeleted) LeaderAndIsr.duringDelete(leaderAndIsr.isr) - else leaderAndIsr - - val partitionStateInfo = new UpdateMetadataPartitionState() - .setTopicName(partition.topic) - .setPartitionIndex(partition.partition) - .setControllerEpoch(controllerEpoch) - .setLeader(updatedLeaderAndIsr.leader) - .setLeaderEpoch(updatedLeaderAndIsr.leaderEpoch) - .setIsr(updatedLeaderAndIsr.isr.map(Integer.valueOf).asJava) - .setZkVersion(updatedLeaderAndIsr.partitionEpoch) - .setReplicas(replicas.map(Integer.valueOf).asJava) - .setOfflineReplicas(offlineReplicas.map(Integer.valueOf).asJava) - updateMetadataRequestPartitionInfoMap.put(partition, partitionStateInfo) - + val updatedLeaderAndIsr = if (beingDeleted) LeaderAndIsr.duringDelete(leaderAndIsr.isr) else leaderAndIsr + val replicas = metadataInstance.partitionReplicaAssignment(partition) + val offlineReplicas = replicas.filter(!metadataInstance.isReplicaOnline(_, partition)) + addUpdateMetadataRequestForBrokers(brokerIds, controllerEpoch, partition, + updatedLeaderAndIsr.leader, updatedLeaderAndIsr.leaderEpoch, updatedLeaderAndIsr.partitionEpoch, + updatedLeaderAndIsr.isr, replicas, offlineReplicas) case None => info(s"Leader not yet assigned for partition $partition. Skip sending UpdateMetadataRequest.") } } + } + + def addUpdateMetadataRequestForBrokers(brokerIds: Seq[Int]): Unit = { + updateMetadataRequestBrokerSet ++= brokerIds.filter(_ >= 0) + } + def addUpdateMetadataRequestForBrokers(brokerIds: Seq[Int], + controllerEpoch: Int, + partition: TopicPartition, + leader: Int, + leaderEpoch: Int, + partitionEpoch: Int, + isrs: List[Int], + replicas: Seq[Int], + offlineReplicas: Seq[Int]): Unit = { updateMetadataRequestBrokerSet ++= brokerIds.filter(_ >= 0) - partitions.foreach(partition => updateMetadataRequestPartitionInfo(partition, - beingDeleted = controllerContext.topicsToBeDeleted.contains(partition.topic))) + val partitionStateInfo = new UpdateMetadataPartitionState() + .setTopicName(partition.topic) + .setPartitionIndex(partition.partition) + .setControllerEpoch(controllerEpoch) + .setLeader(leader) + .setLeaderEpoch(leaderEpoch) + .setIsr(isrs.map(Integer.valueOf).asJava) + .setZkVersion(partitionEpoch) + .setReplicas(replicas.map(Integer.valueOf).asJava) + .setOfflineReplicas(offlineReplicas.map(Integer.valueOf).asJava) + updateMetadataRequestPartitionInfoMap.put(partition, partitionStateInfo) } private def sendLeaderAndIsrRequest(controllerEpoch: Int, stateChangeLog: StateChangeLogger): Unit = { @@ -471,7 +517,7 @@ abstract class AbstractControllerBrokerRequestBatch(config: KafkaConfig, else 0 leaderAndIsrRequestMap.forKeyValue { (broker, leaderAndIsrPartitionStates) => - if (controllerContext.liveOrShuttingDownBrokerIds.contains(broker)) { + if (metadataInstance.liveOrShuttingDownBrokerIds.contains(broker)) { val leaderIds = mutable.Set.empty[Int] var numBecomeLeaders = 0 leaderAndIsrPartitionStates.forKeyValue { (topicPartition, state) => @@ -487,26 +533,28 @@ abstract class AbstractControllerBrokerRequestBatch(config: KafkaConfig, } stateChangeLog.info(s"Sending LeaderAndIsr request to broker $broker with $numBecomeLeaders become-leader " + s"and ${leaderAndIsrPartitionStates.size - numBecomeLeaders} become-follower partitions") - val leaders = controllerContext.liveOrShuttingDownBrokers.filter(b => leaderIds.contains(b.id)).map { + val leaders = metadataInstance.liveOrShuttingDownBrokers.filter(b => leaderIds.contains(b.id)).map { _.node(config.interBrokerListenerName) } - val brokerEpoch = controllerContext.liveBrokerIdAndEpochs(broker) + val brokerEpoch = metadataInstance.liveBrokerIdAndEpochs(broker) val topicIds = leaderAndIsrPartitionStates.keys .map(_.topic) .toSet[String] - .map(topic => (topic, controllerContext.topicIds.getOrElse(topic, Uuid.ZERO_UUID))) + .map(topic => (topic, metadataInstance.topicIds.getOrElse(topic, Uuid.ZERO_UUID))) .toMap val leaderAndIsrRequestBuilder = new LeaderAndIsrRequest.Builder(leaderAndIsrRequestVersion, controllerId, controllerEpoch, brokerEpoch, leaderAndIsrPartitionStates.values.toBuffer.asJava, topicIds.asJava, leaders.asJava) sendRequest(broker, leaderAndIsrRequestBuilder, (r: AbstractResponse) => { val leaderAndIsrResponse = r.asInstanceOf[LeaderAndIsrResponse] - sendEvent(LeaderAndIsrResponseReceived(leaderAndIsrResponse, broker)) + handleLeaderAndIsrResponse(leaderAndIsrResponse, broker) }) } } leaderAndIsrRequestMap.clear() } + def handleLeaderAndIsrResponse(response: LeaderAndIsrResponse, broker: Int): Unit + private def sendUpdateMetadataRequests(controllerEpoch: Int, stateChangeLog: StateChangeLogger): Unit = { stateChangeLog.info(s"Sending UpdateMetadata request to brokers $updateMetadataRequestBrokerSet " + s"for ${updateMetadataRequestPartitionInfoMap.size} partitions") @@ -523,7 +571,7 @@ abstract class AbstractControllerBrokerRequestBatch(config: KafkaConfig, else if (config.interBrokerProtocolVersion.isAtLeast(IBP_0_9_0)) 1 else 0 - val liveBrokers = controllerContext.liveOrShuttingDownBrokers.iterator.map { broker => + val liveBrokers = metadataInstance.liveOrShuttingDownBrokers.iterator.map { broker => val endpoints = if (updateMetadataRequestVersion == 0) { // Version 0 of UpdateMetadataRequest only supports PLAINTEXT val securityProtocol = SecurityProtocol.PLAINTEXT @@ -549,17 +597,17 @@ abstract class AbstractControllerBrokerRequestBatch(config: KafkaConfig, .setRack(broker.rack.orNull) }.toBuffer - updateMetadataRequestBrokerSet.intersect(controllerContext.liveOrShuttingDownBrokerIds).foreach { broker => - val brokerEpoch = controllerContext.liveBrokerIdAndEpochs(broker) + updateMetadataRequestBrokerSet.intersect(metadataInstance.liveOrShuttingDownBrokerIds).foreach { broker => + val brokerEpoch = metadataInstance.liveBrokerIdAndEpochs(broker) val topicIds = partitionStates.map(_.topicName()) .distinct - .filter(controllerContext.topicIds.contains) - .map(topic => (topic, controllerContext.topicIds(topic))).toMap + .filter(metadataInstance.topicIds.contains) + .map(topic => (topic, metadataInstance.topicIds(topic))).toMap val updateMetadataRequestBuilder = new UpdateMetadataRequest.Builder(updateMetadataRequestVersion, controllerId, controllerEpoch, brokerEpoch, partitionStates.asJava, liveBrokers.asJava, topicIds.asJava) sendRequest(broker, updateMetadataRequestBuilder, (r: AbstractResponse) => { val updateMetadataResponse = r.asInstanceOf[UpdateMetadataResponse] - sendEvent(UpdateMetadataResponseReceived(updateMetadataResponse, broker)) + handleUpdateMetadataResponse(updateMetadataResponse, broker) }) } @@ -567,6 +615,8 @@ abstract class AbstractControllerBrokerRequestBatch(config: KafkaConfig, updateMetadataRequestPartitionInfoMap.clear() } + def handleUpdateMetadataResponse(response: UpdateMetadataResponse, broker: Int): Unit + private def sendStopReplicaRequests(controllerEpoch: Int, stateChangeLog: StateChangeLogger): Unit = { val traceEnabled = stateChangeLog.isTraceEnabled val stopReplicaRequestVersion: Short = @@ -582,25 +632,24 @@ abstract class AbstractControllerBrokerRequestBatch(config: KafkaConfig, val partitionErrorsForDeletingTopics = mutable.Map.empty[TopicPartition, Errors] stopReplicaResponse.partitionErrors.forEach { pe => val tp = new TopicPartition(pe.topicName, pe.partitionIndex) - if (controllerContext.isTopicDeletionInProgress(pe.topicName) && + if (metadataInstance.isTopicDeletionInProgress(pe.topicName) && isPartitionDeleted(tp)) { partitionErrorsForDeletingTopics += tp -> Errors.forCode(pe.errorCode) } } if (partitionErrorsForDeletingTopics.nonEmpty) - sendEvent(TopicDeletionStopReplicaResponseReceived(brokerId, stopReplicaResponse.error, - partitionErrorsForDeletingTopics)) + handleStopReplicaResponse(stopReplicaResponse, brokerId, partitionErrorsForDeletingTopics.toMap) } stopReplicaRequestMap.forKeyValue { (brokerId, partitionStates) => - if (controllerContext.liveOrShuttingDownBrokerIds.contains(brokerId)) { + if (metadataInstance.liveOrShuttingDownBrokerIds.contains(brokerId)) { if (traceEnabled) partitionStates.forKeyValue { (topicPartition, partitionState) => stateChangeLog.trace(s"Sending StopReplica request $partitionState to " + s"broker $brokerId for partition $topicPartition") } - val brokerEpoch = controllerContext.liveBrokerIdAndEpochs(brokerId) + val brokerEpoch = metadataInstance.liveBrokerIdAndEpochs(brokerId) if (stopReplicaRequestVersion >= 3) { val stopReplicaTopicState = mutable.Map.empty[String, StopReplicaTopicState] partitionStates.forKeyValue { (topicPartition, partitionState) => @@ -659,6 +708,9 @@ abstract class AbstractControllerBrokerRequestBatch(config: KafkaConfig, stopReplicaRequestMap.clear() } + def handleStopReplicaResponse(stopReplicaResponse: StopReplicaResponse, brokerId: Int, + partitionErrorsForDeletingTopics: Map[TopicPartition, Errors]): Unit + def sendRequestsToBrokers(controllerEpoch: Int): Unit = { try { val stateChangeLog = stateChangeLogger.withControllerEpoch(controllerEpoch) diff --git a/core/src/main/scala/kafka/controller/ControllerContext.scala b/core/src/main/scala/kafka/controller/ControllerContext.scala index 7065d87c4c606..ba0314d473cc3 100644 --- a/core/src/main/scala/kafka/controller/ControllerContext.scala +++ b/core/src/main/scala/kafka/controller/ControllerContext.scala @@ -17,6 +17,7 @@ package kafka.controller +import kafka.api.LeaderAndIsr import kafka.cluster.Broker import kafka.utils.Implicits._ import org.apache.kafka.common.{TopicPartition, Uuid} @@ -72,7 +73,7 @@ case class ReplicaAssignment private (replicas: Seq[Int], s"removingReplicas=${removingReplicas.mkString(",")})" } -class ControllerContext { +class ControllerContext extends ControllerBrokerRequestMetadata { val stats = new ControllerStats var offlinePartitionCount = 0 var preferredReplicaImbalanceCount = 0 @@ -230,7 +231,11 @@ class ControllerContext { }.toSet } - def isReplicaOnline(brokerId: Int, topicPartition: TopicPartition, includeShuttingDownBrokers: Boolean = false): Boolean = { + def isReplicaOnline(brokerId: Int, topicPartition: TopicPartition): Boolean = { + isReplicaOnline(brokerId, topicPartition, includeShuttingDownBrokers = false) + } + + def isReplicaOnline(brokerId: Int, topicPartition: TopicPartition, includeShuttingDownBrokers: Boolean): Boolean = { val brokerOnline = { if (includeShuttingDownBrokers) liveOrShuttingDownBrokerIds.contains(brokerId) else liveBrokerIds.contains(brokerId) @@ -445,6 +450,22 @@ class ControllerContext { Some(replicaAssignment), Some(leaderIsrAndControllerEpoch)) } + def partitionLeaderAndIsr(partition: TopicPartition): Option[LeaderAndIsr] = { + partitionLeadershipInfo.get(partition).map(_.leaderAndIsr) + } + + def leaderEpoch(partition: TopicPartition): Int = { + // A sentinel (-2) is used as an epoch if the topic is queued for deletion. It overrides + // any existing epoch. + if (isTopicQueuedUpForDeletion(partition.topic)) { + LeaderAndIsr.EpochDuringDelete + } else { + partitionLeadershipInfo.get(partition) + .map(_.leaderAndIsr.leaderEpoch) + .getOrElse(LeaderAndIsr.NoEpoch) + } + } + def partitionLeadershipInfo(partition: TopicPartition): Option[LeaderIsrAndControllerEpoch] = { partitionLeadershipInfo.get(partition) } @@ -524,5 +545,4 @@ class ControllerContext { private def isValidPartitionStateTransition(partition: TopicPartition, targetState: PartitionState): Boolean = targetState.validPreviousStates.contains(partitionStates(partition)) - } diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index b473cd7afe61e..c7b4277705ac2 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -90,8 +90,14 @@ class KafkaController(val config: KafkaConfig, private val isAlterPartitionEnabled = config.interBrokerProtocolVersion.isAlterPartitionSupported private val stateChangeLogger = new StateChangeLogger(config.brokerId, inControllerContext = true, None) val controllerContext = new ControllerContext - var controllerChannelManager = new ControllerChannelManager(controllerContext, config, time, metrics, - stateChangeLogger, threadNamePrefix) + var controllerChannelManager = new ControllerChannelManager( + () => controllerContext.epoch, + config, + time, + metrics, + stateChangeLogger, + threadNamePrefix + ) // have a separate scheduler for the controller to be able to start and stop independently of the kafka server // visible for testing @@ -928,7 +934,7 @@ class KafkaController(val config: KafkaConfig, // update the leader and isr cache for all existing partitions from Zookeeper updateLeaderAndIsrCache() // start the channel manager - controllerChannelManager.startup() + controllerChannelManager.startup(controllerContext.liveOrShuttingDownBrokers) info(s"Currently active brokers in the cluster: ${controllerContext.liveBrokerIds}") info(s"Currently shutting brokers in the cluster: ${controllerContext.shuttingDownBrokerIds}") info(s"Current list of topics in the cluster: ${controllerContext.allTopics}") diff --git a/core/src/main/scala/kafka/migration/KRaftControllerToZkBrokersRpcClient.scala b/core/src/main/scala/kafka/migration/KRaftControllerToZkBrokersRpcClient.scala new file mode 100644 index 0000000000000..fe134795e2c51 --- /dev/null +++ b/core/src/main/scala/kafka/migration/KRaftControllerToZkBrokersRpcClient.scala @@ -0,0 +1,331 @@ +package kafka.migration + +import kafka.api.LeaderAndIsr +import kafka.cluster.Broker +import kafka.controller.{AbstractControllerBrokerRequestBatch, ControllerBrokerRequestMetadata, ControllerChannelManager, LeaderIsrAndControllerEpoch, ReplicaAssignment, StateChangeLogger} +import kafka.server.KafkaConfig +import org.apache.kafka.common.{TopicPartition, Uuid} +import org.apache.kafka.common.metrics.Metrics +import org.apache.kafka.common.protocol.Errors +import org.apache.kafka.common.requests._ +import org.apache.kafka.common.utils.Time +import org.apache.kafka.image.{MetadataDelta, MetadataImage} +import org.apache.kafka.metadata.migration.BrokersRpcClient + +import java.util +import scala.collection.mutable.ArrayBuffer +import scala.jdk.CollectionConverters._ + +object KRaftControllerBrokerRequestMetadata { + def isReplicaOnline(image: MetadataImage, brokerId: Int, replicaAssignment: Set[Int]): Boolean = { + val brokerOnline = image.cluster().containsBroker(brokerId) + brokerOnline && replicaAssignment.contains(brokerId) + } + + def partitionReplicaAssignment(image: MetadataImage, tp: TopicPartition): collection.Seq[Int] = { + image.topics().topicsByName().asScala.get(tp.topic()) match { + case Some(topic) => topic.partitions().asScala.get(tp.partition()) match { + case Some(partition) => partition.replicas.toSeq + case None => collection.Seq.empty + } + case None => collection.Seq.empty + } + } + + def partitionLeadershipInfo(image: MetadataImage, topicPartition: TopicPartition): Option[LeaderIsrAndControllerEpoch] = { + image.topics().topicsByName().asScala.get(topicPartition.topic()) match { + case Some(topic) => topic.partitions().asScala.get(topicPartition.partition()) match { + case Some(partition) => + val leaderAndIsr = LeaderAndIsr(partition.leader, partition.leaderEpoch, partition.isr.toList, + partition.leaderRecoveryState, partition.partitionEpoch) + Some(LeaderIsrAndControllerEpoch(leaderAndIsr, image.highestOffsetAndEpoch().epoch())) + case None => None + } + case None => None + } + } +} + +sealed class KRaftControllerBrokerRequestMetadata(val image: MetadataImage) extends + ControllerBrokerRequestMetadata { + override def isTopicDeletionInProgress(topicName: String): Boolean = { + !image.topics().topicsByName().containsKey(topicName) + } + + override val topicIds: collection.Map[String, Uuid] = { + image.topics().topicsByName().asScala.map { + case (name, topic) => name -> topic.id() + }.toMap + } + + override val liveBrokerIdAndEpochs: collection.Map[Int, Long] = { + image.cluster().zkBrokers().asScala.map { + case (brokerId, broker) => brokerId.intValue() -> broker.epoch() + } + } + + override val liveOrShuttingDownBrokers: collection.Set[Broker] = { + image.cluster().zkBrokers().asScala.values.map { registration => + Broker.fromBrokerRegistration(registration) + }.toSet + } + + override def isTopicQueuedUpForDeletion(topic: String): Boolean = { + !image.topics().topicsByName().containsKey(topic) + } + + override def isReplicaOnline(brokerId: Int, partition: TopicPartition): Boolean = { + KRaftControllerBrokerRequestMetadata.isReplicaOnline( + image, brokerId, partitionReplicaAssignment(partition).toSet) + } + + override def partitionReplicaAssignment(tp: TopicPartition): collection.Seq[Int] = { + KRaftControllerBrokerRequestMetadata.partitionReplicaAssignment(image, tp) + } + + override def leaderEpoch(topicPartition: TopicPartition): Int = { + // Topic is deleted use a special sentinel -2 to the indicate the same. + if (isTopicQueuedUpForDeletion(topicPartition.topic())) { + LeaderAndIsr.EpochDuringDelete + } else { + image.topics().topicsByName.asScala.get(topicPartition.topic()) match { + case Some(topic) => topic.partitions().asScala.get(topicPartition.partition()) match { + case Some(partition) => partition.leaderEpoch + case None => LeaderAndIsr.NoEpoch + } + case None => LeaderAndIsr.NoEpoch + } + } + } + + override val liveOrShuttingDownBrokerIds: collection.Set[Int] = liveBrokerIdAndEpochs.keySet + + override def partitionLeadershipInfo(topicPartition: TopicPartition): Option[LeaderIsrAndControllerEpoch] = { + KRaftControllerBrokerRequestMetadata.partitionLeadershipInfo(image, topicPartition) + } +} + +sealed class KRaftControllerBrokerRequestBatch( + config: KafkaConfig, + metadataProvider: () => ControllerBrokerRequestMetadata, + controllerChannelManager: ControllerChannelManager, + stateChangeLogger: StateChangeLogger +) extends AbstractControllerBrokerRequestBatch( + config, + metadataProvider, + stateChangeLogger, + kraftController = true +) { + + override def sendRequest(brokerId: Int, request: AbstractControlRequest.Builder[_ <: AbstractControlRequest], callback: AbstractResponse => Unit): Unit = { + controllerChannelManager.sendRequest(brokerId, request, callback) + } + + override def handleLeaderAndIsrResponse(response: LeaderAndIsrResponse, broker: Int): Unit = { + if (response.error != Errors.NONE) { + stateChangeLogger.error(s"Received error ${response.error} in LeaderAndIsr " + + s"response $response from broker $broker") + return + } + val partitionErrors = response.partitionErrors( + metadataProvider().topicIds.map { case (id, string) => (string, id) }.asJava) + val offlineReplicas = new ArrayBuffer[TopicPartition]() + partitionErrors.forEach{ case(tp, error) => + if (error == Errors.KAFKA_STORAGE_ERROR) { + offlineReplicas += tp + } + } + if (offlineReplicas.nonEmpty) { + stateChangeLogger.error(s"Found ${offlineReplicas.mkString(",")} on broker $broker as offline") + } + } + + override def handleUpdateMetadataResponse(response: UpdateMetadataResponse, broker: Int): Unit = { + if (response.error != Errors.NONE) { + stateChangeLogger.error(s"Received error ${response.error} in UpdateMetadata " + + s"response $response from broker $broker") + } + } + + override def handleStopReplicaResponse(response: StopReplicaResponse, broker: Int, + partitionErrorsForDeletingTopics: Map[TopicPartition, Errors]): Unit = { + if (response.error() != Errors.NONE) { + stateChangeLogger.error(s"Received error ${response.error} in StopReplica " + + s"response $response from broker $broker") + } + partitionErrorsForDeletingTopics.foreach{ case(tp, error) => + if (error != Errors.NONE) { + stateChangeLogger.error(s"Received error $error in StopReplica request for partition $tp " + + s"from broker $broker") + } + } + } +} + +class KRaftControllerToZkBrokersRpcClient( + nodeId: Int, + config: KafkaConfig +) extends BrokersRpcClient { + @volatile private var _image = MetadataImage.EMPTY + val stateChangeLogger = new StateChangeLogger(nodeId, inControllerContext = false, None) + val channelManager = new ControllerChannelManager( + () => _image.highestOffsetAndEpoch().epoch(), + config, + Time.SYSTEM, + new Metrics(), + stateChangeLogger + ) + + val requestBatch = new KRaftControllerBrokerRequestBatch( + config, + metadataProvider, + channelManager, + stateChangeLogger + ) + + private def metadataProvider(): ControllerBrokerRequestMetadata = { + new KRaftControllerBrokerRequestMetadata(_image) + } + + def startup(): Unit = { + channelManager.startup(Set.empty) + } + + def shutdown(): Unit = { + clear() + channelManager.shutdown() + } + + override def publishMetadata(image: MetadataImage): Unit = { + val oldImage = _image + val addedBrokers = new util.HashSet[Integer](image.cluster().brokers().keySet()) + addedBrokers.removeAll(oldImage.cluster().brokers().keySet()) + val removedBrokers = new util.HashSet[Integer](oldImage.cluster().brokers().keySet()) + removedBrokers.removeAll(image.cluster().brokers().keySet()) + + removedBrokers.asScala.foreach(id => channelManager.removeBroker(id)) + addedBrokers.asScala.foreach(id => + channelManager.addBroker(Broker.fromBrokerRegistration(image.cluster().broker(id)))) + _image = image + } + + override def sendRPCsToBrokersFromMetadataDelta(delta: MetadataDelta, image: MetadataImage, + controllerEpoch: Int): Unit = { + publishMetadata(image) + requestBatch.newBatch() + + val newZkBrokers = delta.clusterDelta().newZkBrokers().asScala.map(_.toInt).toSet + val zkBrokers = image.cluster().zkBrokers().keySet().asScala.map(_.toInt).toSet + val oldZkBrokers = zkBrokers -- newZkBrokers + val newBrokersFound = !delta.clusterDelta().newBrokers().isEmpty + + if (newZkBrokers.nonEmpty) { + // Update new Zk brokers about all the metadata. + requestBatch.addUpdateMetadataRequestForBrokers(newZkBrokers.toSeq, image.topics().partitions().keySet().asScala) + // Send these requests first to make sure, we don't add all the partition metadata to the + // old brokers as well. + requestBatch.sendRequestsToBrokers(controllerEpoch) + requestBatch.newBatch() + + // For new the brokers, check if there are partition assignments and add LISR appropriately. + image.topics().partitions().asScala.foreach { case (tp, partitionRegistration) => + val replicas = partitionRegistration.replicas.toSet + val leaderIsrAndControllerEpochOpt = KRaftControllerBrokerRequestMetadata.partitionLeadershipInfo(image, tp) + val newBrokersWithReplicas = replicas.intersect(newZkBrokers) + if (newBrokersWithReplicas.nonEmpty) { + leaderIsrAndControllerEpochOpt match { + case Some(leaderIsrAndControllerEpoch) => + val replicaAssignment = ReplicaAssignment(partitionRegistration.replicas, + partitionRegistration.addingReplicas, partitionRegistration.removingReplicas) + requestBatch.addLeaderAndIsrRequestForBrokers(newBrokersWithReplicas.toSeq, tp, + leaderIsrAndControllerEpoch, replicaAssignment, isNew = true) + case None => + } + } + } + } + + // If there are new brokers (including KRaft brokers) or if there are changes in topic + // metadata, let's send UMR about the changes to the old Zk brokers. + if (newBrokersFound || !delta.topicsDelta().deletedTopicIds().isEmpty || !delta.topicsDelta().changedTopics().isEmpty) { + requestBatch.addUpdateMetadataRequestForBrokers(oldZkBrokers.toSeq) + } + + // Handle deleted topics by sending appropriate StopReplica and UMR requests to the brokers. + delta.topicsDelta().deletedTopicIds().asScala.foreach { deletedTopicId => + val deletedTopic = delta.image().topics().getTopic(deletedTopicId) + deletedTopic.partitions().asScala.foreach { case (partition, partitionRegistration) => + val tp = new TopicPartition(deletedTopic.name(), partition) + val offlineReplicas = partitionRegistration.replicas.filter { + KRaftControllerBrokerRequestMetadata.isReplicaOnline(image, _, partitionRegistration.replicas.toSet) + } + val deletedLeaderAndIsr = LeaderAndIsr.duringDelete(partitionRegistration.isr.toList) + requestBatch.addStopReplicaRequestForBrokers(partitionRegistration.replicas, tp, deletePartition = true) + requestBatch.addUpdateMetadataRequestForBrokers( + oldZkBrokers.toSeq, controllerEpoch, tp, deletedLeaderAndIsr.leader, deletedLeaderAndIsr.leaderEpoch, + deletedLeaderAndIsr.partitionEpoch, deletedLeaderAndIsr.isr, partitionRegistration.replicas, offlineReplicas) + } + } + + // Handle changes in other topics and send appropriate LeaderAndIsr and UMR requests to the + // brokers. + delta.topicsDelta().changedTopics().asScala.foreach { case (_, topicDelta) => + topicDelta.partitionChanges().asScala.foreach { case (partition, partitionRegistration) => + val tp = new TopicPartition(topicDelta.name(), partition) + + // Check for replica leadership changes. + val leaderIsrAndControllerEpochOpt = KRaftControllerBrokerRequestMetadata.partitionLeadershipInfo(image, tp) + leaderIsrAndControllerEpochOpt match { + case Some(leaderIsrAndControllerEpoch) => + val replicaAssignment = ReplicaAssignment(partitionRegistration.replicas, + partitionRegistration.addingReplicas, partitionRegistration.removingReplicas) + requestBatch.addLeaderAndIsrRequestForBrokers(replicaAssignment.replicas, tp, + leaderIsrAndControllerEpoch, replicaAssignment, isNew = true) + case None => + } + + // Check for removed replicas. + val oldReplicas = + Option(delta.image().topics().getPartition(topicDelta.id(), tp.partition())) + .map(_.replicas.toSet) + .getOrElse(Set.empty) + val newReplicas = partitionRegistration.replicas.toSet + val removedReplicas = oldReplicas -- newReplicas + if (removedReplicas.nonEmpty) { + requestBatch.addStopReplicaRequestForBrokers(removedReplicas.toSeq, tp, deletePartition = false) + } + } + } + // Send all the accumulated requests to the broker. + requestBatch.sendRequestsToBrokers(controllerEpoch) + } + + override def sendRPCsToBrokersFromMetadataImage(image: MetadataImage, controllerEpoch: Int): Unit = { + publishMetadata(image) + requestBatch.newBatch() + + // When we need to send RPCs from the image, we're sending 'full' requests meaning we let + // zk every broker know about all the metadata and all the LISR requests it needs to handle. + // Note that we cannot send StopReplica requests from image because we don't have any state + // about brokers that host a replica but not part of the replica set known by the Controller. + val zkBrokers = image.cluster().zkBrokers().keySet().asScala.map(_.toInt).toSeq + val partitions = image.topics().partitions() + partitions.asScala.foreach{ case (tp, partitionRegistration) => + val leaderIsrAndControllerEpochOpt = KRaftControllerBrokerRequestMetadata.partitionLeadershipInfo(image, tp) + leaderIsrAndControllerEpochOpt match { + case Some(leaderIsrAndControllerEpoch) => + val replicaAssignment = ReplicaAssignment(partitionRegistration.replicas, + partitionRegistration.addingReplicas, partitionRegistration.removingReplicas) + requestBatch.addLeaderAndIsrRequestForBrokers(replicaAssignment.replicas, tp, + leaderIsrAndControllerEpoch, replicaAssignment, isNew = true) + case None => None + } + } + requestBatch.addUpdateMetadataRequestForBrokers(zkBrokers, partitions.keySet().asScala) + requestBatch.sendRequestsToBrokers(controllerEpoch) + } + + override def clear(): Unit = { + requestBatch.clear() + } +} diff --git a/core/src/main/scala/kafka/server/ControllerServer.scala b/core/src/main/scala/kafka/server/ControllerServer.scala index e6975c4851023..134e2de62d191 100644 --- a/core/src/main/scala/kafka/server/ControllerServer.scala +++ b/core/src/main/scala/kafka/server/ControllerServer.scala @@ -22,6 +22,7 @@ import java.util.concurrent.locks.ReentrantLock import java.util.concurrent.{CompletableFuture, TimeUnit} import kafka.cluster.Broker.ServerInfo import kafka.metrics.{KafkaMetricsGroup, LinuxIoMetricsCollector} +import kafka.migration.KRaftControllerToZkBrokersRpcClient import kafka.network.{DataPlaneAcceptor, SocketServer} import kafka.raft.KafkaRaftManager import kafka.security.CredentialProvider @@ -64,7 +65,7 @@ case class ControllerMigrationSupport( CoreUtils.swallow(zkClient.close(), logging) } if (brokersRpcClient != null) { - // TODO + CoreUtils.swallow(brokersRpcClient.shutdown(), logging) } } } @@ -241,7 +242,7 @@ class ControllerServer( if (config.migrationEnabled) { val zkClient = KafkaZkClient.createZkClient("KRaft Migration", time, config, KafkaServer.zkClientConfigFromKafkaConfig(config)) val migrationClient = new ZkMigrationClient(zkClient) - val rpcClient: BrokersRpcClient = null + val rpcClient: BrokersRpcClient = new KRaftControllerToZkBrokersRpcClient(config.nodeId, config) val migrationDriver = new KRaftMigrationDriver(config.nodeId, controller.asInstanceOf[QuorumController].zkRecordConsumer(), migrationClient, rpcClient) sharedServer.loader.installPublishers(java.util.Collections.singletonList(migrationDriver)) migrationDriver.start() diff --git a/core/src/test/scala/unit/kafka/controller/ControllerChannelManagerTest.scala b/core/src/test/scala/unit/kafka/controller/ControllerChannelManagerTest.scala index 8e9176af2e9ab..c08b8fdc614f2 100644 --- a/core/src/test/scala/unit/kafka/controller/ControllerChannelManagerTest.scala +++ b/core/src/test/scala/unit/kafka/controller/ControllerChannelManagerTest.scala @@ -941,14 +941,11 @@ class ControllerChannelManagerTest { private case class SentRequest(request: ControlRequest, responseCallback: AbstractResponse => Unit) private class MockControllerBrokerRequestBatch(context: ControllerContext, config: KafkaConfig = config) - extends AbstractControllerBrokerRequestBatch(config, context, logger) { + extends AbstractControllerBrokerRequestBatch(config, () => context, logger) { val sentEvents = ListBuffer.empty[ControllerEvent] val sentRequests = mutable.Map.empty[Int, ListBuffer[SentRequest]] - override def sendEvent(event: ControllerEvent): Unit = { - sentEvents.append(event) - } override def sendRequest(brokerId: Int, request: ControlRequest, callback: AbstractResponse => Unit): Unit = { sentRequests.getOrElseUpdate(brokerId, ListBuffer.empty) sentRequests(brokerId).append(SentRequest(request, callback)) @@ -980,6 +977,21 @@ class ControllerChannelManagerTest { case None => List.empty[LeaderAndIsrRequest] } } + + override def handleLeaderAndIsrResponse(response: LeaderAndIsrResponse, broker: Int): Unit = { + sentEvents.append(LeaderAndIsrResponseReceived(response, broker)) + } + + override def handleUpdateMetadataResponse(response: UpdateMetadataResponse, broker: Int) + : Unit = { + sentEvents.append(UpdateMetadataResponseReceived(response, broker)) + } + + override def handleStopReplicaResponse(stopReplicaResponse: StopReplicaResponse, + brokerId: Int, partitionErrorsForDeletingTopics: Map[TopicPartition, Errors]): Unit = { + if (partitionErrorsForDeletingTopics.nonEmpty) + sentEvents.append(TopicDeletionStopReplicaResponseReceived(brokerId, stopReplicaResponse.error, partitionErrorsForDeletingTopics)) + } } } diff --git a/core/src/test/scala/unit/kafka/server/BrokerEpochIntegrationTest.scala b/core/src/test/scala/unit/kafka/server/BrokerEpochIntegrationTest.scala index 250f87cbd7330..02fd74a5aa045 100755 --- a/core/src/test/scala/unit/kafka/server/BrokerEpochIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/server/BrokerEpochIntegrationTest.scala @@ -131,9 +131,14 @@ class BrokerEpochIntegrationTest extends QuorumTestHarness { val controllerContext = new ControllerContext controllerContext.setLiveBrokers(brokerAndEpochs) val metrics = new Metrics - val controllerChannelManager = new ControllerChannelManager(controllerContext, controllerConfig, Time.SYSTEM, - metrics, new StateChangeLogger(controllerId, inControllerContext = true, None)) - controllerChannelManager.startup() + val controllerChannelManager = new ControllerChannelManager( + () => controllerContext.epoch, + controllerConfig, + Time.SYSTEM, + metrics, + new StateChangeLogger(controllerId, inControllerContext = true, None) + ) + controllerChannelManager.startup(controllerContext.liveOrShuttingDownBrokers) val broker2 = servers(brokerId2) val epochInRequest = broker2.kafkaController.brokerEpoch + epochInRequestDiffFromCurrentEpoch diff --git a/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala b/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala index f0dea91335e65..259ea91df87aa 100755 --- a/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala +++ b/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala @@ -138,9 +138,14 @@ class LeaderElectionTest extends QuorumTestHarness { val controllerContext = new ControllerContext controllerContext.setLiveBrokers(brokerAndEpochs) val metrics = new Metrics - val controllerChannelManager = new ControllerChannelManager(controllerContext, controllerConfig, Time.SYSTEM, - metrics, new StateChangeLogger(controllerId, inControllerContext = true, None)) - controllerChannelManager.startup() + val controllerChannelManager = new ControllerChannelManager( + () => controllerContext.epoch, + controllerConfig, + Time.SYSTEM, + metrics, + new StateChangeLogger(controllerId, inControllerContext = true, None) + ) + controllerChannelManager.startup(controllerContext.liveOrShuttingDownBrokers) try { val staleControllerEpoch = 0 val partitionStates = Seq( diff --git a/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala b/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala index 70554d9427c2f..50b1649803730 100644 --- a/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala +++ b/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala @@ -280,9 +280,13 @@ class ServerShutdownTest extends KafkaServerTestHarness { val controllerConfig = KafkaConfig.fromProps(TestUtils.createBrokerConfig(controllerId, zkConnect)) val controllerContext = new ControllerContext controllerContext.setLiveBrokers(brokerAndEpochs) - controllerChannelManager = new ControllerChannelManager(controllerContext, controllerConfig, Time.SYSTEM, - metrics, new StateChangeLogger(controllerId, inControllerContext = true, None)) - controllerChannelManager.startup() + controllerChannelManager = new ControllerChannelManager( + () => controllerContext.epoch, + controllerConfig, + Time.SYSTEM, + metrics, + new StateChangeLogger(controllerId, inControllerContext = true, None)) + controllerChannelManager.startup(controllerContext.liveOrShuttingDownBrokers) // Initiate a sendRequest and wait until connection is established and one byte is received by the peer val requestBuilder = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, diff --git a/metadata/src/main/java/org/apache/kafka/controller/MigrationControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/MigrationControlManager.java index 3001decb85c81..7f3f700576887 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/MigrationControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/MigrationControlManager.java @@ -1,3 +1,19 @@ +/* + * 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.metadata.ZkMigrationStateRecord; diff --git a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java index 4fc4c576887d7..c57fbdc64d6a0 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java +++ b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java @@ -48,7 +48,24 @@ import org.apache.kafka.common.message.ListPartitionReassignmentsResponseData; import org.apache.kafka.common.message.UpdateFeaturesRequestData; import org.apache.kafka.common.message.UpdateFeaturesResponseData; -import org.apache.kafka.common.metadata.*; +import org.apache.kafka.common.metadata.AccessControlEntryRecord; +import org.apache.kafka.common.metadata.BrokerRegistrationChangeRecord; +import org.apache.kafka.common.metadata.ClientQuotaRecord; +import org.apache.kafka.common.metadata.ConfigRecord; +import org.apache.kafka.common.metadata.FeatureLevelRecord; +import org.apache.kafka.common.metadata.FenceBrokerRecord; +import org.apache.kafka.common.metadata.MetadataRecordType; +import org.apache.kafka.common.metadata.NoOpRecord; +import org.apache.kafka.common.metadata.PartitionChangeRecord; +import org.apache.kafka.common.metadata.PartitionRecord; +import org.apache.kafka.common.metadata.ProducerIdsRecord; +import org.apache.kafka.common.metadata.RegisterBrokerRecord; +import org.apache.kafka.common.metadata.RemoveAccessControlEntryRecord; +import org.apache.kafka.common.metadata.RemoveTopicRecord; +import org.apache.kafka.common.metadata.TopicRecord; +import org.apache.kafka.common.metadata.UnfenceBrokerRecord; +import org.apache.kafka.common.metadata.UnregisterBrokerRecord; +import org.apache.kafka.common.metadata.ZkMigrationStateRecord; import org.apache.kafka.common.protocol.ApiMessage; import org.apache.kafka.common.quota.ClientQuotaAlteration; import org.apache.kafka.common.quota.ClientQuotaEntity; diff --git a/metadata/src/main/java/org/apache/kafka/image/ClusterDelta.java b/metadata/src/main/java/org/apache/kafka/image/ClusterDelta.java index e0dcb7cc580be..f0d0fceb1d63d 100644 --- a/metadata/src/main/java/org/apache/kafka/image/ClusterDelta.java +++ b/metadata/src/main/java/org/apache/kafka/image/ClusterDelta.java @@ -68,6 +68,15 @@ public Set newZkBrokers() { .collect(Collectors.toSet()); } + public Set newBrokers() { + return changedBrokers + .entrySet() + .stream() + .filter(entry -> !image.containsBroker(entry.getKey()) && entry.getValue().isPresent()) + .map(Entry::getKey) + .collect(Collectors.toSet()); + } + public void finishSnapshot() { for (Integer brokerId : image.brokers().keySet()) { if (!changedBrokers.containsKey(brokerId)) { diff --git a/metadata/src/main/java/org/apache/kafka/image/TopicsImage.java b/metadata/src/main/java/org/apache/kafka/image/TopicsImage.java index 5f7db112f0aca..392bea4119b8f 100644 --- a/metadata/src/main/java/org/apache/kafka/image/TopicsImage.java +++ b/metadata/src/main/java/org/apache/kafka/image/TopicsImage.java @@ -17,6 +17,7 @@ package org.apache.kafka.image; +import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.Uuid; import org.apache.kafka.image.writer.ImageWriter; import org.apache.kafka.image.writer.ImageWriterOptions; @@ -24,6 +25,7 @@ import org.apache.kafka.server.util.TranslatedValueMapView; import java.util.Collections; +import java.util.HashMap; import java.util.Map; import java.util.Objects; import java.util.stream.Collectors; @@ -110,6 +112,14 @@ public Map topicIdToNameView() { return new TranslatedValueMapView<>(topicsById, image -> image.name()); } + public Map partitions() { + Map partitions = new HashMap<>(); + topicsById.values().forEach(topic -> { + topic.partitions().forEach((key, value) -> partitions.put(new TopicPartition(topic.name(), key), value)); + }); + return partitions; + } + @Override public String toString() { return "TopicsImage(topicsById=" + topicsById.entrySet().stream(). diff --git a/metadata/src/main/java/org/apache/kafka/metadata/migration/BrokersRpcClient.java b/metadata/src/main/java/org/apache/kafka/metadata/migration/BrokersRpcClient.java index 3d7f21ce956de..727498ec64c14 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/migration/BrokersRpcClient.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/migration/BrokersRpcClient.java @@ -19,15 +19,13 @@ import org.apache.kafka.image.MetadataDelta; import org.apache.kafka.image.MetadataImage; -import java.util.Optional; - public interface BrokersRpcClient { - void addZkBroker(); - void removeZkBroker(); + void startup(); + + void shutdown(); - void publishMetadata(MetadataImage image, - Optional deltaOpt); + void publishMetadata(MetadataImage image); void sendRPCsToBrokersFromMetadataDelta(MetadataDelta delta, MetadataImage image, @@ -35,5 +33,5 @@ void sendRPCsToBrokersFromMetadataDelta(MetadataDelta delta, void sendRPCsToBrokersFromMetadataImage(MetadataImage image, int controllerEpoch); - void reset(); + void clear(); } diff --git a/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java b/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java index 8cd8b4991be97..5baeac8dcf8c5 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java @@ -18,8 +18,6 @@ import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; -import org.apache.kafka.controller.Controller; -import org.apache.kafka.controller.MigrationControlManager; import org.apache.kafka.image.MetadataDelta; import org.apache.kafka.image.MetadataImage; import org.apache.kafka.image.loader.LogDeltaManifest; @@ -85,6 +83,10 @@ public void start() { eventQueue.prepend(new PollEvent()); } + public void shutdown() throws InterruptedException { + eventQueue.close(); + } + private void initializeMigrationState() { log.info("Recovering migration state"); apply("Recovery", zkMigrationClient::getOrCreateMigrationRecoveryState); diff --git a/metadata/src/main/java/org/apache/kafka/metadata/migration/ZkRecordConsumer.java b/metadata/src/main/java/org/apache/kafka/metadata/migration/ZkRecordConsumer.java index 70b2f0aaa0a93..4e35b719d14ba 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/migration/ZkRecordConsumer.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/migration/ZkRecordConsumer.java @@ -1,3 +1,19 @@ +/* + * 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.metadata.migration; import org.apache.kafka.raft.OffsetAndEpoch; From 7da1d1ff30dca9caa072de4c8aecd1d7bbe86233 Mon Sep 17 00:00:00 2001 From: David Arthur Date: Mon, 19 Dec 2022 15:34:40 -0500 Subject: [PATCH 08/30] Progress on KRaft to ZK dual write --- .../scala/kafka/server/ControllerServer.scala | 9 +- .../scala/kafka/server/SharedServer.scala | 4 +- .../scala/kafka/zk/ZkMigrationClient.scala | 24 ++++- .../unit/kafka/zk/ZkMigrationClientTest.scala | 16 +++ .../kafka/controller/QuorumController.java | 1 - .../migration/KRaftMigrationDriver.java | 98 +++++++++++++------ 6 files changed, 111 insertions(+), 41 deletions(-) diff --git a/core/src/main/scala/kafka/server/ControllerServer.scala b/core/src/main/scala/kafka/server/ControllerServer.scala index e6975c4851023..bea2590e4bf33 100644 --- a/core/src/main/scala/kafka/server/ControllerServer.scala +++ b/core/src/main/scala/kafka/server/ControllerServer.scala @@ -242,8 +242,13 @@ class ControllerServer( val zkClient = KafkaZkClient.createZkClient("KRaft Migration", time, config, KafkaServer.zkClientConfigFromKafkaConfig(config)) val migrationClient = new ZkMigrationClient(zkClient) val rpcClient: BrokersRpcClient = null - val migrationDriver = new KRaftMigrationDriver(config.nodeId, controller.asInstanceOf[QuorumController].zkRecordConsumer(), migrationClient, rpcClient) - sharedServer.loader.installPublishers(java.util.Collections.singletonList(migrationDriver)) + val migrationDriver = new KRaftMigrationDriver( + config.nodeId, + controller.asInstanceOf[QuorumController].zkRecordConsumer(), + migrationClient, + rpcClient, + publisher => sharedServer.loader.installPublishers(java.util.Collections.singletonList(publisher)) + ) migrationDriver.start() migrationSupport = Some(ControllerMigrationSupport(zkClient, migrationDriver, rpcClient)) } diff --git a/core/src/main/scala/kafka/server/SharedServer.scala b/core/src/main/scala/kafka/server/SharedServer.scala index 151429a1797db..6a7cf8a830467 100644 --- a/core/src/main/scala/kafka/server/SharedServer.scala +++ b/core/src/main/scala/kafka/server/SharedServer.scala @@ -35,7 +35,7 @@ import org.apache.kafka.server.metrics.KafkaYammerMetrics import java.util import java.util.Collections -import java.util.concurrent.CompletableFuture +import java.util.concurrent.{CompletableFuture, TimeUnit} import java.util.concurrent.atomic.AtomicReference @@ -248,7 +248,7 @@ class SharedServer( setTime(time). setFaultHandler(metadataPublishingFaultHandler). setMaxBytesSinceLastSnapshot(sharedServerConfig.metadataSnapshotMaxNewRecordBytes). - setMaxTimeSinceLastSnapshotNs(sharedServerConfig.metadataSnapshotMaxIntervalMs). + setMaxTimeSinceLastSnapshotNs(TimeUnit.MILLISECONDS.toNanos(sharedServerConfig.metadataSnapshotMaxIntervalMs)). setDisabledReason(snapshotsDiabledReason). build() raftManager.register(loader) diff --git a/core/src/main/scala/kafka/zk/ZkMigrationClient.scala b/core/src/main/scala/kafka/zk/ZkMigrationClient.scala index 3a9490bd7e5e0..c3c0d832149e5 100644 --- a/core/src/main/scala/kafka/zk/ZkMigrationClient.scala +++ b/core/src/main/scala/kafka/zk/ZkMigrationClient.scala @@ -271,8 +271,18 @@ class ZkMigrationClient(zkClient: KafkaZkClient) extends MigrationClient with Lo } val requests = Seq(createTopicZNode, createPartitionsZNode) ++ createPartitionZNodeReqs - val (migrationZkVersion, _) = zkClient.retryMigrationRequestsUntilConnected(requests, state) - state.withMigrationZkVersion(migrationZkVersion) + val (migrationZkVersion, responses) = zkClient.retryMigrationRequestsUntilConnected(requests, state) + val resultCodes = responses.map { response => response.path -> response.resultCode }.toMap + if (resultCodes(TopicZNode.path(topicName)).equals(Code.NODEEXISTS)) { + // topic already created, just return + state + } else if (resultCodes.forall { case (_, code) => code.equals(Code.OK) } ) { + // ok + state.withMigrationZkVersion(migrationZkVersion) + } else { + // not ok + throw new RuntimeException(s"Failed to create or update topic $topicName. ZK operation had results $resultCodes") + } } private def createTopicPartition(topicPartition: TopicPartition): CreateRequest = { @@ -318,8 +328,13 @@ class ZkMigrationClient(zkClient: KafkaZkClient) extends MigrationClient with Lo if (requests.isEmpty) { state } else { - val (migrationZkVersion, _) = zkClient.retryMigrationRequestsUntilConnected(requests.toSeq, state) - state.withMigrationZkVersion(migrationZkVersion) + val (migrationZkVersion, responses) = zkClient.retryMigrationRequestsUntilConnected(requests.toSeq, state) + val resultCodes = responses.map { response => response.path -> response.resultCode }.toMap + if (resultCodes.forall { case (_, code) => code.equals(Code.OK) } ) { + state.withMigrationZkVersion(migrationZkVersion) + } else { + throw new RuntimeException(s"Failed to update partition states: $topicPartitions. ZK transaction had results $resultCodes") + } } } @@ -443,7 +458,6 @@ class ZkMigrationClient(zkClient: KafkaZkClient) extends MigrationClient with Lo override def writeMetadataDeltaToZookeeper(delta: MetadataDelta, image: MetadataImage, state: ZkMigrationLeadershipState): ZkMigrationLeadershipState = { - // TODO state } } diff --git a/core/src/test/scala/unit/kafka/zk/ZkMigrationClientTest.scala b/core/src/test/scala/unit/kafka/zk/ZkMigrationClientTest.scala index a8493d027d5b6..4c77eb5b76316 100644 --- a/core/src/test/scala/unit/kafka/zk/ZkMigrationClientTest.scala +++ b/core/src/test/scala/unit/kafka/zk/ZkMigrationClientTest.scala @@ -143,6 +143,22 @@ class ZkMigrationClientTest extends QuorumTestHarness { assertEquals(List(1, 2, 3), partition1.isr) } + @Test + def testIdempotentCreateTopics(): Unit = { + assertEquals(0, migrationState.migrationZkVersion()) + + val partitions = Map( + 0 -> new PartitionRegistration(Array(0, 1, 2), Array(0, 1, 2), Array(), Array(), 0, LeaderRecoveryState.RECOVERED, 0, -1), + 1 -> new PartitionRegistration(Array(1, 2, 3), Array(1, 2, 3), Array(), Array(), 1, LeaderRecoveryState.RECOVERED, 0, -1) + ).map { case (k, v) => Integer.valueOf(k) -> v }.asJava + val topicId = Uuid.randomUuid() + migrationState = migrationClient.createTopic("test", topicId, partitions, migrationState) + assertEquals(1, migrationState.migrationZkVersion()) + + migrationState = migrationClient.createTopic("test", topicId, partitions, migrationState) + assertEquals(1, migrationState.migrationZkVersion()) + } + // Write Client Quotas using ZkMigrationClient and read them back using AdminZkClient private def writeClientQuotaAndVerify(migrationClient: ZkMigrationClient, adminZkClient: AdminZkClient, diff --git a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java index 4fc4c576887d7..06b5d7340b7d8 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java +++ b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java @@ -820,7 +820,6 @@ class MigrationWriteOperation implements ControllerWriteOperation { } @Override public ControllerResult generateRecordsAndResult() { - log.info("Migrating batch {}", batch); return ControllerResult.atomicOf(batch, null); } diff --git a/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java b/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java index 8cd8b4991be97..cad5e9d74a96f 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java @@ -18,26 +18,25 @@ import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; -import org.apache.kafka.controller.Controller; -import org.apache.kafka.controller.MigrationControlManager; import org.apache.kafka.image.MetadataDelta; import org.apache.kafka.image.MetadataImage; +import org.apache.kafka.image.MetadataProvenance; import org.apache.kafka.image.loader.LogDeltaManifest; import org.apache.kafka.image.loader.SnapshotManifest; import org.apache.kafka.image.publisher.MetadataPublisher; import org.apache.kafka.queue.EventQueue; import org.apache.kafka.queue.KafkaEventQueue; - - import org.apache.kafka.raft.LeaderAndEpoch; import org.apache.kafka.raft.OffsetAndEpoch; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.Collections; import java.util.HashSet; import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; +import java.util.function.Consumer; import java.util.function.Function; import static java.util.concurrent.TimeUnit.NANOSECONDS; @@ -55,17 +54,22 @@ public class KRaftMigrationDriver implements MetadataPublisher { private final BrokersRpcClient rpcClient; private final ZkRecordConsumer zkRecordConsumer; private final KafkaEventQueue eventQueue; + /** + * A callback for when the migration state has been recovered from ZK. This is used to delay the installation of this + * MetadataPublisher with MetadataLoader. + */ + private final Consumer initialZkLoadHandler; private volatile LeaderAndEpoch leaderAndEpoch; private volatile MigrationState migrationState; private volatile ZkMigrationLeadershipState migrationLeadershipState; - private volatile MetadataDelta delta; private volatile MetadataImage image; public KRaftMigrationDriver( int nodeId, ZkRecordConsumer zkRecordConsumer, MigrationClient zkMigrationClient, - BrokersRpcClient rpcClient + BrokersRpcClient rpcClient, + Consumer initialZkLoadHandler ) { this.nodeId = nodeId; this.zkRecordConsumer = zkRecordConsumer; @@ -76,9 +80,9 @@ public KRaftMigrationDriver( this.migrationState = MigrationState.UNINITIALIZED; this.migrationLeadershipState = ZkMigrationLeadershipState.EMPTY; this.eventQueue = new KafkaEventQueue(Time.SYSTEM, new LogContext("KRaftMigrationDriver"), "kraft-migration"); - this.delta = null; this.image = MetadataImage.EMPTY; this.leaderAndEpoch = LeaderAndEpoch.UNKNOWN; + this.initialZkLoadHandler = initialZkLoadHandler; } public void start() { @@ -90,6 +94,7 @@ private void initializeMigrationState() { apply("Recovery", zkMigrationClient::getOrCreateMigrationRecoveryState); String maybeDone = migrationLeadershipState.zkMigrationComplete() ? "done" : "not done"; log.info("Recovered migration state {}. ZK migration is {}.", migrationLeadershipState, maybeDone); + initialZkLoadHandler.accept(this); // Let's transition to INACTIVE state and wait for leadership events. transitionTo(MigrationState.INACTIVE); } @@ -100,6 +105,11 @@ private boolean isControllerQuorumReadyForMigration() { } private boolean areZkBrokersReadyForMigration() { + if (image == MetadataImage.EMPTY) { + // TODO maybe add WAIT_FOR_INITIAL_METADATA_PUBLISH state to avoid this kind of check? + log.info("Waiting for initial metadata publish before checking if Zk brokers are registered."); + return false; + } Set kraftRegisteredZkBrokers = image.cluster().zkBrokers().keySet(); Set zkRegisteredZkBrokers = zkMigrationClient.readBrokerIdsFromTopicAssignments(); zkRegisteredZkBrokers.removeAll(kraftRegisteredZkBrokers); @@ -114,7 +124,7 @@ private boolean areZkBrokersReadyForMigration() { private void apply(String name, Function stateMutator) { ZkMigrationLeadershipState beforeState = this.migrationLeadershipState; ZkMigrationLeadershipState afterState = stateMutator.apply(beforeState); - log.debug("{} transitioned from {} to {}", name, beforeState, afterState); + log.trace("{} transitioned from {} to {}", name, beforeState, afterState); this.migrationLeadershipState = afterState; } @@ -159,7 +169,11 @@ private void transitionTo(MigrationState newState) { log.error("Error transition in migration driver from {} to {}", migrationState, newState); return; } - log.debug("{} transitioning from {} to {} state", nodeId, migrationState, newState); + if (newState != migrationState) { + log.debug("{} transitioning from {} to {} state", nodeId, migrationState, newState); + } else { + log.trace("{} transitioning from {} to {} state", nodeId, migrationState, newState); + } switch (newState) { case UNINITIALIZED: // No state can transition to UNITIALIZED. @@ -179,7 +193,7 @@ public String name() { @Override public void publishSnapshot(MetadataDelta delta, MetadataImage newImage, SnapshotManifest manifest) { - eventQueue.append(new MetadataChangeEvent(delta, newImage)); + eventQueue.append(new MetadataChangeEvent(delta, newImage, manifest.provenance(), true)); } @Override @@ -187,8 +201,7 @@ public void publishLogDelta(MetadataDelta delta, MetadataImage newImage, LogDelt if (!leaderAndEpoch.equals(manifest.leaderAndEpoch())) { eventQueue.append(new KRaftLeaderEvent(manifest.leaderAndEpoch())); } - - eventQueue.append(new MetadataChangeEvent(delta, newImage)); + eventQueue.append(new MetadataChangeEvent(delta, newImage, manifest.provenance(), false)); } @@ -367,10 +380,12 @@ class MigrateMetadataEvent implements EventQueue.Event { @Override public void run() throws Exception { Set brokersInMetadata = new HashSet<>(); + log.info("Starting ZK migration"); zkRecordConsumer.beginMigration(); try { zkMigrationClient.readAllMetadata(batch -> { try { + log.info("Migrating {} records from ZK: {}", batch.size(), batch); CompletableFuture future = zkRecordConsumer.acceptBatch(batch); future.get(); } catch (InterruptedException e) { @@ -380,7 +395,7 @@ public void run() throws Exception { } }, brokersInMetadata::add); OffsetAndEpoch offsetAndEpochAfterMigration = zkRecordConsumer.completeMigration(); - log.debug("Completed migrating metadata from Zookeeper. Current offset is {} and epoch is {}. Saw brokers {}", + log.info("Completed migrating metadata from Zookeeper. Current offset is {} and epoch is {}. Saw brokers {}", offsetAndEpochAfterMigration.offset(), offsetAndEpochAfterMigration.epoch(), brokersInMetadata); @@ -426,31 +441,52 @@ public void run() throws Exception { class MetadataChangeEvent implements EventQueue.Event { private final MetadataDelta delta; private final MetadataImage image; - MetadataChangeEvent(MetadataDelta delta, MetadataImage image) { + private final MetadataProvenance provenance; + private final boolean isSnapshot; + + MetadataChangeEvent(MetadataDelta delta, MetadataImage image, MetadataProvenance provenance, boolean isSnapshot) { this.delta = delta; this.image = image; + this.provenance = provenance; + this.isSnapshot = isSnapshot; } @Override public void run() throws Exception { - KRaftMigrationDriver.this.image = image; - KRaftMigrationDriver.this.delta = delta; + if (migrationState != MigrationState.DUAL_WRITE) { + log.trace("Received metadata change, but the controller is not in dual-write " + + "mode. Ignoring the change to be replicated to Zookeeper"); + } - switch (migrationState) { - case DUAL_WRITE: - if (image.highestOffsetAndEpoch().compareTo(migrationLeadershipState.offsetAndEpoch()) >= 0) { - apply("Write MetadataDelta to Zk", - state -> zkMigrationClient.writeMetadataDeltaToZookeeper(delta, image, state)); - // TODO: Unhappy path: Probably relinquish leadership and let new controller - // retry the write? - rpcClient.sendRPCsToBrokersFromMetadataDelta(delta, image, - migrationLeadershipState.kraftControllerEpoch()); - } - break; - default: - log.debug("Received metadata change, but the controller is not in dual-write " + - "mode. Ignoring the change to be replicated to Zookeeper"); - break; + if (image.highestOffsetAndEpoch().compareTo(migrationLeadershipState.offsetAndEpoch()) >= 0) { + if (delta.topicsDelta() != null) { + delta.topicsDelta().changedTopics().forEach((topicId, topicDelta) -> { + if (image.topics().getTopic(topicId) == null) { + apply("Create topic " + topicDelta.name(), migrationState -> + zkMigrationClient.createTopic( + topicDelta.name(), + topicId, + topicDelta.partitionChanges(), + migrationState)); + } else { + apply("Updating topic " + topicDelta.name(), migrationState -> + zkMigrationClient.updateTopicPartitions( + Collections.singletonMap(topicDelta.name(), topicDelta.partitionChanges()), + migrationState)); + } + }); + } + + KRaftMigrationDriver.this.image = image; + + apply("Write MetadataDelta to Zk", state -> zkMigrationClient.writeMetadataDeltaToZookeeper(delta, image, state)); + // TODO: Unhappy path: Probably relinquish leadership and let new controller + // retry the write? + rpcClient.sendRPCsToBrokersFromMetadataDelta(delta, image, + migrationLeadershipState.kraftControllerEpoch()); + } else { + String metadataType = isSnapshot ? "snapshot" : "delta"; + log.info("Ignoring {} {} which contains metadata that has already been written to ZK.", metadataType, provenance); } } } From c38c7df839936ff21e280230f3d3fdec6e688a35 Mon Sep 17 00:00:00 2001 From: David Arthur Date: Mon, 19 Dec 2022 20:20:02 -0500 Subject: [PATCH 09/30] Progress on end-to-end topic creation Still an issue with broker epochs in UMR from KRaft --- .../controller/ControllerChannelManager.scala | 58 ++++++++++--------- .../KRaftControllerToZkBrokersRpcClient.scala | 9 ++- .../scala/kafka/server/ControllerServer.scala | 7 ++- .../ControllerChannelManagerTest.scala | 2 +- .../org/apache/kafka/image/ClusterDelta.java | 9 +-- .../org/apache/kafka/image/TopicsDelta.java | 8 +++ .../migration/KRaftMigrationDriver.java | 58 ++++++++++--------- 7 files changed, 91 insertions(+), 60 deletions(-) diff --git a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala index ec3a9e9b4511f..202b7fbe5d127 100755 --- a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala +++ b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala @@ -35,6 +35,7 @@ import org.apache.kafka.common.requests._ import org.apache.kafka.common.security.JaasContext import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.utils.{LogContext, Time} +import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.server.common.MetadataVersion._ import java.net.SocketTimeoutException @@ -321,7 +322,8 @@ class ControllerBrokerRequestBatch( ) extends AbstractControllerBrokerRequestBatch( config, () => controllerContext, - stateChangeLogger + stateChangeLogger, + () => config.interBrokerProtocolVersion ) { def sendEvent(event: ControllerEvent): Unit = { @@ -373,6 +375,7 @@ abstract class ControllerBrokerRequestMetadata { abstract class AbstractControllerBrokerRequestBatch(config: KafkaConfig, metadataProvider: () => ControllerBrokerRequestMetadata, stateChangeLogger: StateChangeLogger, + metadataVersionProvider: () => MetadataVersion, kraftController: Boolean = false) extends Logging { val controllerId: Int = config.brokerId val leaderAndIsrRequestMap = mutable.Map.empty[Int, mutable.Map[TopicPartition, LeaderAndIsrPartitionState]] @@ -431,7 +434,7 @@ abstract class AbstractControllerBrokerRequestBatch(config: KafkaConfig, .setRemovingReplicas(replicaAssignment.removingReplicas.map(Integer.valueOf).asJava) .setIsNew(isNew || alreadyNew) - if (config.interBrokerProtocolVersion.isAtLeast(IBP_3_2_IV0)) { + if (metadataVersionProvider.apply().isAtLeast(IBP_3_2_IV0)) { partitionState.setLeaderRecoveryState(leaderAndIsr.leaderRecoveryState.value) } @@ -506,14 +509,15 @@ abstract class AbstractControllerBrokerRequestBatch(config: KafkaConfig, } private def sendLeaderAndIsrRequest(controllerEpoch: Int, stateChangeLog: StateChangeLogger): Unit = { + val metadataVersion = metadataVersionProvider.apply() val leaderAndIsrRequestVersion: Short = - if (config.interBrokerProtocolVersion.isAtLeast(IBP_3_4_IV0)) 7 - else if (config.interBrokerProtocolVersion.isAtLeast(IBP_3_2_IV0)) 6 - else if (config.interBrokerProtocolVersion.isAtLeast(IBP_2_8_IV1)) 5 - else if (config.interBrokerProtocolVersion.isAtLeast(IBP_2_4_IV1)) 4 - else if (config.interBrokerProtocolVersion.isAtLeast(IBP_2_4_IV0)) 3 - else if (config.interBrokerProtocolVersion.isAtLeast(IBP_2_2_IV0)) 2 - else if (config.interBrokerProtocolVersion.isAtLeast(IBP_1_0_IV0)) 1 + if (metadataVersion.isAtLeast(IBP_3_4_IV0)) 7 + else if (metadataVersion.isAtLeast(IBP_3_2_IV0)) 6 + else if (metadataVersion.isAtLeast(IBP_2_8_IV1)) 5 + else if (metadataVersion.isAtLeast(IBP_2_4_IV1)) 4 + else if (metadataVersion.isAtLeast(IBP_2_4_IV0)) 3 + else if (metadataVersion.isAtLeast(IBP_2_2_IV0)) 2 + else if (metadataVersion.isAtLeast(IBP_1_0_IV0)) 1 else 0 leaderAndIsrRequestMap.forKeyValue { (broker, leaderAndIsrPartitionStates) => @@ -543,7 +547,7 @@ abstract class AbstractControllerBrokerRequestBatch(config: KafkaConfig, .map(topic => (topic, metadataInstance.topicIds.getOrElse(topic, Uuid.ZERO_UUID))) .toMap val leaderAndIsrRequestBuilder = new LeaderAndIsrRequest.Builder(leaderAndIsrRequestVersion, controllerId, - controllerEpoch, brokerEpoch, leaderAndIsrPartitionStates.values.toBuffer.asJava, topicIds.asJava, leaders.asJava) + controllerEpoch, brokerEpoch, leaderAndIsrPartitionStates.values.toBuffer.asJava, topicIds.asJava, leaders.asJava, kraftController) sendRequest(broker, leaderAndIsrRequestBuilder, (r: AbstractResponse) => { val leaderAndIsrResponse = r.asInstanceOf[LeaderAndIsrResponse] handleLeaderAndIsrResponse(leaderAndIsrResponse, broker) @@ -560,15 +564,16 @@ abstract class AbstractControllerBrokerRequestBatch(config: KafkaConfig, s"for ${updateMetadataRequestPartitionInfoMap.size} partitions") val partitionStates = updateMetadataRequestPartitionInfoMap.values.toBuffer + val metadataVersion = metadataVersionProvider.apply() val updateMetadataRequestVersion: Short = - if (config.interBrokerProtocolVersion.isAtLeast(IBP_3_4_IV0)) 8 - else if (config.interBrokerProtocolVersion.isAtLeast(IBP_2_8_IV1)) 7 - else if (config.interBrokerProtocolVersion.isAtLeast(IBP_2_4_IV1)) 6 - else if (config.interBrokerProtocolVersion.isAtLeast(IBP_2_2_IV0)) 5 - else if (config.interBrokerProtocolVersion.isAtLeast(IBP_1_0_IV0)) 4 - else if (config.interBrokerProtocolVersion.isAtLeast(IBP_0_10_2_IV0)) 3 - else if (config.interBrokerProtocolVersion.isAtLeast(IBP_0_10_0_IV1)) 2 - else if (config.interBrokerProtocolVersion.isAtLeast(IBP_0_9_0)) 1 + if (metadataVersion.isAtLeast(IBP_3_4_IV0)) 8 + else if (metadataVersion.isAtLeast(IBP_2_8_IV1)) 7 + else if (metadataVersion.isAtLeast(IBP_2_4_IV1)) 6 + else if (metadataVersion.isAtLeast(IBP_2_2_IV0)) 5 + else if (metadataVersion.isAtLeast(IBP_1_0_IV0)) 4 + else if (metadataVersion.isAtLeast(IBP_0_10_2_IV0)) 3 + else if (metadataVersion.isAtLeast(IBP_0_10_0_IV1)) 2 + else if (metadataVersion.isAtLeast(IBP_0_9_0)) 1 else 0 val liveBrokers = metadataInstance.liveOrShuttingDownBrokers.iterator.map { broker => @@ -604,7 +609,7 @@ abstract class AbstractControllerBrokerRequestBatch(config: KafkaConfig, .filter(metadataInstance.topicIds.contains) .map(topic => (topic, metadataInstance.topicIds(topic))).toMap val updateMetadataRequestBuilder = new UpdateMetadataRequest.Builder(updateMetadataRequestVersion, - controllerId, controllerEpoch, brokerEpoch, partitionStates.asJava, liveBrokers.asJava, topicIds.asJava) + controllerId, controllerEpoch, brokerEpoch, partitionStates.asJava, liveBrokers.asJava, topicIds.asJava, kraftController) sendRequest(broker, updateMetadataRequestBuilder, (r: AbstractResponse) => { val updateMetadataResponse = r.asInstanceOf[UpdateMetadataResponse] handleUpdateMetadataResponse(updateMetadataResponse, broker) @@ -619,11 +624,12 @@ abstract class AbstractControllerBrokerRequestBatch(config: KafkaConfig, private def sendStopReplicaRequests(controllerEpoch: Int, stateChangeLog: StateChangeLogger): Unit = { val traceEnabled = stateChangeLog.isTraceEnabled + val metadataVersion = metadataVersionProvider.apply() val stopReplicaRequestVersion: Short = - if (config.interBrokerProtocolVersion.isAtLeast(IBP_3_4_IV0)) 4 - else if (config.interBrokerProtocolVersion.isAtLeast(IBP_2_6_IV0)) 3 - else if (config.interBrokerProtocolVersion.isAtLeast(IBP_2_4_IV1)) 2 - else if (config.interBrokerProtocolVersion.isAtLeast(IBP_2_2_IV0)) 1 + if (metadataVersion.isAtLeast(IBP_3_4_IV0)) 4 + else if (metadataVersion.isAtLeast(IBP_2_6_IV0)) 3 + else if (metadataVersion.isAtLeast(IBP_2_4_IV1)) 2 + else if (metadataVersion.isAtLeast(IBP_2_2_IV0)) 1 else 0 def responseCallback(brokerId: Int, isPartitionDeleted: TopicPartition => Boolean) @@ -662,7 +668,7 @@ abstract class AbstractControllerBrokerRequestBatch(config: KafkaConfig, s"replicas to broker $brokerId") val stopReplicaRequestBuilder = new StopReplicaRequest.Builder( stopReplicaRequestVersion, controllerId, controllerEpoch, brokerEpoch, - false, stopReplicaTopicState.values.toBuffer.asJava) + false, stopReplicaTopicState.values.toBuffer.asJava, kraftController) sendRequest(brokerId, stopReplicaRequestBuilder, responseCallback(brokerId, tp => partitionStates.get(tp).exists(_.deletePartition))) } else { @@ -689,7 +695,7 @@ abstract class AbstractControllerBrokerRequestBatch(config: KafkaConfig, s"$numPartitionStateWithDelete replicas to broker $brokerId") val stopReplicaRequestBuilder = new StopReplicaRequest.Builder( stopReplicaRequestVersion, controllerId, controllerEpoch, brokerEpoch, - true, topicStatesWithDelete.values.toBuffer.asJava) + true, topicStatesWithDelete.values.toBuffer.asJava, kraftController) sendRequest(brokerId, stopReplicaRequestBuilder, responseCallback(brokerId, _ => true)) } @@ -698,7 +704,7 @@ abstract class AbstractControllerBrokerRequestBatch(config: KafkaConfig, s"$numPartitionStateWithoutDelete replicas to broker $brokerId") val stopReplicaRequestBuilder = new StopReplicaRequest.Builder( stopReplicaRequestVersion, controllerId, controllerEpoch, brokerEpoch, - false, topicStatesWithoutDelete.values.toBuffer.asJava) + false, topicStatesWithoutDelete.values.toBuffer.asJava, kraftController) sendRequest(brokerId, stopReplicaRequestBuilder) } } diff --git a/core/src/main/scala/kafka/migration/KRaftControllerToZkBrokersRpcClient.scala b/core/src/main/scala/kafka/migration/KRaftControllerToZkBrokersRpcClient.scala index fe134795e2c51..89dd6b2efd3bd 100644 --- a/core/src/main/scala/kafka/migration/KRaftControllerToZkBrokersRpcClient.scala +++ b/core/src/main/scala/kafka/migration/KRaftControllerToZkBrokersRpcClient.scala @@ -11,6 +11,7 @@ import org.apache.kafka.common.requests._ import org.apache.kafka.common.utils.Time import org.apache.kafka.image.{MetadataDelta, MetadataImage} import org.apache.kafka.metadata.migration.BrokersRpcClient +import org.apache.kafka.server.common.MetadataVersion import java.util import scala.collection.mutable.ArrayBuffer @@ -114,7 +115,8 @@ sealed class KRaftControllerBrokerRequestBatch( config, metadataProvider, stateChangeLogger, - kraftController = true + () => MetadataVersion.IBP_3_4_IV0, // TODO pass this in + kraftController = true, ) { override def sendRequest(brokerId: Int, request: AbstractControlRequest.Builder[_ <: AbstractControlRequest], callback: AbstractResponse => Unit): Unit = { @@ -167,7 +169,7 @@ class KRaftControllerToZkBrokersRpcClient( config: KafkaConfig ) extends BrokersRpcClient { @volatile private var _image = MetadataImage.EMPTY - val stateChangeLogger = new StateChangeLogger(nodeId, inControllerContext = false, None) + val stateChangeLogger = new StateChangeLogger(nodeId, inControllerContext = true, None) val channelManager = new ControllerChannelManager( () => _image.highestOffsetAndEpoch().epoch(), config, @@ -214,6 +216,9 @@ class KRaftControllerToZkBrokersRpcClient( publishMetadata(image) requestBatch.newBatch() + delta.getOrCreateTopicsDelta() + delta.getOrCreateClusterDelta() + val newZkBrokers = delta.clusterDelta().newZkBrokers().asScala.map(_.toInt).toSet val zkBrokers = image.cluster().zkBrokers().keySet().asScala.map(_.toInt).toSet val oldZkBrokers = zkBrokers -- newZkBrokers diff --git a/core/src/main/scala/kafka/server/ControllerServer.scala b/core/src/main/scala/kafka/server/ControllerServer.scala index b737e8e0c8019..a21ffdb19ad37 100644 --- a/core/src/main/scala/kafka/server/ControllerServer.scala +++ b/core/src/main/scala/kafka/server/ControllerServer.scala @@ -248,7 +248,12 @@ class ControllerServer( controller.asInstanceOf[QuorumController].zkRecordConsumer(), migrationClient, rpcClient, - publisher => sharedServer.loader.installPublishers(java.util.Collections.singletonList(publisher)) + publisher => sharedServer.loader.installPublishers(java.util.Collections.singletonList(publisher)), + sharedServer.faultHandlerFactory.build( + "zk migration", + fatal = false, + () => {} + ) ) migrationDriver.start() migrationSupport = Some(ControllerMigrationSupport(zkClient, migrationDriver, rpcClient)) diff --git a/core/src/test/scala/unit/kafka/controller/ControllerChannelManagerTest.scala b/core/src/test/scala/unit/kafka/controller/ControllerChannelManagerTest.scala index c08b8fdc614f2..a7b3cd7a2219d 100644 --- a/core/src/test/scala/unit/kafka/controller/ControllerChannelManagerTest.scala +++ b/core/src/test/scala/unit/kafka/controller/ControllerChannelManagerTest.scala @@ -941,7 +941,7 @@ class ControllerChannelManagerTest { private case class SentRequest(request: ControlRequest, responseCallback: AbstractResponse => Unit) private class MockControllerBrokerRequestBatch(context: ControllerContext, config: KafkaConfig = config) - extends AbstractControllerBrokerRequestBatch(config, () => context, logger) { + extends AbstractControllerBrokerRequestBatch(config, () => context, logger, () => config.interBrokerProtocolVersion) { val sentEvents = ListBuffer.empty[ControllerEvent] val sentRequests = mutable.Map.empty[Int, ListBuffer[SentRequest]] diff --git a/metadata/src/main/java/org/apache/kafka/image/ClusterDelta.java b/metadata/src/main/java/org/apache/kafka/image/ClusterDelta.java index f0d0fceb1d63d..285655d1c47a8 100644 --- a/metadata/src/main/java/org/apache/kafka/image/ClusterDelta.java +++ b/metadata/src/main/java/org/apache/kafka/image/ClusterDelta.java @@ -60,11 +60,12 @@ public BrokerRegistration broker(int nodeId) { public Set newZkBrokers() { return changedBrokers - .entrySet() + .values() .stream() - .filter(entry -> !image.containsBroker(entry.getKey()) - && entry.getValue().filter(BrokerRegistration::isMigratingZkBroker).isPresent()) - .map(Entry::getKey) + .filter(Optional::isPresent) + .map(Optional::get) + .filter(registration -> registration.isMigratingZkBroker() && !registration.fenced()) + .map(BrokerRegistration::id) .collect(Collectors.toSet()); } diff --git a/metadata/src/main/java/org/apache/kafka/image/TopicsDelta.java b/metadata/src/main/java/org/apache/kafka/image/TopicsDelta.java index 66f12102a5935..d3c5888fa7a2b 100644 --- a/metadata/src/main/java/org/apache/kafka/image/TopicsDelta.java +++ b/metadata/src/main/java/org/apache/kafka/image/TopicsDelta.java @@ -53,6 +53,8 @@ public final class TopicsDelta { */ private final Set deletedTopicIds = new HashSet<>(); + private final Set createdTopicIds = new HashSet<>(); + public TopicsDelta(TopicsImage image) { this.image = image; } @@ -69,6 +71,7 @@ public void replay(TopicRecord record) { TopicDelta delta = new TopicDelta( new TopicImage(record.name(), record.topicId(), Collections.emptyMap())); changedTopics.put(record.topicId(), delta); + createdTopicIds.add(record.topicId()); } TopicDelta getOrCreateTopicDelta(Uuid id) { @@ -170,6 +173,10 @@ public Set deletedTopicIds() { return deletedTopicIds; } + public Set createdTopicIds() { + return createdTopicIds; + } + /** * Find the topic partitions that have change based on the replica given. * @@ -212,6 +219,7 @@ public String toString() { return "TopicsDelta(" + "changedTopics=" + changedTopics + ", deletedTopicIds=" + deletedTopicIds + + ", createdTopicIds=" + createdTopicIds + ')'; } } diff --git a/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java b/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java index 0c229e9a73b1f..f64a6fed53fbe 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java @@ -28,6 +28,8 @@ import org.apache.kafka.queue.KafkaEventQueue; import org.apache.kafka.raft.LeaderAndEpoch; import org.apache.kafka.raft.OffsetAndEpoch; +import org.apache.kafka.server.fault.FaultHandler; +import org.apache.kafka.server.fault.LoggingFaultHandler; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -54,6 +56,7 @@ public class KRaftMigrationDriver implements MetadataPublisher { private final BrokersRpcClient rpcClient; private final ZkRecordConsumer zkRecordConsumer; private final KafkaEventQueue eventQueue; + private final FaultHandler faultHandler; /** * A callback for when the migration state has been recovered from ZK. This is used to delay the installation of this * MetadataPublisher with MetadataLoader. @@ -69,7 +72,8 @@ public KRaftMigrationDriver( ZkRecordConsumer zkRecordConsumer, MigrationClient zkMigrationClient, BrokersRpcClient rpcClient, - Consumer initialZkLoadHandler + Consumer initialZkLoadHandler, + FaultHandler faultHandler ) { this.nodeId = nodeId; this.zkRecordConsumer = zkRecordConsumer; @@ -83,6 +87,7 @@ public KRaftMigrationDriver( this.image = MetadataImage.EMPTY; this.leaderAndEpoch = LeaderAndEpoch.UNKNOWN; this.initialZkLoadHandler = initialZkLoadHandler; + this.faultHandler = faultHandler; } public void start() { @@ -215,8 +220,14 @@ public void close() throws Exception { } // Events handled by Migration Driver. + abstract class MigrationEvent implements EventQueue.Event { + @Override + public void handleException(Throwable e) { + KRaftMigrationDriver.this.faultHandler.handleFault("Error during ZK migration", e); + } + } - class PollEvent implements EventQueue.Event { + class PollEvent extends MigrationEvent { @Override public void run() throws Exception { switch (migrationState) { @@ -262,7 +273,7 @@ public void handleException(Throwable e) { } } - class KRaftLeaderEvent implements EventQueue.Event { + class KRaftLeaderEvent extends MigrationEvent { private final LeaderAndEpoch leaderAndEpoch; KRaftLeaderEvent(LeaderAndEpoch leaderAndEpoch) { @@ -304,7 +315,7 @@ public void handleException(Throwable e) { } } - class WaitForControllerQuorumEvent implements EventQueue.Event { + class WaitForControllerQuorumEvent extends MigrationEvent { @Override public void run() throws Exception { @@ -329,7 +340,7 @@ public void handleException(Throwable e) { } } - class BecomeZkControllerEvent implements EventQueue.Event { + class BecomeZkControllerEvent extends MigrationEvent { @Override public void run() throws Exception { switch (migrationState) { @@ -358,7 +369,7 @@ public void handleException(Throwable e) { } } - class WaitForZkBrokersEvent implements EventQueue.Event { + class WaitForZkBrokersEvent extends MigrationEvent { @Override public void run() throws Exception { switch (migrationState) { @@ -380,7 +391,7 @@ public void handleException(Throwable e) { } } - class MigrateMetadataEvent implements EventQueue.Event { + class MigrateMetadataEvent extends MigrationEvent { @Override public void run() throws Exception { Set brokersInMetadata = new HashSet<>(); @@ -420,29 +431,22 @@ public void handleException(Throwable e) { } } - class SendRPCsToBrokersEvent implements EventQueue.Event { + class SendRPCsToBrokersEvent extends MigrationEvent { @Override public void run() throws Exception { - switch (migrationState) { - case KRAFT_CONTROLLER_TO_BROKER_COMM: - if (image.highestOffsetAndEpoch().compareTo(migrationLeadershipState.offsetAndEpoch()) >= 0) { - // TODO do this when we have the real client - //rpcClient.sendRPCsToBrokersFromMetadataImage(image, - // migrationLeadershipState.kraftControllerEpoch()); - // Migration leadership state doesn't change since we're not doing any Zk - // writes. - transitionTo(MigrationState.DUAL_WRITE); - } - break; - default: - // Ignore sending RPCs to the brokers since we're no longer in the state. - break; + // Ignore sending RPCs to the brokers since we're no longer in the state. + if (migrationState == MigrationState.KRAFT_CONTROLLER_TO_BROKER_COMM) { + if (image.highestOffsetAndEpoch().compareTo(migrationLeadershipState.offsetAndEpoch()) >= 0) { + rpcClient.sendRPCsToBrokersFromMetadataImage(image, migrationLeadershipState.kraftControllerEpoch()); + // Migration leadership state doesn't change since we're not doing any Zk writes. + transitionTo(MigrationState.DUAL_WRITE); + } } } } - class MetadataChangeEvent implements EventQueue.Event { + class MetadataChangeEvent extends MigrationEvent { private final MetadataDelta delta; private final MetadataImage image; private final MetadataProvenance provenance; @@ -457,15 +461,18 @@ class MetadataChangeEvent implements EventQueue.Event { @Override public void run() throws Exception { + KRaftMigrationDriver.this.image = image; + if (migrationState != MigrationState.DUAL_WRITE) { log.trace("Received metadata change, but the controller is not in dual-write " + - "mode. Ignoring the change to be replicated to Zookeeper"); + "mode. Ignoring the change to be replicated to Zookeeper"); + return; } if (image.highestOffsetAndEpoch().compareTo(migrationLeadershipState.offsetAndEpoch()) >= 0) { if (delta.topicsDelta() != null) { delta.topicsDelta().changedTopics().forEach((topicId, topicDelta) -> { - if (image.topics().getTopic(topicId) == null) { + if (delta.topicsDelta().createdTopicIds().contains(topicId)) { apply("Create topic " + topicDelta.name(), migrationState -> zkMigrationClient.createTopic( topicDelta.name(), @@ -481,7 +488,6 @@ public void run() throws Exception { }); } - KRaftMigrationDriver.this.image = image; apply("Write MetadataDelta to Zk", state -> zkMigrationClient.writeMetadataDeltaToZookeeper(delta, image, state)); // TODO: Unhappy path: Probably relinquish leadership and let new controller From d4226dc4733f37ee85ae4e4ca64ecc9f95c7ea81 Mon Sep 17 00:00:00 2001 From: David Arthur Date: Tue, 20 Dec 2022 11:18:04 -0500 Subject: [PATCH 10/30] Send ZK broker epoch in registration --- .../requests/BrokerRegistrationRequest.java | 2 +- .../message/BrokerRegistrationRequest.json | 2 +- .../KRaftControllerToZkBrokersRpcClient.scala | 2 +- .../kafka/server/BrokerLifecycleManager.scala | 6 +-- .../main/scala/kafka/server/KafkaServer.scala | 10 ++--- .../BrokerRegistrationRequestTest.scala | 28 ++++++------- .../controller/ClusterControlManager.java | 7 ++-- .../kafka/metadata/BrokerRegistration.java | 42 ++++++++++++------- .../common/metadata/RegisterBrokerRecord.json | 2 +- 9 files changed, 57 insertions(+), 44 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/requests/BrokerRegistrationRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/BrokerRegistrationRequest.java index 18d6a070d0557..9c6e57c6bb0a6 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/BrokerRegistrationRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/BrokerRegistrationRequest.java @@ -36,7 +36,7 @@ public Builder(BrokerRegistrationRequestData data) { @Override public short oldestAllowedVersion() { - if (data.isMigratingZkBroker()) { + if (data.migratingZkBrokerEpoch() != -1) { return (short) 1; } else { return (short) 0; diff --git a/clients/src/main/resources/common/message/BrokerRegistrationRequest.json b/clients/src/main/resources/common/message/BrokerRegistrationRequest.json index 19ad8f249b3ae..39640f0724279 100644 --- a/clients/src/main/resources/common/message/BrokerRegistrationRequest.json +++ b/clients/src/main/resources/common/message/BrokerRegistrationRequest.json @@ -51,7 +51,7 @@ }, { "name": "Rack", "type": "string", "versions": "0+", "nullableVersions": "0+", "about": "The rack which this broker is in." }, - { "name": "IsMigratingZkBroker", "type": "bool", "versions": "1+", "default": "false", + { "name": "MigratingZkBrokerEpoch", "type": "int64", "versions": "1+", "default": "-1", "about": "Set by a ZK broker if the required configurations for ZK migration are present." } ] } diff --git a/core/src/main/scala/kafka/migration/KRaftControllerToZkBrokersRpcClient.scala b/core/src/main/scala/kafka/migration/KRaftControllerToZkBrokersRpcClient.scala index 89dd6b2efd3bd..9ec56cc57c4ac 100644 --- a/core/src/main/scala/kafka/migration/KRaftControllerToZkBrokersRpcClient.scala +++ b/core/src/main/scala/kafka/migration/KRaftControllerToZkBrokersRpcClient.scala @@ -61,7 +61,7 @@ sealed class KRaftControllerBrokerRequestMetadata(val image: MetadataImage) exte override val liveBrokerIdAndEpochs: collection.Map[Int, Long] = { image.cluster().zkBrokers().asScala.map { - case (brokerId, broker) => brokerId.intValue() -> broker.epoch() + case (brokerId, broker) => brokerId.intValue() -> broker.migratingZkBrokerEpoch().orElse(-1L) } } diff --git a/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala b/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala index 9f361275d4ef8..1c4c7c377104f 100644 --- a/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala +++ b/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala @@ -55,7 +55,7 @@ class BrokerLifecycleManager( val config: KafkaConfig, val time: Time, val threadNamePrefix: Option[String], - val isZkBroker: Boolean = false + val zkBrokerEpoch: Option[Long] = None ) extends Logging { val logContext = new LogContext(s"[BrokerLifecycleManager id=${config.nodeId}] ") @@ -270,7 +270,7 @@ class BrokerLifecycleManager( _clusterId = clusterId _advertisedListeners = advertisedListeners.duplicate() _supportedFeatures = new util.HashMap[String, VersionRange](supportedFeatures) - if (!isZkBroker) { + if (zkBrokerEpoch.isDefined) { // ZK brokers don't block on registration during startup eventQueue.scheduleDeferred("initialRegistrationTimeout", new DeadlineFunction(time.nanoseconds() + initialTimeoutNs), @@ -292,7 +292,7 @@ class BrokerLifecycleManager( } val data = new BrokerRegistrationRequestData(). setBrokerId(nodeId). - setIsMigratingZkBroker(isZkBroker). + setMigratingZkBrokerEpoch(zkBrokerEpoch.getOrElse(-1)). setClusterId(_clusterId). setFeatures(features). setIncarnationId(incarnationId). diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index f8e449d8d1e00..0b904d768d914 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -209,11 +209,6 @@ class KafkaServer( if (canStartup) { _brokerState = BrokerState.STARTING - lifecycleManager = new BrokerLifecycleManager(config, - time, - threadNamePrefix, - isZkBroker = true) - /* setup zookeeper */ initZkClient(time) configRepository = new ZkConfigRepository(new AdminZkClient(zkClient)) @@ -359,6 +354,11 @@ class KafkaServer( val brokerInfo = createBrokerInfo val brokerEpoch = zkClient.registerBroker(brokerInfo) + lifecycleManager = new BrokerLifecycleManager(config, + time, + threadNamePrefix, + zkBrokerEpoch = Some(brokerEpoch)) + // Now that the broker is successfully registered, checkpoint its metadata val zkMetaProperties = ZkMetaProperties(clusterId, config.brokerId) checkpointBrokerMetadata(zkMetaProperties) diff --git a/core/src/test/scala/unit/kafka/server/BrokerRegistrationRequestTest.scala b/core/src/test/scala/unit/kafka/server/BrokerRegistrationRequestTest.scala index cab586f23e40c..d7fbf644c8360 100644 --- a/core/src/test/scala/unit/kafka/server/BrokerRegistrationRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/BrokerRegistrationRequestTest.scala @@ -92,7 +92,7 @@ class BrokerRegistrationRequestTest { channelManager: BrokerToControllerChannelManager, clusterId: String, brokerId: Int, - zk: Boolean, + zkEpoch: Option[Long], ibpToSend: Option[(MetadataVersion, MetadataVersion)] ): Errors = { val features = new BrokerRegistrationRequestData.FeatureCollection() @@ -110,7 +110,7 @@ class BrokerRegistrationRequestTest { .setBrokerId(brokerId) .setClusterId(clusterId) .setIncarnationId(Uuid.randomUuid()) - .setIsMigratingZkBroker(zk) + .setMigratingZkBrokerEpoch(zkEpoch.getOrElse(-1L)) .setFeatures(features) Errors.forCode(sendAndRecieve(channelManager, req).errorCode()) @@ -126,19 +126,19 @@ class BrokerRegistrationRequestTest { assertEquals( Errors.BROKER_ID_NOT_REGISTERED, - registerBroker(channelManager, clusterId, 100, true, Some((MetadataVersion.IBP_3_3_IV0, MetadataVersion.IBP_3_3_IV0)))) + registerBroker(channelManager, clusterId, 100, Some(1), Some((MetadataVersion.IBP_3_3_IV0, MetadataVersion.IBP_3_3_IV0)))) assertEquals( Errors.BROKER_ID_NOT_REGISTERED, - registerBroker(channelManager, clusterId, 100, true, None)) + registerBroker(channelManager, clusterId, 100, Some(1), None)) assertEquals( Errors.BROKER_ID_NOT_REGISTERED, - registerBroker(channelManager, clusterId, 100, true, Some((MetadataVersion.IBP_3_4_IV0, MetadataVersion.IBP_3_4_IV0)))) + registerBroker(channelManager, clusterId, 100, Some(1), Some((MetadataVersion.IBP_3_4_IV0, MetadataVersion.IBP_3_4_IV0)))) assertEquals( Errors.NONE, - registerBroker(channelManager, clusterId, 100, false, Some((MetadataVersion.IBP_3_4_IV0, MetadataVersion.IBP_3_4_IV0)))) + registerBroker(channelManager, clusterId, 100, None, Some((MetadataVersion.IBP_3_4_IV0, MetadataVersion.IBP_3_4_IV0)))) } finally { channelManager.shutdown() } @@ -154,19 +154,19 @@ class BrokerRegistrationRequestTest { assertEquals( Errors.BROKER_ID_NOT_REGISTERED, - registerBroker(channelManager, clusterId, 100, true, Some((MetadataVersion.IBP_3_3_IV0, MetadataVersion.IBP_3_3_IV0)))) + registerBroker(channelManager, clusterId, 100, Some(1), Some((MetadataVersion.IBP_3_3_IV0, MetadataVersion.IBP_3_3_IV0)))) assertEquals( Errors.BROKER_ID_NOT_REGISTERED, - registerBroker(channelManager, clusterId, 100, true, None)) + registerBroker(channelManager, clusterId, 100, Some(1), None)) assertEquals( Errors.BROKER_ID_NOT_REGISTERED, - registerBroker(channelManager, clusterId, 100, true, Some((MetadataVersion.IBP_3_4_IV0, MetadataVersion.IBP_3_4_IV0)))) + registerBroker(channelManager, clusterId, 100, Some(1), Some((MetadataVersion.IBP_3_4_IV0, MetadataVersion.IBP_3_4_IV0)))) assertEquals( Errors.NONE, - registerBroker(channelManager, clusterId, 100, false, Some((MetadataVersion.IBP_3_3_IV3, MetadataVersion.IBP_3_4_IV0)))) + registerBroker(channelManager, clusterId, 100, None, Some((MetadataVersion.IBP_3_3_IV3, MetadataVersion.IBP_3_4_IV0)))) } finally { channelManager.shutdown() } @@ -182,19 +182,19 @@ class BrokerRegistrationRequestTest { assertEquals( Errors.NONE, - registerBroker(channelManager, clusterId, 100, true, Some((MetadataVersion.IBP_3_4_IV0, MetadataVersion.IBP_3_4_IV0)))) + registerBroker(channelManager, clusterId, 100, Some(1), Some((MetadataVersion.IBP_3_4_IV0, MetadataVersion.IBP_3_4_IV0)))) assertEquals( Errors.UNSUPPORTED_VERSION, - registerBroker(channelManager, clusterId, 100, true, None)) + registerBroker(channelManager, clusterId, 100, Some(1), None)) assertEquals( Errors.UNSUPPORTED_VERSION, - registerBroker(channelManager, clusterId, 100, true, Some((MetadataVersion.IBP_3_3_IV3, MetadataVersion.IBP_3_3_IV3)))) + registerBroker(channelManager, clusterId, 100, Some(1), Some((MetadataVersion.IBP_3_3_IV3, MetadataVersion.IBP_3_3_IV3)))) assertEquals( Errors.NONE, - registerBroker(channelManager, clusterId, 100, false, Some((MetadataVersion.IBP_3_4_IV0, MetadataVersion.IBP_3_4_IV0)))) + registerBroker(channelManager, clusterId, 100, None, Some((MetadataVersion.IBP_3_4_IV0, MetadataVersion.IBP_3_4_IV0)))) } finally { channelManager.shutdown() } diff --git a/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java index ba6c0e1d1c215..0f63277e04ee1 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java @@ -353,13 +353,13 @@ public ControllerResult registerBroker( } } - if (request.isMigratingZkBroker() && !zkRegistrationAllowed()) { + if (request.migratingZkBrokerEpoch() != -1 && !zkRegistrationAllowed()) { throw new BrokerIdNotRegisteredException("Controller does not support registering ZK brokers."); } RegisterBrokerRecord record = new RegisterBrokerRecord(). setBrokerId(brokerId). - setIsMigratingZkBroker(request.isMigratingZkBroker()). + setMigratingZkBrokerEpoch(request.migratingZkBrokerEpoch()). setIncarnationId(request.incarnationId()). setBrokerEpoch(brokerEpoch). setRack(request.rack()); @@ -436,12 +436,13 @@ public void replay(RegisterBrokerRecord record, long offset) { feature.minSupportedVersion(), feature.maxSupportedVersion())); } + // Update broker registrations. BrokerRegistration prevRegistration = brokerRegistrations.put(brokerId, new BrokerRegistration(brokerId, record.brokerEpoch(), record.incarnationId(), listeners, features, Optional.ofNullable(record.rack()), record.fenced(), - record.inControlledShutdown(), record.isMigratingZkBroker())); + record.inControlledShutdown(), BrokerRegistration.zkBrokerEpoch(record.migratingZkBrokerEpoch()))); updateMetrics(prevRegistration, brokerRegistrations.get(brokerId)); if (heartbeatManager != null) { if (prevRegistration != null) heartbeatManager.remove(brokerId); diff --git a/metadata/src/main/java/org/apache/kafka/metadata/BrokerRegistration.java b/metadata/src/main/java/org/apache/kafka/metadata/BrokerRegistration.java index 6aa40195070a6..b365ae81266d7 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/BrokerRegistration.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/BrokerRegistration.java @@ -49,6 +49,14 @@ private static Map listenersToMap(Collection listene return listenersMap; } + public static Optional zkBrokerEpoch(long value) { + if (value == -1) { + return Optional.empty(); + } else { + return Optional.of(value); + } + } + private final int id; private final long epoch; private final Uuid incarnationId; @@ -57,7 +65,7 @@ private static Map listenersToMap(Collection listene private final Optional rack; private final boolean fenced; private final boolean inControlledShutdown; - private final boolean isMigratingZkBroker; + private final Optional migratingZkBrokerEpoch; // Visible for testing public BrokerRegistration(int id, @@ -69,7 +77,7 @@ public BrokerRegistration(int id, boolean fenced, boolean inControlledShutdown) { this(id, epoch, incarnationId, listenersToMap(listeners), supportedFeatures, rack, - fenced, inControlledShutdown, false); + fenced, inControlledShutdown, Optional.empty()); } public BrokerRegistration(int id, @@ -80,9 +88,9 @@ public BrokerRegistration(int id, Optional rack, boolean fenced, boolean inControlledShutdown, - boolean isMigratingZkBroker) { + Optional migratingZkBrokerEpoch) { this(id, epoch, incarnationId, listenersToMap(listeners), supportedFeatures, rack, - fenced, inControlledShutdown, isMigratingZkBroker); + fenced, inControlledShutdown, migratingZkBrokerEpoch); } // Visible for testing @@ -94,7 +102,7 @@ public BrokerRegistration(int id, Optional rack, boolean fenced, boolean inControlledShutdown) { - this(id, epoch, incarnationId, listeners, supportedFeatures, rack, fenced, inControlledShutdown, false); + this(id, epoch, incarnationId, listeners, supportedFeatures, rack, fenced, inControlledShutdown, Optional.empty()); } public BrokerRegistration(int id, @@ -105,7 +113,7 @@ public BrokerRegistration(int id, Optional rack, boolean fenced, boolean inControlledShutdown, - boolean isMigratingZkBroker) { + Optional migratingZkBrokerEpoch) { this.id = id; this.epoch = epoch; this.incarnationId = incarnationId; @@ -123,7 +131,7 @@ public BrokerRegistration(int id, this.rack = rack; this.fenced = fenced; this.inControlledShutdown = inControlledShutdown; - this.isMigratingZkBroker = isMigratingZkBroker; + this.migratingZkBrokerEpoch = migratingZkBrokerEpoch; } public static BrokerRegistration fromRecord(RegisterBrokerRecord record) { @@ -147,7 +155,7 @@ public static BrokerRegistration fromRecord(RegisterBrokerRecord record) { Optional.ofNullable(record.rack()), record.fenced(), record.inControlledShutdown(), - record.isMigratingZkBroker()); + zkBrokerEpoch(record.migratingZkBrokerEpoch())); } public int id() { @@ -191,7 +199,11 @@ public boolean inControlledShutdown() { } public boolean isMigratingZkBroker() { - return isMigratingZkBroker; + return migratingZkBrokerEpoch.isPresent(); + } + + public Optional migratingZkBrokerEpoch() { + return migratingZkBrokerEpoch; } public ApiMessageAndVersion toRecord(ImageWriterOptions options) { @@ -210,9 +222,9 @@ public ApiMessageAndVersion toRecord(ImageWriterOptions options) { } } - if (isMigratingZkBroker) { + if (migratingZkBrokerEpoch.isPresent()) { if (options.metadataVersion().isMigrationSupported()) { - registrationRecord.setIsMigratingZkBroker(isMigratingZkBroker); + registrationRecord.setMigratingZkBrokerEpoch(migratingZkBrokerEpoch.get()); } else { options.handleLoss("the isMigratingZkBroker state of one or more brokers"); } @@ -241,7 +253,7 @@ public ApiMessageAndVersion toRecord(ImageWriterOptions options) { @Override public int hashCode() { return Objects.hash(id, epoch, incarnationId, listeners, supportedFeatures, - rack, fenced, inControlledShutdown, isMigratingZkBroker); + rack, fenced, inControlledShutdown, migratingZkBrokerEpoch); } @Override @@ -256,7 +268,7 @@ public boolean equals(Object o) { other.rack.equals(rack) && other.fenced == fenced && other.inControlledShutdown == inControlledShutdown && - other.isMigratingZkBroker == isMigratingZkBroker; + other.migratingZkBrokerEpoch == migratingZkBrokerEpoch; } @Override @@ -277,7 +289,7 @@ public String toString() { bld.append(", rack=").append(rack); bld.append(", fenced=").append(fenced); bld.append(", inControlledShutdown=").append(inControlledShutdown); - bld.append(", isMigratingZkBroker=").append(isMigratingZkBroker); + bld.append(", migratingZkBrokerEpoch=").append(migratingZkBrokerEpoch.orElse(-1L)); bld.append(")"); return bld.toString(); } @@ -301,7 +313,7 @@ public BrokerRegistration cloneWith( rack, newFenced, newInControlledShutdownChange, - isMigratingZkBroker + migratingZkBrokerEpoch ); } } diff --git a/metadata/src/main/resources/common/metadata/RegisterBrokerRecord.json b/metadata/src/main/resources/common/metadata/RegisterBrokerRecord.json index 80657abb5feb4..18dd6e06dd879 100644 --- a/metadata/src/main/resources/common/metadata/RegisterBrokerRecord.json +++ b/metadata/src/main/resources/common/metadata/RegisterBrokerRecord.json @@ -22,7 +22,7 @@ "fields": [ { "name": "BrokerId", "type": "int32", "versions": "0+", "entityType": "brokerId", "about": "The broker id." }, - { "name": "IsMigratingZkBroker", "type": "bool", "versions": "2+", "default": false, + { "name": "MigratingZkBrokerEpoch", "type": "int64", "versions": "2+", "default": "-1", "about": "True if the registering broker is a ZK broker." }, { "name": "IncarnationId", "type": "uuid", "versions": "0+", "about": "The incarnation ID of the broker process" }, From 69e87f91de7f00ffd1a44975c461fde817ac22aa Mon Sep 17 00:00:00 2001 From: David Arthur Date: Tue, 20 Dec 2022 13:28:27 -0500 Subject: [PATCH 11/30] track ZK controller epoch and version --- .../main/scala/kafka/zk/KafkaZkClient.scala | 4 +- core/src/main/scala/kafka/zk/ZkData.scala | 3 +- .../scala/kafka/zk/ZkMigrationClient.scala | 11 +++-- .../unit/kafka/zk/KafkaZkClientTest.scala | 8 ++-- .../unit/kafka/zk/ZkMigrationClientTest.scala | 18 +++---- .../migration/KRaftMigrationDriver.java | 6 +-- .../migration/ZkMigrationLeadershipState.java | 48 +++++++++++++------ .../tests/core/zookeeper_migration_test.py | 0 8 files changed, 61 insertions(+), 37 deletions(-) create mode 100644 tests/kafkatest/tests/core/zookeeper_migration_test.py diff --git a/core/src/main/scala/kafka/zk/KafkaZkClient.scala b/core/src/main/scala/kafka/zk/KafkaZkClient.scala index 361c74c5a7c73..c61bace77aa8d 100644 --- a/core/src/main/scala/kafka/zk/KafkaZkClient.scala +++ b/core/src/main/scala/kafka/zk/KafkaZkClient.scala @@ -1952,7 +1952,7 @@ class KafkaZkClient private[zk] (zooKeeperClient: ZooKeeperClient, isSecure: Boo def wrapMigrationRequest(request: Req, lastRequestInBatch: Boolean): MultiRequest = { // Wrap a single request with the multi-op transactional request. - val checkOp = CheckOp(ControllerEpochZNode.path, migrationState.controllerZkVersion()) + val checkOp = CheckOp(ControllerEpochZNode.path, migrationState.zkControllerEpochZkVersion()) val migrationOp = if (lastRequestInBatch) { SetDataOp(MigrationZNode.path, MigrationZNode.encode(migrationState), migrationState.migrationZkVersion()) } else { @@ -2037,7 +2037,7 @@ class KafkaZkClient private[zk] (zooKeeperClient: ZooKeeperClient, isSecure: Boo } } - migrationState.controllerZkVersion() match { + migrationState.zkControllerEpochZkVersion() match { case ZkVersion.MatchAnyVersion => throw new IllegalArgumentException( s"Expected a controller epoch zkVersion when making migration writes, not -1.") case version if version >= 0 => diff --git a/core/src/main/scala/kafka/zk/ZkData.scala b/core/src/main/scala/kafka/zk/ZkData.scala index b0337b90062de..cc48c46ef72cb 100644 --- a/core/src/main/scala/kafka/zk/ZkData.scala +++ b/core/src/main/scala/kafka/zk/ZkData.scala @@ -1068,7 +1068,8 @@ object MigrationZNode { metadataEpoch, modifyTimeMs, zkVersion, - ZkVersion.UnknownVersion)) + ZkMigrationLeadershipState.EMPTY.zkControllerEpoch(), + ZkMigrationLeadershipState.EMPTY.zkControllerEpochZkVersion())) }.getOrElse(throw new KafkaException(s"Failed to parse the migration json $jsonDataAsString")) } } diff --git a/core/src/main/scala/kafka/zk/ZkMigrationClient.scala b/core/src/main/scala/kafka/zk/ZkMigrationClient.scala index c3c0d832149e5..1ab05d5a75dc6 100644 --- a/core/src/main/scala/kafka/zk/ZkMigrationClient.scala +++ b/core/src/main/scala/kafka/zk/ZkMigrationClient.scala @@ -58,19 +58,20 @@ class ZkMigrationClient(zkClient: KafkaZkClient) extends MigrationClient with Lo override def claimControllerLeadership(state: ZkMigrationLeadershipState): ZkMigrationLeadershipState = { zkClient.tryRegisterKRaftControllerAsActiveController(state.kraftControllerId(), state.kraftControllerEpoch()) match { - case SuccessfulRegistrationResult(_, controllerEpochZkVersion) => state.withControllerZkVersion(controllerEpochZkVersion) - case FailedRegistrationResult() => state.withControllerZkVersion(-1) + case SuccessfulRegistrationResult(controllerEpoch, controllerEpochZkVersion) => + state.withZkController(controllerEpoch, controllerEpochZkVersion) + case FailedRegistrationResult() => state.withUnknownZkController() } } override def releaseControllerLeadership(state: ZkMigrationLeadershipState): ZkMigrationLeadershipState = { try { - zkClient.deleteController(state.controllerZkVersion()) - state.withControllerZkVersion(-1) + zkClient.deleteController(state.zkControllerEpochZkVersion()) + state.withUnknownZkController() } catch { case _: ControllerMovedException => // If the controller moved, no need to release - state.withControllerZkVersion(-1) + state.withUnknownZkController() case t: Throwable => throw new RuntimeException("Could not release controller leadership due to underlying error", t) } diff --git a/core/src/test/scala/unit/kafka/zk/KafkaZkClientTest.scala b/core/src/test/scala/unit/kafka/zk/KafkaZkClientTest.scala index 416abd23eb23b..62b9cccbcaeba 100644 --- a/core/src/test/scala/unit/kafka/zk/KafkaZkClientTest.scala +++ b/core/src/test/scala/unit/kafka/zk/KafkaZkClientTest.scala @@ -1441,8 +1441,8 @@ class KafkaZkClientTest extends QuorumTestHarness { @Test def testFailToUpdateMigrationZNode(): Unit = { - val (_, stat) = zkClient.getControllerEpoch.get - var migrationState = new ZkMigrationLeadershipState(3000, 42, 100, 42, Time.SYSTEM.milliseconds(), -1, stat.getVersion) + val (controllerEpoch, stat) = zkClient.getControllerEpoch.get + var migrationState = new ZkMigrationLeadershipState(3000, 42, 100, 42, Time.SYSTEM.milliseconds(), -1, controllerEpoch, stat.getVersion) migrationState = zkClient.getOrCreateMigrationState(migrationState) assertEquals(0, migrationState.migrationZkVersion()) @@ -1454,7 +1454,7 @@ class KafkaZkClientTest extends QuorumTestHarness { CreateRequest("/foo", Array(), zkClient.defaultAcls("/foo"), CreateMode.PERSISTENT), ) - migrationState = migrationState.withControllerZkVersion(stat.getVersion) + migrationState = migrationState.withZkController(controllerEpoch, stat.getVersion) zkClient.retryMigrationRequestsUntilConnected(requests_bad, migrationState) match { case (zkVersion: Int, requests: Seq[AsyncRequest#Response]) => assertEquals(0, zkVersion) @@ -1476,7 +1476,7 @@ class KafkaZkClientTest extends QuorumTestHarness { CreateRequest("/foo/bar/eggs", Array(), zkClient.defaultAcls("/foo"), CreateMode.PERSISTENT), ) - migrationState = migrationState.withControllerZkVersion(stat.getVersion) + migrationState = migrationState.withZkController(controllerEpoch, stat.getVersion) zkClient.retryMigrationRequestsUntilConnected(requests_good, migrationState) match { case (zkVersion: Int, requests: Seq[AsyncRequest#Response]) => assertEquals(1, zkVersion) diff --git a/core/src/test/scala/unit/kafka/zk/ZkMigrationClientTest.scala b/core/src/test/scala/unit/kafka/zk/ZkMigrationClientTest.scala index 4c77eb5b76316..b6d017877852f 100644 --- a/core/src/test/scala/unit/kafka/zk/ZkMigrationClientTest.scala +++ b/core/src/test/scala/unit/kafka/zk/ZkMigrationClientTest.scala @@ -60,8 +60,8 @@ class ZkMigrationClientTest extends QuorumTestHarness { } private def initialMigrationState: ZkMigrationLeadershipState = { - val (_, stat) = zkClient.getControllerEpoch.get - new ZkMigrationLeadershipState(3000, InitialControllerEpoch, 100, InitialKRaftEpoch, Time.SYSTEM.milliseconds(), -1, stat.getVersion) + val (epoch, stat) = zkClient.getControllerEpoch.get + new ZkMigrationLeadershipState(3000, InitialControllerEpoch, 100, InitialKRaftEpoch, Time.SYSTEM.milliseconds(), -1, epoch, stat.getVersion) } @Test @@ -233,20 +233,22 @@ class ZkMigrationClientTest extends QuorumTestHarness { def testClaimAbsentController(): Unit = { assertEquals(0, migrationState.migrationZkVersion()) migrationState = migrationClient.claimControllerLeadership(migrationState) - assertEquals(1, migrationState.controllerZkVersion()) + assertEquals(1, migrationState.zkControllerEpochZkVersion()) } @Test def testExistingKRaftControllerClaim(): Unit = { assertEquals(0, migrationState.migrationZkVersion()) migrationState = migrationClient.claimControllerLeadership(migrationState) - assertEquals(1, migrationState.controllerZkVersion()) + assertEquals(1, migrationState.zkControllerEpochZkVersion()) // We don't require a KRaft controller to release the controller in ZK before another KRaft controller // can claim it. This is because KRaft leadership comes from Raft and we are just synchronizing it to ZK. - var otherNodeState = new ZkMigrationLeadershipState(3001, 43, 100, 42, Time.SYSTEM.milliseconds(), -1, -1) + var otherNodeState = ZkMigrationLeadershipState.EMPTY + .withNewKRaftController(3001, 43) + .withKRaftMetadataOffsetAndEpoch(100, 42); otherNodeState = migrationClient.claimControllerLeadership(otherNodeState) - assertEquals(2, otherNodeState.controllerZkVersion()) + assertEquals(2, otherNodeState.zkControllerEpochZkVersion()) assertEquals(3001, otherNodeState.kraftControllerId()) assertEquals(43, otherNodeState.kraftControllerEpoch()) } @@ -257,7 +259,7 @@ class ZkMigrationClientTest extends QuorumTestHarness { migrationState = migrationState.withNewKRaftController(3001, InitialControllerEpoch) migrationState = migrationClient.claimControllerLeadership(migrationState) - assertEquals(1, migrationState.controllerZkVersion()) + assertEquals(1, migrationState.zkControllerEpochZkVersion()) migrationState = migrationState.withNewKRaftController(3001, InitialControllerEpoch - 1) val t1 = assertThrows(classOf[ControllerMovedException], () => migrationClient.claimControllerLeadership(migrationState)) @@ -282,7 +284,7 @@ class ZkMigrationClientTest extends QuorumTestHarness { assertEquals(zkVersion, 1) migrationState = migrationClient.claimControllerLeadership(migrationState) - assertEquals(2, migrationState.controllerZkVersion()) + assertEquals(2, migrationState.zkControllerEpochZkVersion()) zkClient.getControllerEpoch match { case Some((zkEpoch, stat)) => assertEquals(3, zkEpoch) diff --git a/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java b/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java index f64a6fed53fbe..9ab0b00667dd3 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java @@ -347,7 +347,7 @@ public void run() throws Exception { case BECOME_CONTROLLER: // TODO: Handle unhappy path. apply("BecomeZkLeaderEvent", zkMigrationClient::claimControllerLeadership); - if (migrationLeadershipState.controllerZkVersion() == -1) { + if (migrationLeadershipState.zkControllerEpochZkVersion() == -1) { // We could not claim leadership, stay in BECOME_CONTROLLER to retry } else { if (!migrationLeadershipState.zkMigrationComplete()) { @@ -438,7 +438,7 @@ public void run() throws Exception { // Ignore sending RPCs to the brokers since we're no longer in the state. if (migrationState == MigrationState.KRAFT_CONTROLLER_TO_BROKER_COMM) { if (image.highestOffsetAndEpoch().compareTo(migrationLeadershipState.offsetAndEpoch()) >= 0) { - rpcClient.sendRPCsToBrokersFromMetadataImage(image, migrationLeadershipState.kraftControllerEpoch()); + rpcClient.sendRPCsToBrokersFromMetadataImage(image, migrationLeadershipState.zkControllerEpoch()); // Migration leadership state doesn't change since we're not doing any Zk writes. transitionTo(MigrationState.DUAL_WRITE); } @@ -493,7 +493,7 @@ public void run() throws Exception { // TODO: Unhappy path: Probably relinquish leadership and let new controller // retry the write? rpcClient.sendRPCsToBrokersFromMetadataDelta(delta, image, - migrationLeadershipState.kraftControllerEpoch()); + migrationLeadershipState.zkControllerEpoch()); } else { String metadataType = isSnapshot ? "snapshot" : "delta"; log.info("Ignoring {} {} which contains metadata that has already been written to ZK.", metadataType, provenance); diff --git a/metadata/src/main/java/org/apache/kafka/metadata/migration/ZkMigrationLeadershipState.java b/metadata/src/main/java/org/apache/kafka/metadata/migration/ZkMigrationLeadershipState.java index 8c7af0dfead20..1d27a4e41cdaf 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/migration/ZkMigrationLeadershipState.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/migration/ZkMigrationLeadershipState.java @@ -27,7 +27,9 @@ */ public class ZkMigrationLeadershipState { - public static final ZkMigrationLeadershipState EMPTY = new ZkMigrationLeadershipState(-1, -1, -1, -1, -1, -1, -1); + // Use -2 as sentinel for "unknown version" for ZK versions to avoid sending an actual -1 "any version" + // when doing ZK writes + public static final ZkMigrationLeadershipState EMPTY = new ZkMigrationLeadershipState(-1, -1, -1, -1, -1, -2, -1, -2); private final int kraftControllerId; @@ -41,36 +43,46 @@ public class ZkMigrationLeadershipState { private final int migrationZkVersion; - private final int controllerZkVersion; + private final int zkControllerEpoch; + + private final int zkControllerEpochZkVersion; + public ZkMigrationLeadershipState(int kraftControllerId, int kraftControllerEpoch, long kraftMetadataOffset, int kraftMetadataEpoch, - long lastUpdatedTimeMs, int migrationZkVersion, int controllerZkVersion) { + long lastUpdatedTimeMs, int migrationZkVersion, + int zkControllerEpoch, int zkControllerEpochZkVersion) { this.kraftControllerId = kraftControllerId; this.kraftControllerEpoch = kraftControllerEpoch; this.kraftMetadataOffset = kraftMetadataOffset; this.kraftMetadataEpoch = kraftMetadataEpoch; this.lastUpdatedTimeMs = lastUpdatedTimeMs; this.migrationZkVersion = migrationZkVersion; - this.controllerZkVersion = controllerZkVersion; + this.zkControllerEpoch = zkControllerEpoch; + this.zkControllerEpochZkVersion = zkControllerEpochZkVersion; } public ZkMigrationLeadershipState withMigrationZkVersion(int zkVersion) { return new ZkMigrationLeadershipState( this.kraftControllerId, this.kraftControllerEpoch, this.kraftMetadataOffset, - this.kraftMetadataEpoch, this.lastUpdatedTimeMs, zkVersion, this.controllerZkVersion); + this.kraftMetadataEpoch, this.lastUpdatedTimeMs, zkVersion, this.zkControllerEpoch, this.zkControllerEpochZkVersion); } - public ZkMigrationLeadershipState withControllerZkVersion(int zkVersion) { + public ZkMigrationLeadershipState withZkController(int zkControllerEpoch, int zkControllerEpochZkVersion) { return new ZkMigrationLeadershipState( this.kraftControllerId, this.kraftControllerEpoch, this.kraftMetadataOffset, - this.kraftMetadataEpoch, this.lastUpdatedTimeMs, this.migrationZkVersion, zkVersion); + this.kraftMetadataEpoch, this.lastUpdatedTimeMs, this.migrationZkVersion, zkControllerEpoch, zkControllerEpochZkVersion); + } + + public ZkMigrationLeadershipState withUnknownZkController() { + return withZkController(EMPTY.zkControllerEpoch, EMPTY.zkControllerEpochZkVersion); } + public ZkMigrationLeadershipState withNewKRaftController(int controllerId, int controllerEpoch) { return new ZkMigrationLeadershipState( controllerId, controllerEpoch, this.kraftMetadataOffset, - this.kraftMetadataEpoch, this.lastUpdatedTimeMs, this.migrationZkVersion, this.controllerZkVersion); + this.kraftMetadataEpoch, this.lastUpdatedTimeMs, this.migrationZkVersion, this.zkControllerEpoch, this.zkControllerEpochZkVersion); } public ZkMigrationLeadershipState withKRaftMetadataOffsetAndEpoch(long metadataOffset, @@ -82,7 +94,8 @@ public ZkMigrationLeadershipState withKRaftMetadataOffsetAndEpoch(long metadataO metadataEpoch, this.lastUpdatedTimeMs, this.migrationZkVersion, - this.controllerZkVersion); + this.zkControllerEpoch, + this.zkControllerEpochZkVersion); } public int kraftControllerId() { @@ -109,8 +122,12 @@ public int migrationZkVersion() { return migrationZkVersion; } - public int controllerZkVersion() { - return controllerZkVersion; + public int zkControllerEpoch() { + return zkControllerEpoch; + } + + public int zkControllerEpochZkVersion() { + return zkControllerEpochZkVersion; } public boolean zkMigrationComplete() { @@ -130,7 +147,8 @@ public String toString() { ", kraftMetadataEpoch=" + kraftMetadataEpoch + ", lastUpdatedTimeMs=" + lastUpdatedTimeMs + ", migrationZkVersion=" + migrationZkVersion + - ", controllerZkVersion=" + controllerZkVersion + + ", controllerZkEpoch=" + zkControllerEpoch + + ", controllerZkVersion=" + zkControllerEpochZkVersion + '}'; } @@ -145,7 +163,8 @@ public boolean equals(Object o) { && kraftMetadataEpoch == that.kraftMetadataEpoch && lastUpdatedTimeMs == that.lastUpdatedTimeMs && migrationZkVersion == that.migrationZkVersion - && controllerZkVersion == that.controllerZkVersion; + && zkControllerEpoch == that.zkControllerEpoch + && zkControllerEpochZkVersion == that.zkControllerEpochZkVersion; } @Override @@ -157,6 +176,7 @@ public int hashCode() { kraftMetadataEpoch, lastUpdatedTimeMs, migrationZkVersion, - controllerZkVersion); + zkControllerEpoch, + zkControllerEpochZkVersion); } } diff --git a/tests/kafkatest/tests/core/zookeeper_migration_test.py b/tests/kafkatest/tests/core/zookeeper_migration_test.py new file mode 100644 index 0000000000000..e69de29bb2d1d From de66d21ee797b13c808d58c536762def6121f759 Mon Sep 17 00:00:00 2001 From: David Arthur Date: Tue, 20 Dec 2022 13:55:43 -0500 Subject: [PATCH 12/30] Add better logging for migration --- .../metadata/migration/KRaftMigrationDriver.java | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java b/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java index 9ab0b00667dd3..d844dc7c641a0 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java @@ -29,15 +29,13 @@ import org.apache.kafka.raft.LeaderAndEpoch; import org.apache.kafka.raft.OffsetAndEpoch; import org.apache.kafka.server.fault.FaultHandler; -import org.apache.kafka.server.fault.LoggingFaultHandler; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.Collections; -import java.util.HashSet; -import java.util.Set; +import java.util.*; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; +import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Consumer; import java.util.function.Function; @@ -398,10 +396,12 @@ public void run() throws Exception { log.info("Starting ZK migration"); zkRecordConsumer.beginMigration(); try { + AtomicInteger count = new AtomicInteger(0); zkMigrationClient.readAllMetadata(batch -> { try { log.info("Migrating {} records from ZK: {}", batch.size(), batch); CompletableFuture future = zkRecordConsumer.acceptBatch(batch); + count.addAndGet(batch.size()); future.get(); } catch (InterruptedException e) { throw new RuntimeException(e); @@ -410,9 +410,13 @@ public void run() throws Exception { } }, brokersInMetadata::add); OffsetAndEpoch offsetAndEpochAfterMigration = zkRecordConsumer.completeMigration(); - log.info("Completed migrating metadata from Zookeeper. Current offset is {} and epoch is {}. Saw brokers {}", + log.info("Completed migration of metadata from Zookeeper to KRaft. A total of {} metadata records were " + + "generated. The current metadata offset is now {} with an epoch of {}. Saw {} brokers in the " + + "migrated metadata {}.", + count.get(), offsetAndEpochAfterMigration.offset(), offsetAndEpochAfterMigration.epoch(), + brokersInMetadata.size(), brokersInMetadata); ZkMigrationLeadershipState newState = migrationLeadershipState.withKRaftMetadataOffsetAndEpoch( offsetAndEpochAfterMigration.offset(), From b93409c9e10d5598c674bcc7342f58bc2115e2c9 Mon Sep 17 00:00:00 2001 From: David Arthur Date: Tue, 20 Dec 2022 16:42:45 -0500 Subject: [PATCH 13/30] fixup after merge --- .../java/org/apache/kafka/controller/ClusterControlManager.java | 1 - 1 file changed, 1 deletion(-) diff --git a/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java index 85aab5f247e76..d636b4b8fc6ca 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java @@ -426,7 +426,6 @@ public void replay(RegisterBrokerRecord record, long offset) { record.incarnationId(), listeners, features, Optional.ofNullable(record.rack()), record.fenced(), record.inControlledShutdown(), BrokerRegistration.zkBrokerEpoch(record.migratingZkBrokerEpoch()))); - updateMetrics(prevRegistration, brokerRegistrations.get(brokerId)); if (heartbeatManager != null) { if (prevRegistration != null) heartbeatManager.remove(brokerId); heartbeatManager.register(brokerId, record.fenced()); From 1f874b9680122f795fe0182cc5b1d9e88012e72b Mon Sep 17 00:00:00 2001 From: David Arthur Date: Wed, 21 Dec 2022 21:39:34 -0500 Subject: [PATCH 14/30] Add system test for online migration --- .../BrokerToControllerChannelManager.scala | 2 +- .../controller/ControllerMetricsManager.java | 1 + tests/kafkatest/services/kafka/kafka.py | 30 ++++++++-- tests/kafkatest/services/kafka/quorum.py | 24 ++++++-- .../tests/core/zookeeper_migration_test.py | 58 +++++++++++++++++++ 5 files changed, 104 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/kafka/server/BrokerToControllerChannelManager.scala b/core/src/main/scala/kafka/server/BrokerToControllerChannelManager.scala index 3446f83b647d8..73a77d339bd06 100644 --- a/core/src/main/scala/kafka/server/BrokerToControllerChannelManager.scala +++ b/core/src/main/scala/kafka/server/BrokerToControllerChannelManager.scala @@ -304,7 +304,7 @@ class BrokerToControllerRequestThread( initialize: Boolean = false): Unit = { if (initialize || isZkController != controllerInformation.isZkController) { if (!initialize) { - debug("Controller changed to " + (if (isZkController) "zk" else "kraft") + " mode. " + + debug("Controller changed to " + (if (isZkController) "kraft" else "zk") + " mode. " + "Resetting network client") } // Close existing network client. diff --git a/metadata/src/main/java/org/apache/kafka/controller/ControllerMetricsManager.java b/metadata/src/main/java/org/apache/kafka/controller/ControllerMetricsManager.java index d034f42cdd4e5..60c48312038ad 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ControllerMetricsManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ControllerMetricsManager.java @@ -145,6 +145,7 @@ void replay(ApiMessage message) { case ACCESS_CONTROL_ENTRY_RECORD: case REMOVE_ACCESS_CONTROL_ENTRY_RECORD: case NO_OP_RECORD: + case ZK_MIGRATION_STATE_RECORD: // These record types do not affect metrics break; default: diff --git a/tests/kafkatest/services/kafka/kafka.py b/tests/kafkatest/services/kafka/kafka.py index d3fed586b16e2..53fcb6f528e9c 100644 --- a/tests/kafkatest/services/kafka/kafka.py +++ b/tests/kafkatest/services/kafka/kafka.py @@ -203,6 +203,7 @@ def __init__(self, context, num_nodes, zk, security_protocol=SecurityConfig.PLAI remote_kafka=None, controller_num_nodes_override=0, allow_zk_with_kraft=False, + quorum_info_provider=None ): """ :param context: test context @@ -262,13 +263,16 @@ def __init__(self, context, num_nodes, zk, security_protocol=SecurityConfig.PLAI :param KafkaService remote_kafka: process.roles=controller for this cluster when not None; ignored when using ZooKeeper :param int controller_num_nodes_override: the number of nodes to use in the cluster, instead of 5, 3, or 1 based on num_nodes, if positive, not using ZooKeeper, and remote_kafka is not None; ignored otherwise :param bool allow_zk_with_kraft: if True, then allow a KRaft broker or controller to also use ZooKeeper - + :param quorum_info_provider: A function that takes this KafkaService as an argument and returns a ServiceQuorumInfo. If this is None, then the ServiceQuorumInfo is generated from the test context """ self.zk = zk self.remote_kafka = remote_kafka self.allow_zk_with_kraft = allow_zk_with_kraft - self.quorum_info = quorum.ServiceQuorumInfo(self, context) + if quorum_info_provider is None: + self.quorum_info = quorum.ServiceQuorumInfo.from_test_context(self, context) + else: + self.quorum_info = quorum_info_provider(self) self.controller_quorum = None # will define below if necessary self.remote_controller_quorum = None # will define below if necessary @@ -427,6 +431,20 @@ def __init__(self, context, num_nodes, zk, security_protocol=SecurityConfig.PLAI self.colocated_nodes_started = 0 self.nodes_to_start = self.nodes + def reconfigure_zk_for_migration(self, kraft_quorum): + #self.remote_controller_quorum = kraft_quorum + self.server_prop_overrides.extend([ + ["zookeeper.metadata.migration.enable", "true"], + ["controller.quorum.voters", kraft_quorum.controller_quorum_voters], + ["controller.listener.names", kraft_quorum.controller_listener_names] + ]) + self.port_mappings[kraft_quorum.controller_listener_names] = kraft_quorum.port_mappings.get(kraft_quorum.controller_listener_names) + + def reconfigure_zk_as_kraft(self, kraft_quorum): + self.quorum_info = quorum.ServiceQuorumInfo(quorum.remote_kraft, self) + self.remote_controller_quorum = kraft_quorum + self.controller_quorum = kraft_quorum + def num_kraft_controllers(self, num_nodes_broker_role, controller_num_nodes_override): if controller_num_nodes_override < 0: raise Exception("controller_num_nodes_override must not be negative: %i" % controller_num_nodes_override) @@ -567,7 +585,7 @@ def start_minikdc_if_necessary(self, add_principals=""): def alive(self, node): return len(self.pids(node)) > 0 - def start(self, add_principals="", nodes_to_skip=[], timeout_sec=60): + def start(self, add_principals="", nodes_to_skip=[], timeout_sec=60, **kwargs): """ Start the Kafka broker and wait until it registers its ID in ZooKeeper Startup will be skipped for any nodes in nodes_to_skip. These nodes can be started later via add_broker @@ -606,7 +624,9 @@ def start(self, add_principals="", nodes_to_skip=[], timeout_sec=60): if self.remote_controller_quorum: self.remote_controller_quorum.start() - Service.start(self) + + Service.start(self, **kwargs) + if self.concurrent_start: # We didn't wait while starting each individual node, so wait for them all now for node in self.nodes_to_start: @@ -775,7 +795,7 @@ def controller_listener_name_list(self, node): self.controller_quorum.intercontroller_security_protocol != self.controller_quorum.controller_security_protocol) \ else [broker_to_controller_listener_name] - def start_node(self, node, timeout_sec=60): + def start_node(self, node, timeout_sec=60, **kwargs): if node not in self.nodes_to_start: return node.account.mkdirs(KafkaService.PERSISTENT_ROOT) diff --git a/tests/kafkatest/services/kafka/quorum.py b/tests/kafkatest/services/kafka/quorum.py index d188eecb9beec..499085b2a2ea7 100644 --- a/tests/kafkatest/services/kafka/quorum.py +++ b/tests/kafkatest/services/kafka/quorum.py @@ -78,22 +78,21 @@ class ServiceQuorumInfo: True iff quorum_type==COLOCATED_KRAFT """ - def __init__(self, kafka, context): + def __init__(self, quorum_type, kafka): """ - :param kafka : KafkaService - The service for which this instance exposes quorum-related - information + :param quorum_type : str + The type of quorum being used. Either "ZK", "COLOCATED_KRAFT", or "REMOTE_KRAFT" :param context : TestContext The test context within which the this instance and the given Kafka service is being instantiated """ - quorum_type = for_test(context) if quorum_type != zk and kafka.zk and not kafka.allow_zk_with_kraft: raise Exception("Cannot use ZooKeeper while specifying a KRaft metadata quorum unless explicitly allowing it") if kafka.remote_kafka and quorum_type != remote_kraft: raise Exception("Cannot specify a remote Kafka service unless using a remote KRaft metadata quorum (should not happen)") + self.kafka = kafka self.quorum_type = quorum_type self.using_zk = quorum_type == zk @@ -102,6 +101,21 @@ def __init__(self, kafka, context): self.has_controllers = quorum_type == colocated_kraft or kafka.remote_kafka self.has_brokers_and_controllers = quorum_type == colocated_kraft + @staticmethod + def from_test_context(kafka, context): + """ + :param kafka : KafkaService + The service for which this instance exposes quorum-related + information + :param context : TestContext + The test context within which the this instance and the + given Kafka service is being instantiated + """ + + quorum_type = for_test(context) + return ServiceQuorumInfo(quorum_type, kafka) + + class NodeQuorumInfo: """ Exposes quorum-related information for a node in a KafkaService diff --git a/tests/kafkatest/tests/core/zookeeper_migration_test.py b/tests/kafkatest/tests/core/zookeeper_migration_test.py index e69de29bb2d1d..a0ac3b8b68d76 100644 --- a/tests/kafkatest/tests/core/zookeeper_migration_test.py +++ b/tests/kafkatest/tests/core/zookeeper_migration_test.py @@ -0,0 +1,58 @@ +from functools import partial + +from kafkatest.services.kafka import KafkaService +from kafkatest.services.kafka.config_property import CLUSTER_ID +from kafkatest.services.kafka.quorum import remote_kraft, ServiceQuorumInfo, zk +from kafkatest.services.zookeeper import ZookeeperService +from kafkatest.tests.end_to_end import EndToEndTest +from kafkatest.version import DEV_BRANCH + + +class TestMigration(EndToEndTest): + def __init__(self, test_context): + super(TestMigration, self).__init__(topic="zk-topic", test_context=test_context) + + def test_offline_migration(self): + zk_quorum = partial(ServiceQuorumInfo, zk) + self.zk = ZookeeperService(self.test_context, num_nodes=1, version=DEV_BRANCH) + self.kafka = KafkaService(self.test_context, + num_nodes=3, + zk=self.zk, + version=DEV_BRANCH, + quorum_info_provider=zk_quorum, + allow_zk_with_kraft=True, + server_prop_overrides=[["zookeeper.metadata.migration.enable", "false"]]) + self.kafka.security_protocol = "PLAINTEXT" + self.kafka.interbroker_security_protocol = "PLAINTEXT" + self.zk.start() + + self.logger.info("Pre-generating clusterId for ZK.") + cluster_id_json = """{"version": "1", "id": "%s"}""" % CLUSTER_ID + self.zk.create(path="/cluster") + self.zk.create(path="/cluster/id", value=cluster_id_json) + self.kafka.start() + + # Start up KRaft controller in migration mode + remote_quorum = partial(ServiceQuorumInfo, remote_kraft) + controller = KafkaService(self.test_context, num_nodes=1, zk=self.zk, version=DEV_BRANCH, + allow_zk_with_kraft=True, + remote_kafka=self.kafka, + server_prop_overrides=[["zookeeper.connect", self.zk.connect_setting()], + ["zookeeper.metadata.migration.enable", "true"]], + quorum_info_provider=remote_quorum) + controller.start() + + self.create_producer() + self.producer.start() + + self.create_consumer(log_level="DEBUG") + self.consumer.start() + + self.logger.info("Restarting ZK brokers in migration mode") + self.kafka.stop() + self.kafka.reconfigure_zk_for_migration(controller) + self.kafka.start() + + self.await_startup() + self.run_validation() + # self.kafka.reconfigure_zk_as_kraft(controller) From 4634fccee57f679a1d8f4ca443899fc1deaf74ae Mon Sep 17 00:00:00 2001 From: David Arthur Date: Tue, 27 Dec 2022 10:50:52 -0500 Subject: [PATCH 15/30] Send ZK broker epoch in registration Conflicts: clients/src/main/resources/common/message/BrokerRegistrationRequest.json core/src/main/scala/kafka/server/BrokerLifecycleManager.scala core/src/main/scala/kafka/server/BrokerToControllerChannelManager.scala core/src/main/scala/kafka/server/KafkaServer.scala metadata/src/main/java/org/apache/kafka/metadata/BrokerRegistration.java metadata/src/main/resources/common/metadata/RegisterBrokerRecord.json --- .gitignore | 1 + build.gradle | 3 +- checkstyle/suppressions.xml | 10 +- .../consumer/internals/SubscriptionState.java | 7 +- .../common/requests/ApiVersionsResponse.java | 4 + .../apache/kafka/common/utils/ByteUtils.java | 10 + .../kafka/common/utils/PrimitiveRef.java | 23 + .../common/message/ApiVersionsResponse.json | 5 +- .../message/BrokerRegistrationRequest.json | 2 +- .../consumer/internals/FetcherTest.java | 36 + .../common/network/SslTransportLayerTest.java | 2 +- .../kafka/common/utils/ByteUtilsTest.java | 13 + .../kafka/common/utils/PrimitiveRefTest.java | 43 + .../kafka/connect/cli/AbstractConnectCli.java | 153 ++ .../kafka/connect/cli/ConnectDistributed.java | 105 +- .../kafka/connect/cli/ConnectStandalone.java | 128 +- .../runtime/ErrorHandlingTaskTest.java | 313 ++--- .../server/builders/LogManagerBuilder.java | 2 +- .../builders/ReplicaManagerBuilder.java | 2 +- .../main/scala/kafka/cluster/Partition.scala | 1 + .../group/GroupMetadataManager.scala | 7 +- .../transaction/TransactionStateManager.scala | 8 +- core/src/main/scala/kafka/log/LazyIndex.scala | 166 --- core/src/main/scala/kafka/log/LocalLog.scala | 4 +- .../src/main/scala/kafka/log/LogCleaner.scala | 10 +- .../scala/kafka/log/LogCleanerManager.scala | 2 +- core/src/main/scala/kafka/log/LogLoader.scala | 4 +- .../src/main/scala/kafka/log/LogManager.scala | 5 + .../src/main/scala/kafka/log/LogSegment.scala | 15 +- .../main/scala/kafka/log/LogValidator.scala | 592 -------- core/src/main/scala/kafka/log/OffsetMap.scala | 201 --- .../kafka/log/ProducerStateManager.scala | 8 +- core/src/main/scala/kafka/log/TimeIndex.scala | 228 --- .../src/main/scala/kafka/log/UnifiedLog.scala | 113 +- .../kafka/log/remote/RemoteIndexCache.scala | 12 +- .../scala/kafka/raft/KafkaMetadataLog.scala | 7 +- .../kafka/server/BrokerLifecycleManager.scala | 8 +- .../scala/kafka/server/BrokerServer.scala | 1 + .../BrokerToControllerChannelManager.scala | 46 +- .../main/scala/kafka/server/KafkaApis.scala | 6 +- .../main/scala/kafka/server/KafkaServer.scala | 17 +- .../kafka/server/LogDirFailureChannel.scala | 62 - .../scala/kafka/server/MetadataCache.scala | 9 +- .../kafka/server/PartitionMetadataFile.scala | 2 +- .../scala/kafka/server/ReplicaManager.scala | 4 +- .../CheckpointFileWithFailureHandler.scala | 2 +- .../LeaderEpochCheckpointFile.scala | 2 +- .../checkpoints/OffsetCheckpointFile.scala | 2 +- .../scala/kafka/tools/DumpLogSegments.scala | 4 +- .../main/scala/kafka/utils/CoreUtils.scala | 10 - .../api/TransactionsExpirationTest.scala | 26 +- .../BrokerToControllerRequestThreadTest.scala | 50 +- .../server/LocalLeaderEndPointTest.scala | 6 +- .../scala/other/kafka/StressTestLog.scala | 3 +- .../other/kafka/TestLinearWriteSpeed.scala | 3 +- .../kafka/cluster/PartitionLockTest.scala | 3 +- .../unit/kafka/cluster/PartitionTest.scala | 13 +- .../AbstractCoordinatorConcurrencyTest.scala | 4 +- .../group/GroupCoordinatorTest.scala | 10 +- .../group/GroupMetadataManagerTest.scala | 45 +- .../TransactionStateManagerTest.scala | 14 +- .../AbstractLogCleanerIntegrationTest.scala | 3 +- .../kafka/log/BrokerCompressionTest.scala | 3 +- .../scala/unit/kafka/log/LocalLogTest.scala | 3 +- .../kafka/log/LogCleanerManagerTest.scala | 6 +- .../scala/unit/kafka/log/LogCleanerTest.scala | 68 +- .../unit/kafka/log/LogConcurrencyTest.scala | 3 +- .../scala/unit/kafka/log/LogLoaderTest.scala | 4 +- .../scala/unit/kafka/log/LogManagerTest.scala | 5 +- .../scala/unit/kafka/log/LogTestUtils.scala | 11 +- .../unit/kafka/log/LogValidatorTest.scala | 1246 ++++++++--------- .../scala/unit/kafka/log/OffsetMapTest.scala | 2 +- .../kafka/log/ProducerStateManagerTest.scala | 40 +- .../scala/unit/kafka/log/TimeIndexTest.scala | 6 +- .../scala/unit/kafka/log/UnifiedLogTest.scala | 22 +- .../log/remote/RemoteIndexCacheTest.scala | 8 +- .../log/remote/RemoteLogManagerTest.scala | 6 +- .../server/HighwatermarkPersistenceTest.scala | 1 + .../unit/kafka/server/IsrExpirationTest.scala | 1 + .../unit/kafka/server/KafkaApisTest.scala | 10 +- .../ReplicaManagerConcurrencyTest.scala | 5 +- .../server/ReplicaManagerQuotasTest.scala | 1 + .../kafka/server/ReplicaManagerTest.scala | 30 +- ...CheckpointFileWithFailureHandlerTest.scala | 2 +- .../epoch/OffsetsForLeaderEpochTest.scala | 1 + .../kafka/tools/DumpLogSegmentsTest.scala | 8 +- .../unit/kafka/utils/CoreUtilsTest.scala | 10 - .../unit/kafka/utils/SchedulerTest.scala | 3 +- .../scala/unit/kafka/utils/TestUtils.scala | 1 + gradle/spotbugs-exclude.xml | 7 + .../ReplicaFetcherThreadBenchmark.java | 2 +- .../PartitionMakeFollowerBenchmark.java | 2 +- .../UpdateFollowerFetchStateBenchmark.java | 2 +- .../jmh/record/BaseRecordBatchBenchmark.java | 5 +- ...pressedRecordBatchValidationBenchmark.java | 21 +- ...pressedRecordBatchValidationBenchmark.java | 17 +- .../kafka/jmh/server/CheckpointBench.java | 2 +- .../jmh/server/PartitionCreationBench.java | 2 +- .../controller/ClusterControlManager.java | 1 + .../kafka/metadata/BrokerRegistration.java | 2 +- .../common/metadata/RegisterBrokerRecord.json | 2 +- .../metadata/BrokerRegistrationTest.java | 25 +- .../server/log/internals/AbstractIndex.java | 57 +- .../server/log/internals/AppendOrigin.java | 45 + .../kafka/server/log/internals/LazyIndex.java | 245 ++++ .../log/internals/LogDirFailureChannel.java | 77 + .../server/log/internals/LogValidator.java | 631 +++++++++ .../kafka/server/log/internals/OffsetMap.java | 29 +- .../internals/RecordValidationException.java | 41 + .../server/log/internals/SkimpyOffsetMap.java | 242 ++++ .../kafka/server/log/internals/TimeIndex.java | 286 ++++ .../internals/DefaultStateUpdater.java | 3 +- .../internals/StreamsPartitionAssignor.java | 1 - .../StoreQueryIntegrationTest.java | 66 +- ...tionStoreReceiveProcessorSupplierTest.java | 21 +- ...bstractRocksDBSegmentedBytesStoreTest.java | 1 + ...sedTableConfigWithAccessibleCacheTest.java | 9 +- ...OptionsColumnFamilyOptionsAdapterTest.java | 34 +- tests/kafkatest/services/kafka/kafka.py | 2 +- .../tools/MetadataQuorumCommandTest.java | 26 +- 120 files changed, 3314 insertions(+), 2698 deletions(-) create mode 100644 clients/src/test/java/org/apache/kafka/common/utils/PrimitiveRefTest.java create mode 100644 connect/runtime/src/main/java/org/apache/kafka/connect/cli/AbstractConnectCli.java delete mode 100644 core/src/main/scala/kafka/log/LazyIndex.scala delete mode 100644 core/src/main/scala/kafka/log/LogValidator.scala delete mode 100755 core/src/main/scala/kafka/log/OffsetMap.scala delete mode 100644 core/src/main/scala/kafka/log/TimeIndex.scala delete mode 100644 core/src/main/scala/kafka/server/LogDirFailureChannel.scala mode change 100755 => 100644 core/src/test/scala/unit/kafka/log/LogCleanerTest.scala create mode 100644 storage/src/main/java/org/apache/kafka/server/log/internals/AppendOrigin.java create mode 100644 storage/src/main/java/org/apache/kafka/server/log/internals/LazyIndex.java create mode 100644 storage/src/main/java/org/apache/kafka/server/log/internals/LogDirFailureChannel.java create mode 100644 storage/src/main/java/org/apache/kafka/server/log/internals/LogValidator.java rename core/src/main/scala/kafka/common/RecordValidationException.scala => storage/src/main/java/org/apache/kafka/server/log/internals/OffsetMap.java (53%) create mode 100644 storage/src/main/java/org/apache/kafka/server/log/internals/RecordValidationException.java create mode 100644 storage/src/main/java/org/apache/kafka/server/log/internals/SkimpyOffsetMap.java create mode 100644 storage/src/main/java/org/apache/kafka/server/log/internals/TimeIndex.java diff --git a/.gitignore b/.gitignore index 095a3141d1bad..f466af2c59828 100644 --- a/.gitignore +++ b/.gitignore @@ -6,6 +6,7 @@ build/ build_eclipse/ out/ .gradle/ +.vscode/ lib_managed/ src_managed/ project/boot/ diff --git a/build.gradle b/build.gradle index 9bf33767f0917..776bbbcc699fa 100644 --- a/build.gradle +++ b/build.gradle @@ -412,7 +412,7 @@ subprojects { // connect tests "**/ConnectorPluginsResourceTest.*", "**/DistributedHerderTest.*", "**/FileOffsetBakingStoreTest.*", - "**/ErrorHandlingTaskTest.*", "**/KafkaConfigBackingStoreTest.*", + "**/KafkaConfigBackingStoreTest.*", "**/KafkaBasedLogTest.*", "**/OffsetStorageWriterTest.*", "**/StandaloneHerderTest.*", "**/SourceTaskOffsetCommitterTest.*", "**/WorkerSinkTaskTest.*", "**/WorkerSinkTaskThreadedTest.*", @@ -2413,6 +2413,7 @@ project(':jmh-benchmarks') { implementation project(':server-common') implementation project(':clients') implementation project(':metadata') + implementation project(':storage') implementation project(':streams') implementation project(':core') implementation project(':clients').sourceSets.test.output diff --git a/checkstyle/suppressions.xml b/checkstyle/suppressions.xml index cea6a193790a8..f49c3e8edb65e 100644 --- a/checkstyle/suppressions.xml +++ b/checkstyle/suppressions.xml @@ -318,9 +318,13 @@ - - + + + + preferredReadReplica(TopicPartition tp, lo * @return the removed preferred read replica if set, None otherwise. */ public synchronized Optional clearPreferredReadReplica(TopicPartition tp) { - return assignedState(tp).clearPreferredReadReplica(); + final TopicPartitionState topicPartitionState = assignedStateOrNull(tp); + if (topicPartitionState == null) { + return Optional.empty(); + } else { + return topicPartitionState.clearPreferredReadReplica(); + } } public synchronized Map allConsumed() { diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java index a903e50b15d9e..17986fa10862c 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java @@ -84,6 +84,10 @@ public boolean shouldClientThrottle(short version) { return version >= 2; } + public boolean zkMigrationReady() { + return data.zkMigrationReady(); + } + public static ApiVersionsResponse parse(ByteBuffer buffer, short version) { // Fallback to version 0 for ApiVersions response. If a client sends an ApiVersionsRequest // using a version higher than that supported by the broker, a version 0 response is sent diff --git a/clients/src/main/java/org/apache/kafka/common/utils/ByteUtils.java b/clients/src/main/java/org/apache/kafka/common/utils/ByteUtils.java index 7bd1d9257f84c..663ef4709a73e 100644 --- a/clients/src/main/java/org/apache/kafka/common/utils/ByteUtils.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/ByteUtils.java @@ -81,6 +81,16 @@ public static int readUnsignedIntLE(byte[] buffer, int offset) { | ((buffer[offset + 3] & 0xff) << 24); } + /** + * Read a big-endian integer from a byte array + */ + public static int readIntBE(byte[] buffer, int offset) { + return ((buffer[offset] & 0xFF) << 24) + | ((buffer[offset + 1] & 0xFF) << 16) + | ((buffer[offset + 2] & 0xFF) << 8) + | (buffer[offset + 3] & 0xFF); + } + /** * Write the given long value as a 4 byte unsigned integer. Overflow is ignored. * diff --git a/clients/src/main/java/org/apache/kafka/common/utils/PrimitiveRef.java b/clients/src/main/java/org/apache/kafka/common/utils/PrimitiveRef.java index e1bbfe373161c..d41fad2f98e94 100644 --- a/clients/src/main/java/org/apache/kafka/common/utils/PrimitiveRef.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/PrimitiveRef.java @@ -26,11 +26,34 @@ public static IntRef ofInt(int value) { return new IntRef(value); } + public static LongRef ofLong(long value) { + return new LongRef(value); + } + public static class IntRef { public int value; IntRef(int value) { this.value = value; } + + @Override + public String toString() { + return "IntRef(" + value + ")"; + } + + } + + public static class LongRef { + public long value; + + LongRef(long value) { + this.value = value; + } + + @Override + public String toString() { + return "LongRef(" + value + ")"; + } } } diff --git a/clients/src/main/resources/common/message/ApiVersionsResponse.json b/clients/src/main/resources/common/message/ApiVersionsResponse.json index 06c343b6fa3dc..9fda953e10e57 100644 --- a/clients/src/main/resources/common/message/ApiVersionsResponse.json +++ b/clients/src/main/resources/common/message/ApiVersionsResponse.json @@ -69,6 +69,9 @@ {"name": "MinVersionLevel", "type": "int16", "versions": "3+", "about": "The cluster-wide finalized min version level for the feature."} ] - } + }, + { "name": "ZkMigrationReady", "type": "bool", "versions": "3+", "taggedVersions": "3+", + "tag": 3, "ignorable": true, "default": "false", + "about": "Set by a KRaft controller if the required configurations for ZK migration are present" } ] } diff --git a/clients/src/main/resources/common/message/BrokerRegistrationRequest.json b/clients/src/main/resources/common/message/BrokerRegistrationRequest.json index 39640f0724279..43b126ec0278a 100644 --- a/clients/src/main/resources/common/message/BrokerRegistrationRequest.json +++ b/clients/src/main/resources/common/message/BrokerRegistrationRequest.json @@ -52,6 +52,6 @@ { "name": "Rack", "type": "string", "versions": "0+", "nullableVersions": "0+", "about": "The rack which this broker is in." }, { "name": "MigratingZkBrokerEpoch", "type": "int64", "versions": "1+", "default": "-1", - "about": "Set by a ZK broker if the required configurations for ZK migration are present." } + "about": "If the required configurations for ZK migration are present, this value is set to the ZK broker epoch" } ] } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java index 0e14355ecc81a..595f6404d63ad 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java @@ -137,6 +137,7 @@ import static java.util.Arrays.asList; import static java.util.Collections.emptyList; import static java.util.Collections.emptyMap; +import static java.util.Collections.emptySet; import static java.util.Collections.singleton; import static java.util.Collections.singletonList; import static java.util.Collections.singletonMap; @@ -4762,6 +4763,41 @@ public void testFetchDisconnectedShouldClearPreferredReadReplica() { assertEquals(-1, selected.id()); } + @Test + public void testFetchDisconnectedShouldNotClearPreferredReadReplicaIfUnassigned() { + buildFetcher(new MetricConfig(), OffsetResetStrategy.EARLIEST, new BytesDeserializer(), new BytesDeserializer(), + Integer.MAX_VALUE, IsolationLevel.READ_COMMITTED, Duration.ofMinutes(5).toMillis()); + + subscriptions.assignFromUser(singleton(tp0)); + client.updateMetadata(RequestTestUtils.metadataUpdateWithIds(2, singletonMap(topicName, 4), tp -> validLeaderEpoch, topicIds, false)); + subscriptions.seek(tp0, 0); + assertEquals(1, fetcher.sendFetches()); + + // Set preferred read replica to node=1 + client.prepareResponse(fullFetchResponse(tidp0, this.records, Errors.NONE, 100L, + FetchResponse.INVALID_LAST_STABLE_OFFSET, 0, Optional.of(1))); + consumerClient.poll(time.timer(0)); + assertTrue(fetcher.hasCompletedFetches()); + fetchedRecords(); + + // Verify + Node selected = fetcher.selectReadReplica(tp0, Node.noNode(), time.milliseconds()); + assertEquals(1, selected.id()); + assertEquals(1, fetcher.sendFetches()); + assertFalse(fetcher.hasCompletedFetches()); + + // Disconnect and remove tp0 from assignment + client.prepareResponse(fullFetchResponse(tidp0, this.records, Errors.NONE, 100L, 0), true); + subscriptions.assignFromUser(emptySet()); + + // Preferred read replica should not be cleared + consumerClient.poll(time.timer(0)); + assertFalse(fetcher.hasCompletedFetches()); + fetchedRecords(); + selected = fetcher.selectReadReplica(tp0, Node.noNode(), time.milliseconds()); + assertEquals(-1, selected.id()); + } + @Test public void testFetchErrorShouldClearPreferredReadReplica() { buildFetcher(new MetricConfig(), OffsetResetStrategy.EARLIEST, new BytesDeserializer(), new BytesDeserializer(), diff --git a/clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java b/clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java index d78e5f44b27a2..a391208faaf0e 100644 --- a/clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java +++ b/clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java @@ -867,7 +867,7 @@ private void testIOExceptionsDuringHandshake(Args args, channelBuilder.flushFailureAction = flushFailureAction; channelBuilder.failureIndex = i; channelBuilder.configure(args.sslClientConfigs); - this.selector = new Selector(5000, new Metrics(), time, "MetricGroup", channelBuilder, new LogContext()); + this.selector = new Selector(10000, new Metrics(), time, "MetricGroup", channelBuilder, new LogContext()); InetSocketAddress addr = new InetSocketAddress("localhost", server.port()); selector.connect(node, addr, BUFFER_SIZE, BUFFER_SIZE); diff --git a/clients/src/test/java/org/apache/kafka/common/utils/ByteUtilsTest.java b/clients/src/test/java/org/apache/kafka/common/utils/ByteUtilsTest.java index 5f855fa4a9c76..6f3b6904febe7 100644 --- a/clients/src/test/java/org/apache/kafka/common/utils/ByteUtilsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/utils/ByteUtilsTest.java @@ -284,6 +284,19 @@ public void testSizeOfVarlong() { assertEquals(simpleImplementation.apply(0), ByteUtils.sizeOfVarlong(0)); } + @Test + public void testReadInt() { + int[] values = { + 0, 1, -1, Byte.MAX_VALUE, Short.MAX_VALUE, 2 * Short.MAX_VALUE, Integer.MAX_VALUE / 2, + Integer.MIN_VALUE / 2, Integer.MAX_VALUE, Integer.MIN_VALUE, Integer.MAX_VALUE + }; + ByteBuffer buffer = ByteBuffer.allocate(4 * values.length); + for (int i = 0; i < values.length; ++i) { + buffer.putInt(i * 4, values[i]); + assertEquals(values[i], ByteUtils.readIntBE(buffer.array(), i * 4), "Written value should match read value."); + } + } + private void assertUnsignedVarintSerde(int value, byte[] expectedEncoding) throws IOException { ByteBuffer buf = ByteBuffer.allocate(32); ByteUtils.writeUnsignedVarint(value, buf); diff --git a/clients/src/test/java/org/apache/kafka/common/utils/PrimitiveRefTest.java b/clients/src/test/java/org/apache/kafka/common/utils/PrimitiveRefTest.java new file mode 100644 index 0000000000000..1242136afd5ff --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/utils/PrimitiveRefTest.java @@ -0,0 +1,43 @@ +/* + * 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.common.utils; + +import org.junit.jupiter.api.Test; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +public class PrimitiveRefTest { + + @Test + public void testIntRef() { + PrimitiveRef.IntRef ref = PrimitiveRef.ofInt(3); + assertEquals(3, ref.value++); + assertEquals(4, ref.value); + assertEquals(5, ++ref.value); + assertEquals(5, ref.value); + } + + @Test + public void testLongRef() { + PrimitiveRef.LongRef ref = PrimitiveRef.ofLong(5L); + assertEquals(5L, ref.value++); + assertEquals(6L, ref.value); + assertEquals(7L, ++ref.value); + assertEquals(7L, ref.value); + } + +} diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/cli/AbstractConnectCli.java b/connect/runtime/src/main/java/org/apache/kafka/connect/cli/AbstractConnectCli.java new file mode 100644 index 0000000000000..8831081bf4443 --- /dev/null +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/cli/AbstractConnectCli.java @@ -0,0 +1,153 @@ +/* + * 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.connect.cli; + +import org.apache.kafka.common.utils.Exit; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.connect.connector.policy.ConnectorClientConfigOverridePolicy; +import org.apache.kafka.connect.runtime.Connect; +import org.apache.kafka.connect.runtime.Herder; +import org.apache.kafka.connect.runtime.WorkerConfig; +import org.apache.kafka.connect.runtime.WorkerInfo; +import org.apache.kafka.connect.runtime.isolation.Plugins; +import org.apache.kafka.connect.runtime.rest.RestClient; +import org.apache.kafka.connect.runtime.rest.RestServer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.net.URI; +import java.util.Arrays; +import java.util.Collections; +import java.util.Map; + +/** + * Common initialization logic for Kafka Connect, intended for use by command line utilities + * + * @param the type of {@link WorkerConfig} to be used + */ +public abstract class AbstractConnectCli { + + private static final Logger log = LoggerFactory.getLogger(AbstractConnectCli.class); + private final String[] args; + private final Time time = Time.SYSTEM; + + /** + * + * @param args the CLI arguments to be processed. Note that if one or more arguments are passed, the first argument is + * assumed to be the Connect worker properties file and is processed in {@link #run()}. The remaining arguments + * can be handled in {@link #processExtraArgs(Herder, Connect, String[])} + */ + protected AbstractConnectCli(String... args) { + this.args = args; + } + + protected abstract String usage(); + + /** + * The first CLI argument is assumed to be the Connect worker properties file and is processed by default. This method + * can be overridden if there are more arguments that need to be processed. + * + * @param herder the {@link Herder} instance that can be used to perform operations on the Connect cluster + * @param connect the {@link Connect} instance that can be stopped (via {@link Connect#stop()}) if there's an error + * encountered while processing the additional CLI arguments. + * @param extraArgs the extra CLI arguments that need to be processed + */ + protected void processExtraArgs(Herder herder, Connect connect, String[] extraArgs) { + } + + protected abstract Herder createHerder(T config, String workerId, Plugins plugins, + ConnectorClientConfigOverridePolicy connectorClientConfigOverridePolicy, + RestServer restServer, RestClient restClient); + + protected abstract T createConfig(Map workerProps); + + /** + * Validate {@link #args}, process worker properties from the first CLI argument, and start {@link Connect} + */ + public void run() { + if (args.length < 1 || Arrays.asList(args).contains("--help")) { + log.info("Usage: {}", usage()); + Exit.exit(1); + } + + try { + String workerPropsFile = args[0]; + Map workerProps = !workerPropsFile.isEmpty() ? + Utils.propsToStringMap(Utils.loadProps(workerPropsFile)) : Collections.emptyMap(); + String[] extraArgs = Arrays.copyOfRange(args, 1, args.length); + Connect connect = startConnect(workerProps, extraArgs); + + // Shutdown will be triggered by Ctrl-C or via HTTP shutdown request + connect.awaitStop(); + + } catch (Throwable t) { + log.error("Stopping due to error", t); + Exit.exit(2); + } + } + + /** + * Initialize and start an instance of {@link Connect} + * + * @param workerProps the worker properties map used to initialize the {@link WorkerConfig} + * @param extraArgs any additional CLI arguments that may need to be processed via + * {@link #processExtraArgs(Herder, Connect, String[])} + * @return a started instance of {@link Connect} + */ + public Connect startConnect(Map workerProps, String... extraArgs) { + log.info("Kafka Connect worker initializing ..."); + long initStart = time.hiResClockMs(); + + WorkerInfo initInfo = new WorkerInfo(); + initInfo.logAll(); + + log.info("Scanning for plugin classes. This might take a moment ..."); + Plugins plugins = new Plugins(workerProps); + plugins.compareAndSwapWithDelegatingLoader(); + T config = createConfig(workerProps); + log.debug("Kafka cluster ID: {}", config.kafkaClusterId()); + + RestClient restClient = new RestClient(config); + + RestServer restServer = new RestServer(config, restClient); + restServer.initializeServer(); + + URI advertisedUrl = restServer.advertisedUrl(); + String workerId = advertisedUrl.getHost() + ":" + advertisedUrl.getPort(); + + ConnectorClientConfigOverridePolicy connectorClientConfigOverridePolicy = plugins.newPlugin( + config.getString(WorkerConfig.CONNECTOR_CLIENT_POLICY_CLASS_CONFIG), + config, ConnectorClientConfigOverridePolicy.class); + + Herder herder = createHerder(config, workerId, plugins, connectorClientConfigOverridePolicy, restServer, restClient); + + final Connect connect = new Connect(herder, restServer); + log.info("Kafka Connect worker initialization took {}ms", time.hiResClockMs() - initStart); + try { + connect.start(); + } catch (Exception e) { + log.error("Failed to start Connect", e); + connect.stop(); + Exit.exit(3); + } + + processExtraArgs(herder, connect, extraArgs); + + return connect; + } +} diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectDistributed.java b/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectDistributed.java index a786a172a2901..1da9fa406225b 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectDistributed.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectDistributed.java @@ -16,15 +16,11 @@ */ package org.apache.kafka.connect.cli; -import org.apache.kafka.common.utils.Exit; import org.apache.kafka.common.utils.Time; -import org.apache.kafka.common.utils.Utils; import org.apache.kafka.connect.connector.policy.ConnectorClientConfigOverridePolicy; -import org.apache.kafka.connect.runtime.Connect; +import org.apache.kafka.connect.runtime.Herder; import org.apache.kafka.connect.runtime.Worker; -import org.apache.kafka.connect.runtime.WorkerConfig; import org.apache.kafka.connect.runtime.WorkerConfigTransformer; -import org.apache.kafka.connect.runtime.WorkerInfo; import org.apache.kafka.connect.runtime.distributed.DistributedConfig; import org.apache.kafka.connect.runtime.distributed.DistributedHerder; import org.apache.kafka.connect.runtime.isolation.Plugins; @@ -41,9 +37,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.net.URI; -import java.util.Arrays; -import java.util.Collections; import java.util.HashMap; import java.util.Map; @@ -51,65 +44,32 @@ /** *

- * Command line utility that runs Kafka Connect in distributed mode. In this mode, the process joints a group of other workers - * and work is distributed among them. This is useful for running Connect as a service, where connectors can be - * submitted to the cluster to be automatically executed in a scalable, distributed fashion. This also allows you to - * easily scale out horizontally, elastically adding or removing capacity simply by starting or stopping worker - * instances. + * Command line utility that runs Kafka Connect in distributed mode. In this mode, the process joins a group of other + * workers and work (connectors and tasks) is distributed among them. This is useful for running Connect as a service, + * where connectors can be submitted to the cluster to be automatically executed in a scalable, distributed fashion. + * This also allows you to easily scale out horizontally, elastically adding or removing capacity simply by starting or + * stopping worker instances. *

*/ -public class ConnectDistributed { +public class ConnectDistributed extends AbstractConnectCli { private static final Logger log = LoggerFactory.getLogger(ConnectDistributed.class); - private final Time time = Time.SYSTEM; - private final long initStart = time.hiResClockMs(); - - public static void main(String[] args) { - - if (args.length < 1 || Arrays.asList(args).contains("--help")) { - log.info("Usage: ConnectDistributed worker.properties"); - Exit.exit(1); - } - - try { - WorkerInfo initInfo = new WorkerInfo(); - initInfo.logAll(); - - String workerPropsFile = args[0]; - Map workerProps = !workerPropsFile.isEmpty() ? - Utils.propsToStringMap(Utils.loadProps(workerPropsFile)) : Collections.emptyMap(); - - ConnectDistributed connectDistributed = new ConnectDistributed(); - Connect connect = connectDistributed.startConnect(workerProps); - - // Shutdown will be triggered by Ctrl-C or via HTTP shutdown request - connect.awaitStop(); + public ConnectDistributed(String... args) { + super(args); + } - } catch (Throwable t) { - log.error("Stopping due to error", t); - Exit.exit(2); - } + @Override + protected String usage() { + return "ConnectDistributed worker.properties"; } - public Connect startConnect(Map workerProps) { - log.info("Scanning for plugin classes. This might take a moment ..."); - Plugins plugins = new Plugins(workerProps); - plugins.compareAndSwapWithDelegatingLoader(); - DistributedConfig config = new DistributedConfig(workerProps); + @Override + protected Herder createHerder(DistributedConfig config, String workerId, Plugins plugins, + ConnectorClientConfigOverridePolicy connectorClientConfigOverridePolicy, + RestServer restServer, RestClient restClient) { String kafkaClusterId = config.kafkaClusterId(); - log.debug("Kafka cluster ID: {}", kafkaClusterId); - - RestClient restClient = new RestClient(config); - - RestServer rest = new RestServer(config, restClient); - rest.initializeServer(); - - URI advertisedUrl = rest.advertisedUrl(); - String workerId = advertisedUrl.getHost() + ":" + advertisedUrl.getPort(); - String clientIdBase = ConnectUtils.clientIdBase(config); - // Create the admin client to be shared by all backing stores. Map adminProps = new HashMap<>(config.originals()); ConnectUtils.addMetricsContextProperties(adminProps, config, kafkaClusterId); @@ -119,15 +79,11 @@ public Connect startConnect(Map workerProps) { KafkaOffsetBackingStore offsetBackingStore = new KafkaOffsetBackingStore(sharedAdmin, () -> clientIdBase); offsetBackingStore.configure(config); - ConnectorClientConfigOverridePolicy connectorClientConfigOverridePolicy = plugins.newPlugin( - config.getString(WorkerConfig.CONNECTOR_CLIENT_POLICY_CLASS_CONFIG), - config, ConnectorClientConfigOverridePolicy.class); - - Worker worker = new Worker(workerId, time, plugins, config, offsetBackingStore, connectorClientConfigOverridePolicy); + Worker worker = new Worker(workerId, Time.SYSTEM, plugins, config, offsetBackingStore, connectorClientConfigOverridePolicy); WorkerConfigTransformer configTransformer = worker.configTransformer(); Converter internalValueConverter = worker.getInternalValueConverter(); - StatusBackingStore statusBackingStore = new KafkaStatusBackingStore(time, internalValueConverter, sharedAdmin, clientIdBase); + StatusBackingStore statusBackingStore = new KafkaStatusBackingStore(Time.SYSTEM, internalValueConverter, sharedAdmin, clientIdBase); statusBackingStore.configure(config); ConfigBackingStore configBackingStore = new KafkaConfigBackingStore( @@ -139,21 +95,18 @@ public Connect startConnect(Map workerProps) { // Pass the shared admin to the distributed herder as an additional AutoCloseable object that should be closed when the // herder is stopped. This is easier than having to track and own the lifecycle ourselves. - DistributedHerder herder = new DistributedHerder(config, time, worker, + return new DistributedHerder(config, Time.SYSTEM, worker, kafkaClusterId, statusBackingStore, configBackingStore, - advertisedUrl.toString(), restClient, connectorClientConfigOverridePolicy, sharedAdmin); - - final Connect connect = new Connect(herder, rest); - log.info("Kafka Connect distributed worker initialization took {}ms", time.hiResClockMs() - initStart); - try { - connect.start(); - } catch (Exception e) { - log.error("Failed to start Connect", e); - connect.stop(); - Exit.exit(3); - } + restServer.advertisedUrl().toString(), restClient, connectorClientConfigOverridePolicy, sharedAdmin); + } - return connect; + @Override + protected DistributedConfig createConfig(Map workerProps) { + return new DistributedConfig(workerProps); } + public static void main(String[] args) { + ConnectDistributed connectDistributed = new ConnectDistributed(args); + connectDistributed.run(); + } } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectStandalone.java b/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectStandalone.java index 8b0f4cd2b58a6..49b59dc7e46c7 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectStandalone.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectStandalone.java @@ -24,9 +24,8 @@ import org.apache.kafka.connect.runtime.ConnectorConfig; import org.apache.kafka.connect.runtime.Herder; import org.apache.kafka.connect.runtime.Worker; -import org.apache.kafka.connect.runtime.WorkerConfig; -import org.apache.kafka.connect.runtime.WorkerInfo; import org.apache.kafka.connect.runtime.isolation.Plugins; +import org.apache.kafka.connect.runtime.rest.RestClient; import org.apache.kafka.connect.runtime.rest.RestServer; import org.apache.kafka.connect.runtime.rest.entities.ConnectorInfo; import org.apache.kafka.connect.runtime.standalone.StandaloneConfig; @@ -37,98 +36,75 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.net.URI; -import java.util.Arrays; -import java.util.Collections; import java.util.Map; /** *

- * Command line utility that runs Kafka Connect as a standalone process. In this mode, work is not - * distributed. Instead, all the normal Connect machinery works within a single process. This is - * useful for ad hoc, small, or experimental jobs. + * Command line utility that runs Kafka Connect as a standalone process. In this mode, work (connectors and tasks) is not + * distributed. Instead, all the normal Connect machinery works within a single process. This is useful for ad hoc, + * small, or experimental jobs. *

*

- * By default, no job configs or offset data is persistent. You can make jobs persistent and - * fault tolerant by overriding the settings to use file storage for both. + * Connector and task configs are stored in memory and are not persistent. However, connector offset data is persistent + * since it uses file storage (configurable via {@link StandaloneConfig#OFFSET_STORAGE_FILE_FILENAME_CONFIG}) *

*/ -public class ConnectStandalone { +public class ConnectStandalone extends AbstractConnectCli { private static final Logger log = LoggerFactory.getLogger(ConnectStandalone.class); - public static void main(String[] args) { + protected ConnectStandalone(String... args) { + super(args); + } - if (args.length < 1 || Arrays.asList(args).contains("--help")) { - log.info("Usage: ConnectStandalone worker.properties [connector1.properties connector2.properties ...]"); - Exit.exit(1); - } + @Override + protected String usage() { + return "ConnectStandalone worker.properties [connector1.properties connector2.properties ...]"; + } + @Override + protected void processExtraArgs(Herder herder, Connect connect, String[] extraArgs) { try { - Time time = Time.SYSTEM; - log.info("Kafka Connect standalone worker initializing ..."); - long initStart = time.hiResClockMs(); - WorkerInfo initInfo = new WorkerInfo(); - initInfo.logAll(); - - String workerPropsFile = args[0]; - Map workerProps = !workerPropsFile.isEmpty() ? - Utils.propsToStringMap(Utils.loadProps(workerPropsFile)) : Collections.emptyMap(); - - log.info("Scanning for plugin classes. This might take a moment ..."); - Plugins plugins = new Plugins(workerProps); - plugins.compareAndSwapWithDelegatingLoader(); - StandaloneConfig config = new StandaloneConfig(workerProps); - - String kafkaClusterId = config.kafkaClusterId(); - log.debug("Kafka cluster ID: {}", kafkaClusterId); - - // Do not initialize a RestClient because the ConnectorsResource will not use it in standalone mode. - RestServer rest = new RestServer(config, null); - rest.initializeServer(); - - URI advertisedUrl = rest.advertisedUrl(); - String workerId = advertisedUrl.getHost() + ":" + advertisedUrl.getPort(); + for (final String connectorPropsFile : extraArgs) { + Map connectorProps = Utils.propsToStringMap(Utils.loadProps(connectorPropsFile)); + FutureCallback> cb = new FutureCallback<>((error, info) -> { + if (error != null) + log.error("Failed to create connector for {}", connectorPropsFile); + else + log.info("Created connector {}", info.result().name()); + }); + herder.putConnectorConfig( + connectorProps.get(ConnectorConfig.NAME_CONFIG), + connectorProps, false, cb); + cb.get(); + } + } catch (Throwable t) { + log.error("Stopping after connector error", t); + connect.stop(); + Exit.exit(3); + } + } - OffsetBackingStore offsetBackingStore = new FileOffsetBackingStore(); - offsetBackingStore.configure(config); + @Override + protected Herder createHerder(StandaloneConfig config, String workerId, Plugins plugins, + ConnectorClientConfigOverridePolicy connectorClientConfigOverridePolicy, + RestServer restServer, RestClient restClient) { - ConnectorClientConfigOverridePolicy connectorClientConfigOverridePolicy = plugins.newPlugin( - config.getString(WorkerConfig.CONNECTOR_CLIENT_POLICY_CLASS_CONFIG), - config, ConnectorClientConfigOverridePolicy.class); - Worker worker = new Worker(workerId, time, plugins, config, offsetBackingStore, - connectorClientConfigOverridePolicy); + OffsetBackingStore offsetBackingStore = new FileOffsetBackingStore(); + offsetBackingStore.configure(config); - Herder herder = new StandaloneHerder(worker, kafkaClusterId, connectorClientConfigOverridePolicy); - final Connect connect = new Connect(herder, rest); - log.info("Kafka Connect standalone worker initialization took {}ms", time.hiResClockMs() - initStart); + Worker worker = new Worker(workerId, Time.SYSTEM, plugins, config, offsetBackingStore, + connectorClientConfigOverridePolicy); - try { - connect.start(); - for (final String connectorPropsFile : Arrays.copyOfRange(args, 1, args.length)) { - Map connectorProps = Utils.propsToStringMap(Utils.loadProps(connectorPropsFile)); - FutureCallback> cb = new FutureCallback<>((error, info) -> { - if (error != null) - log.error("Failed to create job for {}", connectorPropsFile); - else - log.info("Created connector {}", info.result().name()); - }); - herder.putConnectorConfig( - connectorProps.get(ConnectorConfig.NAME_CONFIG), - connectorProps, false, cb); - cb.get(); - } - } catch (Throwable t) { - log.error("Stopping after connector error", t); - connect.stop(); - Exit.exit(3); - } + return new StandaloneHerder(worker, config.kafkaClusterId(), connectorClientConfigOverridePolicy); + } - // Shutdown will be triggered by Ctrl-C or via HTTP shutdown request - connect.awaitStop(); + @Override + protected StandaloneConfig createConfig(Map workerProps) { + return new StandaloneConfig(workerProps); + } - } catch (Throwable t) { - log.error("Stopping due to error", t); - Exit.exit(2); - } + public static void main(String[] args) { + ConnectStandalone connectStandalone = new ConnectStandalone(args); + connectStandalone.run(); } } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ErrorHandlingTaskTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ErrorHandlingTaskTest.java index 780ca4e790e94..4de13128dbccc 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ErrorHandlingTaskTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ErrorHandlingTaskTest.java @@ -58,33 +58,29 @@ import org.apache.kafka.connect.transforms.Transformation; import org.apache.kafka.connect.transforms.util.SimpleConfig; import org.apache.kafka.connect.util.ConnectorTaskId; -import org.apache.kafka.connect.util.ParameterizedTest; import org.apache.kafka.connect.util.TopicAdmin; import org.apache.kafka.connect.util.TopicCreationGroup; -import org.easymock.Capture; -import org.easymock.EasyMock; -import org.easymock.IExpectationSetters; import org.junit.After; import org.junit.Before; +import org.junit.Rule; import org.junit.Test; import org.junit.runner.RunWith; -import org.powermock.api.easymock.PowerMock; -import org.powermock.api.easymock.annotation.Mock; -import org.powermock.core.classloader.annotations.PowerMockIgnore; -import org.powermock.core.classloader.annotations.PrepareForTest; -import org.powermock.modules.junit4.PowerMockRunner; -import org.powermock.modules.junit4.PowerMockRunnerDelegate; +import org.junit.runners.Parameterized; +import org.mockito.Mock; +import org.mockito.junit.MockitoJUnit; +import org.mockito.junit.MockitoRule; +import org.mockito.quality.Strictness; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; import java.time.Duration; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; import java.util.HashMap; import java.util.Map; +import java.util.Arrays; import java.util.Set; +import java.util.Collections; +import java.util.Collection; import java.util.concurrent.Executor; import static java.util.Collections.emptyMap; @@ -103,18 +99,29 @@ import static org.apache.kafka.connect.runtime.WorkerConfig.TOPIC_CREATION_ENABLE_CONFIG; import static org.junit.Assert.assertEquals; -@RunWith(PowerMockRunner.class) -@PowerMockRunnerDelegate(ParameterizedTest.class) -@PrepareForTest({WorkerSinkTask.class, WorkerSourceTask.class}) -@PowerMockIgnore("javax.management.*") +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.when; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.doReturn; + + +@RunWith(Parameterized.class) public class ErrorHandlingTaskTest { + @Rule + public MockitoRule rule = MockitoJUnit.rule().strictness(Strictness.STRICT_STUBS); private static final String TOPIC = "test"; private static final int PARTITION1 = 12; private static final int PARTITION2 = 13; private static final long FIRST_OFFSET = 45; - @Mock Plugins plugins; + @Mock + Plugins plugins; private static final Map TASK_PROPS = new HashMap<>(); @@ -139,7 +146,6 @@ public class ErrorHandlingTaskTest { @SuppressWarnings("unused") @Mock private SourceTask sourceTask; - private Capture sinkTaskContext = EasyMock.newCapture(); private WorkerConfig workerConfig; private SourceConnectorConfig sourceConfig; @Mock @@ -164,8 +170,6 @@ public class ErrorHandlingTaskTest { OffsetStorageWriter offsetWriter; @Mock private ConnectorOffsetBackingStore offsetStore; - - private Capture rebalanceListener = EasyMock.newCapture(); @SuppressWarnings("unused") @Mock private TaskStatus.Listener statusListener; @@ -179,7 +183,7 @@ public class ErrorHandlingTaskTest { private boolean enableTopicCreation; - @ParameterizedTest.Parameters + @Parameterized.Parameters public static Collection parameters() { return Arrays.asList(false, true); } @@ -197,7 +201,6 @@ public void setup() { workerProps.put("value.converter", "org.apache.kafka.connect.json.JsonConverter"); workerProps.put("offset.storage.file.filename", "/tmp/connect.offsets"); workerProps.put(TOPIC_CREATION_ENABLE_CONFIG, String.valueOf(enableTopicCreation)); - pluginLoader = PowerMock.createMock(PluginClassLoader.class); workerConfig = new StandaloneConfig(workerProps); sourceConfig = new SourceConnectorConfig(plugins, sourceConnectorProps(TOPIC), true); errorHandlingMetrics = new ErrorHandlingMetrics(taskId, metrics); @@ -228,81 +231,58 @@ public void tearDown() { @Test public void testSinkTasksCloseErrorReporters() throws Exception { - ErrorReporter reporter = EasyMock.mock(ErrorReporter.class); + ErrorReporter reporter = mock(ErrorReporter.class); RetryWithToleranceOperator retryWithToleranceOperator = operator(); retryWithToleranceOperator.reporters(singletonList(reporter)); createSinkTask(initialState, retryWithToleranceOperator); - - expectInitializeTask(); - reporter.close(); - EasyMock.expectLastCall(); - sinkTask.stop(); - EasyMock.expectLastCall(); - - consumer.close(); - EasyMock.expectLastCall(); - - headerConverter.close(); - EasyMock.expectLastCall(); - - PowerMock.replayAll(); - workerSinkTask.initialize(TASK_CONFIG); workerSinkTask.initializeAndStart(); workerSinkTask.close(); - - PowerMock.verifyAll(); + // verify if invocation happened exactly 1 time + verifyInitializeSink(); + verify(reporter).close(); + verify(sinkTask).stop(); + verify(consumer).close(); + verify(headerConverter).close(); } @Test - public void testSourceTasksCloseErrorReporters() { - ErrorReporter reporter = EasyMock.mock(ErrorReporter.class); + public void testSourceTasksCloseErrorReporters() throws IOException { + ErrorReporter reporter = mock(ErrorReporter.class); RetryWithToleranceOperator retryWithToleranceOperator = operator(); retryWithToleranceOperator.reporters(singletonList(reporter)); createSourceTask(initialState, retryWithToleranceOperator); - expectClose(); - - reporter.close(); - EasyMock.expectLastCall(); - - PowerMock.replayAll(); - workerSourceTask.initialize(TASK_CONFIG); workerSourceTask.close(); - - PowerMock.verifyAll(); + verifyCloseSource(); + verify(reporter).close(); } @Test - public void testCloseErrorReportersExceptionPropagation() { - ErrorReporter reporterA = EasyMock.mock(ErrorReporter.class); - ErrorReporter reporterB = EasyMock.mock(ErrorReporter.class); + public void testCloseErrorReportersExceptionPropagation() throws IOException { + ErrorReporter reporterA = mock(ErrorReporter.class); + ErrorReporter reporterB = mock(ErrorReporter.class); RetryWithToleranceOperator retryWithToleranceOperator = operator(); retryWithToleranceOperator.reporters(Arrays.asList(reporterA, reporterB)); createSourceTask(initialState, retryWithToleranceOperator); - expectClose(); - // Even though the reporters throw exceptions, they should both still be closed. - reporterA.close(); - EasyMock.expectLastCall().andThrow(new RuntimeException()); - - reporterB.close(); - EasyMock.expectLastCall().andThrow(new RuntimeException()); - - PowerMock.replayAll(); + doThrow(new RuntimeException()).when(reporterA).close(); + doThrow(new RuntimeException()).when(reporterB).close(); workerSourceTask.initialize(TASK_CONFIG); workerSourceTask.close(); - PowerMock.verifyAll(); + verify(reporterA).close(); + verify(reporterB).close(); + verifyCloseSource(); } @Test @@ -316,21 +296,19 @@ public void testErrorHandlingInSinkTasks() throws Exception { retryWithToleranceOperator.reporters(singletonList(reporter)); createSinkTask(initialState, retryWithToleranceOperator); - expectInitializeTask(); - expectTaskGetTopic(true); // valid json - ConsumerRecord record1 = new ConsumerRecord<>(TOPIC, PARTITION1, FIRST_OFFSET, null, "{\"a\": 10}".getBytes()); + ConsumerRecord record1 = new ConsumerRecord<>( + TOPIC, PARTITION1, FIRST_OFFSET, + null, "{\"a\": 10}".getBytes()); // bad json - ConsumerRecord record2 = new ConsumerRecord<>(TOPIC, PARTITION2, FIRST_OFFSET, null, "{\"a\" 10}".getBytes()); + ConsumerRecord record2 = new ConsumerRecord<>( + TOPIC, PARTITION2, FIRST_OFFSET, + null, "{\"a\" 10}".getBytes()); - EasyMock.expect(consumer.poll(Duration.ofMillis(EasyMock.anyLong()))).andReturn(records(record1)); - EasyMock.expect(consumer.poll(Duration.ofMillis(EasyMock.anyLong()))).andReturn(records(record2)); - - sinkTask.put(EasyMock.anyObject()); - EasyMock.expectLastCall().times(2); - - PowerMock.replayAll(); + when(consumer.poll(any())) + .thenReturn(records(record1)) + .thenReturn(records(record2)); workerSinkTask.initialize(TASK_CONFIG); workerSinkTask.initializeAndStart(); @@ -338,6 +316,9 @@ public void testErrorHandlingInSinkTasks() throws Exception { workerSinkTask.iteration(); + verifyInitializeSink(); + verify(sinkTask, times(2)).put(any()); + // two records were consumed from Kafka assertSinkMetricValue("sink-record-read-total", 2.0); // only one was written to the task @@ -348,12 +329,12 @@ public void testErrorHandlingInSinkTasks() throws Exception { assertErrorHandlingMetricValue("total-record-failures", 3.0); // one record completely failed (converter issues), and thus was skipped assertErrorHandlingMetricValue("total-records-skipped", 1.0); - - PowerMock.verifyAll(); } private RetryWithToleranceOperator operator() { - return new RetryWithToleranceOperator(OPERATOR_RETRY_TIMEOUT_MILLIS, OPERATOR_RETRY_MAX_DELAY_MILLIS, OPERATOR_TOLERANCE_TYPE, SYSTEM, errorHandlingMetrics); + return new RetryWithToleranceOperator(OPERATOR_RETRY_TIMEOUT_MILLIS, + OPERATOR_RETRY_MAX_DELAY_MILLIS, OPERATOR_TOLERANCE_TYPE, + SYSTEM, errorHandlingMetrics); } @Test @@ -374,30 +355,29 @@ public void testErrorHandlingInSourceTasks() throws Exception { Struct struct2 = new Struct(valSchema).put("val", 6789); SourceRecord record2 = new SourceRecord(emptyMap(), emptyMap(), TOPIC, PARTITION1, valSchema, struct2); - EasyMock.expect(workerSourceTask.isStopping()).andReturn(false); - EasyMock.expect(workerSourceTask.isStopping()).andReturn(false); - EasyMock.expect(workerSourceTask.isStopping()).andReturn(true); + when(workerSourceTask.isStopping()) + .thenReturn(false) + .thenReturn(false) + .thenReturn(true); - EasyMock.expect(workerSourceTask.commitOffsets()).andReturn(true); + doReturn(true).when(workerSourceTask).commitOffsets(); - offsetStore.start(); - EasyMock.expectLastCall(); - sourceTask.initialize(EasyMock.anyObject()); - EasyMock.expectLastCall(); - sourceTask.start(EasyMock.anyObject()); - EasyMock.expectLastCall(); + when(sourceTask.poll()) + .thenReturn(singletonList(record1)) + .thenReturn(singletonList(record2)); - EasyMock.expect(sourceTask.poll()).andReturn(singletonList(record1)); - EasyMock.expect(sourceTask.poll()).andReturn(singletonList(record2)); expectTopicCreation(TOPIC); - EasyMock.expect(producer.send(EasyMock.anyObject(), EasyMock.anyObject())).andReturn(null).times(2); - - PowerMock.replayAll(); workerSourceTask.initialize(TASK_CONFIG); workerSourceTask.initializeAndStart(); workerSourceTask.execute(); - + verify(workerSourceTask, times(3)).isStopping(); + verify(workerSourceTask).commitOffsets(); + verify(offsetStore).start(); + verify(sourceTask).initialize(any()); + verify(sourceTask).start(any()); + verify(sourceTask, times(2)).poll(); + verify(producer, times(2)).send(any(), any()); // two records were consumed from Kafka assertSourceMetricValue("source-record-poll-total", 2.0); // only one was written to the task @@ -408,8 +388,6 @@ public void testErrorHandlingInSourceTasks() throws Exception { assertErrorHandlingMetricValue("total-record-failures", 4.0); // one record completely failed (converter issues), and thus was skipped assertErrorHandlingMetricValue("total-records-skipped", 0.0); - - PowerMock.verifyAll(); } private ConnectorConfig connConfig(Map connProps) { @@ -438,30 +416,20 @@ public void testErrorHandlingInSourceTasksWthBadConverter() throws Exception { Struct struct2 = new Struct(valSchema).put("val", 6789); SourceRecord record2 = new SourceRecord(emptyMap(), emptyMap(), TOPIC, PARTITION1, valSchema, struct2); - EasyMock.expect(workerSourceTask.isStopping()).andReturn(false); - EasyMock.expect(workerSourceTask.isStopping()).andReturn(false); - EasyMock.expect(workerSourceTask.isStopping()).andReturn(true); + when(workerSourceTask.isStopping()) + .thenReturn(false) + .thenReturn(false) + .thenReturn(true); - EasyMock.expect(workerSourceTask.commitOffsets()).andReturn(true); + doReturn(true).when(workerSourceTask).commitOffsets(); - offsetStore.start(); - EasyMock.expectLastCall(); - sourceTask.initialize(EasyMock.anyObject()); - EasyMock.expectLastCall(); - sourceTask.start(EasyMock.anyObject()); - EasyMock.expectLastCall(); - - EasyMock.expect(sourceTask.poll()).andReturn(singletonList(record1)); - EasyMock.expect(sourceTask.poll()).andReturn(singletonList(record2)); + when(sourceTask.poll()) + .thenReturn(singletonList(record1)) + .thenReturn(singletonList(record2)); expectTopicCreation(TOPIC); - EasyMock.expect(producer.send(EasyMock.anyObject(), EasyMock.anyObject())).andReturn(null).times(2); - - PowerMock.replayAll(); - workerSourceTask.initialize(TASK_CONFIG); workerSourceTask.initializeAndStart(); workerSourceTask.execute(); - // two records were consumed from Kafka assertSourceMetricValue("source-record-poll-total", 2.0); // only one was written to the task @@ -473,7 +441,13 @@ public void testErrorHandlingInSourceTasksWthBadConverter() throws Exception { // one record completely failed (converter issues), and thus was skipped assertErrorHandlingMetricValue("total-records-skipped", 0.0); - PowerMock.verifyAll(); + verify(workerSourceTask, times(3)).isStopping(); + verify(workerSourceTask).commitOffsets(); + verify(offsetStore).start(); + verify(sourceTask).initialize(any()); + verify(sourceTask).start(any()); + verify(sourceTask, times(2)).poll(); + verify(producer, times(2)).send(any(), any()); } private void assertSinkMetricValue(String name, double expected) { @@ -482,6 +456,13 @@ private void assertSinkMetricValue(String name, double expected) { assertEquals(expected, measured, 0.001d); } + private void verifyInitializeSink() { + verify(sinkTask).start(TASK_PROPS); + verify(sinkTask).initialize(any(WorkerSinkTaskContext.class)); + verify(consumer).subscribe(eq(singletonList(TOPIC)), + any(ConsumerRebalanceListener.class)); + } + private void assertSourceMetricValue(String name, double expected) { ConnectMetrics.MetricGroup sinkTaskGroup = workerSourceTask.sourceTaskMetricsGroup().metricGroup(); double measured = metrics.currentMetricValueAsDouble(sinkTaskGroup, name); @@ -494,73 +475,22 @@ private void assertErrorHandlingMetricValue(String name, double expected) { assertEquals(expected, measured, 0.001d); } - private void expectInitializeTask() { - consumer.subscribe(EasyMock.eq(singletonList(TOPIC)), EasyMock.capture(rebalanceListener)); - PowerMock.expectLastCall(); - - sinkTask.initialize(EasyMock.capture(sinkTaskContext)); - PowerMock.expectLastCall(); - sinkTask.start(TASK_PROPS); - PowerMock.expectLastCall(); - } - - private void expectTaskGetTopic(boolean anyTimes) { - final Capture connectorCapture = EasyMock.newCapture(); - final Capture topicCapture = EasyMock.newCapture(); - IExpectationSetters expect = EasyMock.expect(statusBackingStore.getTopic( - EasyMock.capture(connectorCapture), - EasyMock.capture(topicCapture))); - if (anyTimes) { - expect.andStubAnswer(() -> new TopicStatus( - topicCapture.getValue(), - new ConnectorTaskId(connectorCapture.getValue(), 0), - Time.SYSTEM.milliseconds())); - } else { - expect.andAnswer(() -> new TopicStatus( - topicCapture.getValue(), - new ConnectorTaskId(connectorCapture.getValue(), 0), - Time.SYSTEM.milliseconds())); - } - if (connectorCapture.hasCaptured() && topicCapture.hasCaptured()) { - assertEquals("job", connectorCapture.getValue()); - assertEquals(TOPIC, topicCapture.getValue()); - } - } - - private void expectClose() { - producer.close(EasyMock.anyObject(Duration.class)); - EasyMock.expectLastCall(); - - admin.close(EasyMock.anyObject(Duration.class)); - EasyMock.expectLastCall(); - - offsetReader.close(); - EasyMock.expectLastCall(); - - offsetStore.stop(); - EasyMock.expectLastCall(); - - try { - headerConverter.close(); - } catch (IOException e) { - throw new RuntimeException(e); - } - EasyMock.expectLastCall(); + private void verifyCloseSource() throws IOException { + verify(producer).close(any(Duration.class)); + verify(admin).close(any(Duration.class)); + verify(offsetReader).close(); + verify(offsetStore).stop(); + // headerConverter.close() can throw IOException + verify(headerConverter).close(); } private void expectTopicCreation(String topic) { - if (workerConfig.topicCreationEnable()) { - EasyMock.expect(admin.describeTopics(topic)).andReturn(Collections.emptyMap()); - Capture newTopicCapture = EasyMock.newCapture(); - - if (enableTopicCreation) { - Set created = Collections.singleton(topic); - Set existing = Collections.emptySet(); - TopicAdmin.TopicCreationResponse response = new TopicAdmin.TopicCreationResponse(created, existing); - EasyMock.expect(admin.createOrFindTopics(EasyMock.capture(newTopicCapture))).andReturn(response); - } else { - EasyMock.expect(admin.createTopic(EasyMock.capture(newTopicCapture))).andReturn(true); - } + if (enableTopicCreation) { + when(admin.describeTopics(topic)).thenReturn(Collections.emptyMap()); + Set created = Collections.singleton(topic); + Set existing = Collections.emptySet(); + TopicAdmin.TopicCreationResponse response = new TopicAdmin.TopicCreationResponse(created, existing); + when(admin.createOrFindTopics(any(NewTopic.class))).thenReturn(response); } } @@ -571,13 +501,15 @@ private void createSinkTask(TargetState initialState, RetryWithToleranceOperator oo.put("schemas.enable", "false"); converter.configure(oo); - TransformationChain sinkTransforms = new TransformationChain<>(singletonList(new FaultyPassthrough()), retryWithToleranceOperator); + TransformationChain sinkTransforms = + new TransformationChain<>(singletonList(new FaultyPassthrough()), retryWithToleranceOperator); workerSinkTask = new WorkerSinkTask( taskId, sinkTask, statusListener, initialState, workerConfig, ClusterConfigState.EMPTY, metrics, converter, converter, errorHandlingMetrics, headerConverter, sinkTransforms, consumer, pluginLoader, time, - retryWithToleranceOperator, workerErrantRecordReporter, statusBackingStore); + retryWithToleranceOperator, workerErrantRecordReporter, + statusBackingStore); } private void createSourceTask(TargetState initialState, RetryWithToleranceOperator retryWithToleranceOperator) { @@ -602,13 +534,16 @@ private Converter badConverter() { private void createSourceTask(TargetState initialState, RetryWithToleranceOperator retryWithToleranceOperator, Converter converter) { TransformationChain sourceTransforms = new TransformationChain<>(singletonList(new FaultyPassthrough()), retryWithToleranceOperator); - workerSourceTask = PowerMock.createPartialMock( - WorkerSourceTask.class, new String[]{"commitOffsets", "isStopping"}, - taskId, sourceTask, statusListener, initialState, converter, converter, errorHandlingMetrics, headerConverter, sourceTransforms, - producer, admin, TopicCreationGroup.configuredGroups(sourceConfig), - offsetReader, offsetWriter, offsetStore, workerConfig, - ClusterConfigState.EMPTY, metrics, pluginLoader, time, retryWithToleranceOperator, - statusBackingStore, (Executor) Runnable::run); + workerSourceTask = spy(new WorkerSourceTask( + taskId, sourceTask, statusListener, initialState, converter, + converter, errorHandlingMetrics, headerConverter, + sourceTransforms, producer, admin, + TopicCreationGroup.configuredGroups(sourceConfig), + offsetReader, offsetWriter, offsetStore, workerConfig, + ClusterConfigState.EMPTY, metrics, pluginLoader, time, + retryWithToleranceOperator, + statusBackingStore, (Executor) Runnable::run)); + } private ConsumerRecords records(ConsumerRecord record) { diff --git a/core/src/main/java/kafka/server/builders/LogManagerBuilder.java b/core/src/main/java/kafka/server/builders/LogManagerBuilder.java index de43d55203f56..2e5e293b120f7 100644 --- a/core/src/main/java/kafka/server/builders/LogManagerBuilder.java +++ b/core/src/main/java/kafka/server/builders/LogManagerBuilder.java @@ -22,11 +22,11 @@ import kafka.log.LogManager; import kafka.log.ProducerStateManagerConfig; import kafka.server.BrokerTopicStats; -import kafka.server.LogDirFailureChannel; import kafka.server.metadata.ConfigRepository; import kafka.utils.Scheduler; import org.apache.kafka.common.utils.Time; import org.apache.kafka.server.common.MetadataVersion; +import org.apache.kafka.server.log.internals.LogDirFailureChannel; import scala.collection.JavaConverters; import java.io.File; diff --git a/core/src/main/java/kafka/server/builders/ReplicaManagerBuilder.java b/core/src/main/java/kafka/server/builders/ReplicaManagerBuilder.java index e6b46f41c7e51..bc7ebd8a2e72c 100644 --- a/core/src/main/java/kafka/server/builders/ReplicaManagerBuilder.java +++ b/core/src/main/java/kafka/server/builders/ReplicaManagerBuilder.java @@ -27,7 +27,6 @@ import kafka.server.DelayedOperationPurgatory; import kafka.server.DelayedProduce; import kafka.server.KafkaConfig; -import kafka.server.LogDirFailureChannel; import kafka.server.MetadataCache; import kafka.server.QuotaFactory.QuotaManagers; import kafka.server.ReplicaManager; @@ -35,6 +34,7 @@ import kafka.zk.KafkaZkClient; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.utils.Time; +import org.apache.kafka.server.log.internals.LogDirFailureChannel; import scala.compat.java8.OptionConverters; import java.util.Collections; diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala index 0fe584fa6f681..c4df79219a40f 100755 --- a/core/src/main/scala/kafka/cluster/Partition.scala +++ b/core/src/main/scala/kafka/cluster/Partition.scala @@ -44,6 +44,7 @@ import org.apache.kafka.common.utils.Time import org.apache.kafka.common.{IsolationLevel, TopicPartition, Uuid} import org.apache.kafka.metadata.LeaderRecoveryState import org.apache.kafka.server.common.MetadataVersion +import org.apache.kafka.server.log.internals.AppendOrigin import scala.collection.{Map, Seq} import scala.jdk.CollectionConverters._ diff --git a/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala b/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala index daf64bb35f7fa..0b908d2c89cc6 100644 --- a/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala +++ b/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala @@ -24,11 +24,9 @@ import java.util.Optional import java.util.concurrent.atomic.AtomicBoolean import java.util.concurrent.locks.ReentrantLock import java.util.concurrent.{ConcurrentHashMap, TimeUnit} - import com.yammer.metrics.core.Gauge import kafka.common.OffsetAndMetadata import kafka.internals.generated.{GroupMetadataValue, OffsetCommitKey, OffsetCommitValue, GroupMetadataKey => GroupMetadataKeyData} -import kafka.log.AppendOrigin import kafka.metrics.KafkaMetricsGroup import kafka.server.{FetchLogEnd, ReplicaManager, RequestLocal} import kafka.utils.CoreUtils.inLock @@ -48,6 +46,7 @@ import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.common.{KafkaException, MessageFormatter, TopicPartition} import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.server.common.MetadataVersion.{IBP_0_10_1_IV0, IBP_2_1_IV0, IBP_2_1_IV1, IBP_2_3_IV0} +import org.apache.kafka.server.log.internals.AppendOrigin import scala.collection._ import scala.collection.mutable.ArrayBuffer @@ -331,7 +330,7 @@ class GroupMetadataManager(brokerId: Int, timeout = config.offsetCommitTimeoutMs.toLong, requiredAcks = config.offsetCommitRequiredAcks, internalTopicsAllowed = true, - origin = AppendOrigin.Coordinator, + origin = AppendOrigin.COORDINATOR, entriesPerPartition = records, delayedProduceLock = Some(group.lock), responseCallback = callback, @@ -890,7 +889,7 @@ class GroupMetadataManager(brokerId: Int, // do not need to require acks since even if the tombstone is lost, // it will be appended again in the next purge cycle val records = MemoryRecords.withRecords(magicValue, 0L, compressionType, timestampType, tombstones.toArray: _*) - partition.appendRecordsToLeader(records, origin = AppendOrigin.Coordinator, requiredAcks = 0, + partition.appendRecordsToLeader(records, origin = AppendOrigin.COORDINATOR, requiredAcks = 0, requestLocal = requestLocal) offsetsRemoved += removedOffsets.size diff --git a/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala b/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala index ad721ccc38057..9073d491abc28 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala @@ -16,12 +16,13 @@ */ package kafka.coordinator.transaction +import kafka.log.LogConfig + import java.nio.ByteBuffer import java.util.Properties import java.util.concurrent.TimeUnit import java.util.concurrent.atomic.AtomicBoolean import java.util.concurrent.locks.ReentrantReadWriteLock -import kafka.log.{AppendOrigin, LogConfig} import kafka.server.{Defaults, FetchLogEnd, ReplicaManager, RequestLocal} import kafka.utils.CoreUtils.{inReadLock, inWriteLock} import kafka.utils.{Logging, Pool, Scheduler} @@ -36,6 +37,7 @@ import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse import org.apache.kafka.common.requests.TransactionResult import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.common.{KafkaException, TopicPartition} +import org.apache.kafka.server.log.internals.AppendOrigin import org.apache.kafka.server.record.BrokerCompressionType import scala.jdk.CollectionConverters._ @@ -282,7 +284,7 @@ class TransactionStateManager(brokerId: Int, config.requestTimeoutMs, TransactionLog.EnforcedRequiredAcks, internalTopicsAllowed = true, - origin = AppendOrigin.Coordinator, + origin = AppendOrigin.COORDINATOR, entriesPerPartition = Map(transactionPartition -> tombstoneRecords), removeFromCacheCallback, requestLocal = RequestLocal.NoCaching) @@ -761,7 +763,7 @@ class TransactionStateManager(brokerId: Int, newMetadata.txnTimeoutMs.toLong, TransactionLog.EnforcedRequiredAcks, internalTopicsAllowed = true, - origin = AppendOrigin.Coordinator, + origin = AppendOrigin.COORDINATOR, recordsPerPartition, updateCacheCallback, requestLocal = requestLocal) diff --git a/core/src/main/scala/kafka/log/LazyIndex.scala b/core/src/main/scala/kafka/log/LazyIndex.scala deleted file mode 100644 index 6725d034a1bee..0000000000000 --- a/core/src/main/scala/kafka/log/LazyIndex.scala +++ /dev/null @@ -1,166 +0,0 @@ -/** - * 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.log - -import java.io.File -import java.nio.file.{Files, NoSuchFileException} -import java.util.concurrent.locks.ReentrantLock -import LazyIndex._ -import kafka.utils.CoreUtils.inLock -import kafka.utils.threadsafe -import org.apache.kafka.common.utils.Utils -import org.apache.kafka.server.log.internals.{AbstractIndex, OffsetIndex} - -/** - * A wrapper over an `AbstractIndex` instance that provides a mechanism to defer loading - * (i.e. memory mapping) the underlying index until it is accessed for the first time via the - * `get` method. - * - * In addition, this class exposes a number of methods (e.g. updateParentDir, renameTo, close, - * etc.) that provide the desired behavior without causing the index to be loaded. If the index - * had previously been loaded, the methods in this class simply delegate to the relevant method in - * the index. - * - * This is an important optimization with regards to broker start-up and shutdown time if it has a - * large number of segments. - * - * Methods of this class are thread safe. Make sure to check `AbstractIndex` subclasses - * documentation to establish their thread safety. - * - * @param loadIndex A function that takes a `File` pointing to an index and returns a loaded - * `AbstractIndex` instance. - */ -@threadsafe -class LazyIndex[T <: AbstractIndex] private (@volatile private var indexWrapper: IndexWrapper, loadIndex: File => T) { - - private val lock = new ReentrantLock() - - def file: File = indexWrapper.file - - def get: T = { - indexWrapper match { - case indexValue: IndexValue[_] => indexValue.index.asInstanceOf[T] - case _: IndexFile => - inLock(lock) { - indexWrapper match { - case indexValue: IndexValue[_] => indexValue.index.asInstanceOf[T] - case indexFile: IndexFile => - val indexValue = new IndexValue(loadIndex(indexFile.file)) - indexWrapper = indexValue - indexValue.index - } - } - } - } - - def updateParentDir(parentDir: File): Unit = { - inLock(lock) { - indexWrapper.updateParentDir(parentDir) - } - } - - def renameTo(f: File): Unit = { - inLock(lock) { - indexWrapper.renameTo(f) - } - } - - def deleteIfExists(): Boolean = { - inLock(lock) { - indexWrapper.deleteIfExists() - } - } - - def close(): Unit = { - inLock(lock) { - indexWrapper.close() - } - } - - def closeHandler(): Unit = { - inLock(lock) { - indexWrapper.closeHandler() - } - } - -} - -object LazyIndex { - - def forOffset(file: File, baseOffset: Long, maxIndexSize: Int = -1, writable: Boolean = true): LazyIndex[OffsetIndex] = - new LazyIndex(new IndexFile(file), file => new OffsetIndex(file, baseOffset, maxIndexSize, writable)) - - def forTime(file: File, baseOffset: Long, maxIndexSize: Int = -1, writable: Boolean = true): LazyIndex[TimeIndex] = - new LazyIndex(new IndexFile(file), file => new TimeIndex(file, baseOffset, maxIndexSize, writable)) - - private sealed trait IndexWrapper { - - def file: File - - def updateParentDir(f: File): Unit - - def renameTo(f: File): Unit - - def deleteIfExists(): Boolean - - def close(): Unit - - def closeHandler(): Unit - - } - - private class IndexFile(@volatile private var _file: File) extends IndexWrapper { - - def file: File = _file - - def updateParentDir(parentDir: File): Unit = _file = new File(parentDir, file.getName) - - def renameTo(f: File): Unit = { - try Utils.atomicMoveWithFallback(file.toPath, f.toPath, false) - catch { - case _: NoSuchFileException if !file.exists => () - } - finally _file = f - } - - def deleteIfExists(): Boolean = Files.deleteIfExists(file.toPath) - - def close(): Unit = () - - def closeHandler(): Unit = () - - } - - private class IndexValue[T <: AbstractIndex](val index: T) extends IndexWrapper { - - def file: File = index.file - - def updateParentDir(parentDir: File): Unit = index.updateParentDir(parentDir) - - def renameTo(f: File): Unit = index.renameTo(f) - - def deleteIfExists(): Boolean = index.deleteIfExists() - - def close(): Unit = index.close() - - def closeHandler(): Unit = index.closeHandler() - - } - -} - diff --git a/core/src/main/scala/kafka/log/LocalLog.scala b/core/src/main/scala/kafka/log/LocalLog.scala index 7d2524091a671..10d5217069b2c 100644 --- a/core/src/main/scala/kafka/log/LocalLog.scala +++ b/core/src/main/scala/kafka/log/LocalLog.scala @@ -23,14 +23,14 @@ import java.text.NumberFormat import java.util.concurrent.atomic.AtomicLong import java.util.regex.Pattern import kafka.metrics.KafkaMetricsGroup -import kafka.server.{FetchDataInfo, LogDirFailureChannel, LogOffsetMetadata} +import kafka.server.{FetchDataInfo, LogOffsetMetadata} import kafka.utils.{Logging, Scheduler} import org.apache.kafka.common.{KafkaException, TopicPartition} import org.apache.kafka.common.errors.{KafkaStorageException, OffsetOutOfRangeException} import org.apache.kafka.common.message.FetchResponseData import org.apache.kafka.common.record.MemoryRecords import org.apache.kafka.common.utils.{Time, Utils} -import org.apache.kafka.server.log.internals.{AbortedTxn, OffsetPosition} +import org.apache.kafka.server.log.internals.{AbortedTxn, LogDirFailureChannel, OffsetPosition} import scala.jdk.CollectionConverters._ import scala.collection.{Seq, immutable} diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala index 8bafc0aae60f1..423ff8866674a 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -23,7 +23,7 @@ import java.util.Date import java.util.concurrent.TimeUnit import kafka.common._ import kafka.metrics.KafkaMetricsGroup -import kafka.server.{BrokerReconfigurable, KafkaConfig, LogDirFailureChannel} +import kafka.server.{BrokerReconfigurable, KafkaConfig} import kafka.utils._ import org.apache.kafka.common.{KafkaException, TopicPartition} import org.apache.kafka.common.config.ConfigException @@ -32,7 +32,7 @@ import org.apache.kafka.common.record.MemoryRecords.RecordFilter import org.apache.kafka.common.record.MemoryRecords.RecordFilter.BatchRetention import org.apache.kafka.common.record._ import org.apache.kafka.common.utils.{BufferSupplier, Time} -import org.apache.kafka.server.log.internals.{AbortedTxn, TransactionIndex} +import org.apache.kafka.server.log.internals.{AbortedTxn, LogDirFailureChannel, OffsetMap, SkimpyOffsetMap, TransactionIndex} import scala.jdk.CollectionConverters._ import scala.collection.mutable.ListBuffer @@ -307,8 +307,8 @@ class LogCleaner(initialConfig: CleanerConfig, warn("Cannot use more than 2G of cleaner buffer space per cleaner thread, ignoring excess buffer space...") val cleaner = new Cleaner(id = threadId, - offsetMap = new SkimpyOffsetMap(memory = math.min(config.dedupeBufferSize / config.numThreads, Int.MaxValue).toInt, - hashAlgorithm = config.hashAlgorithm), + offsetMap = new SkimpyOffsetMap(math.min(config.dedupeBufferSize / config.numThreads, Int.MaxValue).toInt, + config.hashAlgorithm), ioBufferSize = config.ioBufferSize / config.numThreads / 2, maxIoBufferSize = config.maxMessageSize, dupBufferLoadFactor = config.dedupeBufferLoadFactor, @@ -789,7 +789,7 @@ private[log] class Cleaner(val id: Int, transactionMetadata.onBatchRead(batch) } - private def shouldRetainRecord(map: kafka.log.OffsetMap, + private def shouldRetainRecord(map: OffsetMap, retainDeletesForLegacyRecords: Boolean, batch: RecordBatch, record: Record, diff --git a/core/src/main/scala/kafka/log/LogCleanerManager.scala b/core/src/main/scala/kafka/log/LogCleanerManager.scala index 48f4d49b6d621..14f55cc051867 100755 --- a/core/src/main/scala/kafka/log/LogCleanerManager.scala +++ b/core/src/main/scala/kafka/log/LogCleanerManager.scala @@ -23,13 +23,13 @@ import java.util.concurrent.locks.ReentrantLock import kafka.common.LogCleaningAbortedException import kafka.metrics.KafkaMetricsGroup -import kafka.server.LogDirFailureChannel import kafka.server.checkpoints.OffsetCheckpointFile import kafka.utils.CoreUtils._ import kafka.utils.{Logging, Pool} import org.apache.kafka.common.{KafkaException, TopicPartition} import org.apache.kafka.common.errors.KafkaStorageException import org.apache.kafka.common.utils.Time +import org.apache.kafka.server.log.internals.LogDirFailureChannel import scala.collection.{Iterable, Seq, mutable} diff --git a/core/src/main/scala/kafka/log/LogLoader.scala b/core/src/main/scala/kafka/log/LogLoader.scala index 2eb055ba167d5..e69283f71a649 100644 --- a/core/src/main/scala/kafka/log/LogLoader.scala +++ b/core/src/main/scala/kafka/log/LogLoader.scala @@ -21,14 +21,14 @@ import java.io.{File, IOException} import java.nio.file.{Files, NoSuchFileException} import kafka.common.LogSegmentOffsetOverflowException import kafka.log.UnifiedLog.{CleanedFileSuffix, DeletedFileSuffix, SwapFileSuffix, isIndexFile, isLogFile, offsetFromFile} -import kafka.server.{LogDirFailureChannel, LogOffsetMetadata} +import kafka.server.LogOffsetMetadata import kafka.server.epoch.LeaderEpochFileCache import kafka.utils.{CoreUtils, Logging, Scheduler} import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.errors.InvalidOffsetException import org.apache.kafka.common.utils.Time import org.apache.kafka.snapshot.Snapshots -import org.apache.kafka.server.log.internals.CorruptIndexException +import org.apache.kafka.server.log.internals.{CorruptIndexException, LogDirFailureChannel} import java.util.concurrent.{ConcurrentHashMap, ConcurrentMap} import scala.collection.{Set, mutable} diff --git a/core/src/main/scala/kafka/log/LogManager.scala b/core/src/main/scala/kafka/log/LogManager.scala index dc599b544ae03..2f6bf52f67e40 100755 --- a/core/src/main/scala/kafka/log/LogManager.scala +++ b/core/src/main/scala/kafka/log/LogManager.scala @@ -42,6 +42,7 @@ import kafka.utils.Implicits._ import java.util.Properties import org.apache.kafka.server.common.MetadataVersion +import org.apache.kafka.server.log.internals.LogDirFailureChannel import scala.annotation.nowarn @@ -1080,6 +1081,9 @@ class LogManager(logDirs: Seq[File], throw new KafkaStorageException(s"The future replica for $topicPartition is offline") destLog.renameDir(UnifiedLog.logDirName(topicPartition), true) + // the metrics tags still contain "future", so we have to remove it. + // we will add metrics back after sourceLog remove the metrics + destLog.removeLogMetrics() destLog.updateHighWatermark(sourceLog.highWatermark) // Now that future replica has been successfully renamed to be the current replica @@ -1101,6 +1105,7 @@ class LogManager(logDirs: Seq[File], checkpointRecoveryOffsetsInDir(logDir, logsToCheckpoint) checkpointLogStartOffsetsInDir(logDir, logsToCheckpoint) sourceLog.removeLogMetrics() + destLog.newMetrics() addLogToBeDeleted(sourceLog) } catch { case e: KafkaStorageException => diff --git a/core/src/main/scala/kafka/log/LogSegment.scala b/core/src/main/scala/kafka/log/LogSegment.scala index fbe50f8225756..93677bee65385 100755 --- a/core/src/main/scala/kafka/log/LogSegment.scala +++ b/core/src/main/scala/kafka/log/LogSegment.scala @@ -32,7 +32,7 @@ import org.apache.kafka.common.errors.CorruptRecordException import org.apache.kafka.common.record.FileRecords.{LogOffsetPosition, TimestampAndOffset} import org.apache.kafka.common.record._ import org.apache.kafka.common.utils.{BufferSupplier, Time} -import org.apache.kafka.server.log.internals.{AbortedTxn, CompletedTxn, OffsetIndex, OffsetPosition, TimestampOffset, TransactionIndex, TxnIndexSearchResult} +import org.apache.kafka.server.log.internals.{AbortedTxn, AppendOrigin, CompletedTxn, LazyIndex, OffsetIndex, OffsetPosition, TimeIndex, TimestampOffset, TransactionIndex, TxnIndexSearchResult} import java.util.Optional import scala.jdk.CollectionConverters._ @@ -248,7 +248,7 @@ class LogSegment private[log] (val log: FileRecords, private def updateProducerState(producerStateManager: ProducerStateManager, batch: RecordBatch): Unit = { if (batch.hasProducerId) { val producerId = batch.producerId - val appendInfo = producerStateManager.prepareUpdate(producerId, origin = AppendOrigin.Replication) + val appendInfo = producerStateManager.prepareUpdate(producerId, origin = AppendOrigin.REPLICATION) val maybeCompletedTxn = appendInfo.append(batch, firstOffsetMetadataOpt = None) producerStateManager.update(appendInfo) maybeCompletedTxn.foreach { completedTxn => @@ -381,7 +381,7 @@ class LogSegment private[log] (val log: FileRecords, log.truncateTo(validBytes) offsetIndex.trimToValidSize() // A normally closed segment always appends the biggest timestamp ever seen into log segment, we do this as well. - timeIndex.maybeAppend(maxTimestampSoFar, offsetOfMaxTimestampSoFar, skipFullCheck = true) + timeIndex.maybeAppend(maxTimestampSoFar, offsetOfMaxTimestampSoFar, true) timeIndex.trimToValidSize() truncated } @@ -511,7 +511,7 @@ class LogSegment private[log] (val log: FileRecords, * The time index entry appended will be used to decide when to delete the segment. */ def onBecomeInactiveSegment(): Unit = { - timeIndex.maybeAppend(maxTimestampSoFar, offsetOfMaxTimestampSoFar, skipFullCheck = true) + timeIndex.maybeAppend(maxTimestampSoFar, offsetOfMaxTimestampSoFar, true) offsetIndex.trimToValidSize() timeIndex.trimToValidSize() log.trim() @@ -593,8 +593,7 @@ class LogSegment private[log] (val log: FileRecords, */ def close(): Unit = { if (_maxTimestampAndOffsetSoFar != TimestampOffset.UNKNOWN) - CoreUtils.swallow(timeIndex.maybeAppend(maxTimestampSoFar, offsetOfMaxTimestampSoFar, - skipFullCheck = true), this) + CoreUtils.swallow(timeIndex.maybeAppend(maxTimestampSoFar, offsetOfMaxTimestampSoFar, true), this) CoreUtils.swallow(lazyOffsetIndex.close(), this) CoreUtils.swallow(lazyTimeIndex.close(), this) CoreUtils.swallow(log.close(), this) @@ -673,8 +672,8 @@ object LogSegment { val maxIndexSize = config.maxIndexSize new LogSegment( FileRecords.open(UnifiedLog.logFile(dir, baseOffset, fileSuffix), fileAlreadyExists, initFileSize, preallocate), - LazyIndex.forOffset(UnifiedLog.offsetIndexFile(dir, baseOffset, fileSuffix), baseOffset = baseOffset, maxIndexSize = maxIndexSize), - LazyIndex.forTime(UnifiedLog.timeIndexFile(dir, baseOffset, fileSuffix), baseOffset = baseOffset, maxIndexSize = maxIndexSize), + LazyIndex.forOffset(UnifiedLog.offsetIndexFile(dir, baseOffset, fileSuffix), baseOffset, maxIndexSize, true), + LazyIndex.forTime(UnifiedLog.timeIndexFile(dir, baseOffset, fileSuffix), baseOffset, maxIndexSize, true), new TransactionIndex(baseOffset, UnifiedLog.transactionIndexFile(dir, baseOffset, fileSuffix)), baseOffset, indexIntervalBytes = config.indexInterval, diff --git a/core/src/main/scala/kafka/log/LogValidator.scala b/core/src/main/scala/kafka/log/LogValidator.scala deleted file mode 100644 index e36aceab01d2b..0000000000000 --- a/core/src/main/scala/kafka/log/LogValidator.scala +++ /dev/null @@ -1,592 +0,0 @@ -/** - * 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.log - -import java.nio.ByteBuffer - -import kafka.common.{LongRef, RecordValidationException} -import kafka.server.{BrokerTopicStats, RequestLocal} -import kafka.utils.Logging -import org.apache.kafka.common.errors.{CorruptRecordException, InvalidTimestampException, UnsupportedCompressionTypeException, UnsupportedForMessageFormatException} -import org.apache.kafka.common.record.{AbstractRecords, CompressionType, MemoryRecords, Record, RecordBatch, RecordConversionStats, TimestampType} -import org.apache.kafka.common.InvalidRecordException -import org.apache.kafka.common.TopicPartition -import org.apache.kafka.common.protocol.Errors -import org.apache.kafka.common.requests.ProduceResponse.RecordError -import org.apache.kafka.common.utils.Time -import org.apache.kafka.server.common.MetadataVersion -import org.apache.kafka.server.common.MetadataVersion.IBP_2_1_IV0 - -import scala.collection.{Seq, mutable} -import scala.jdk.CollectionConverters._ -import scala.collection.mutable.ArrayBuffer - -/** - * The source of an append to the log. This is used when determining required validations. - */ -private[kafka] sealed trait AppendOrigin -private[kafka] object AppendOrigin { - - /** - * The log append came through replication from the leader. This typically implies minimal validation. - * Particularly, we do not decompress record batches in order to validate records individually. - */ - case object Replication extends AppendOrigin - - /** - * The log append came from either the group coordinator or the transaction coordinator. We validate - * producer epochs for normal log entries (specifically offset commits from the group coordinator) and - * we validate coordinate end transaction markers from the transaction coordinator. - */ - case object Coordinator extends AppendOrigin - - /** - * The log append came from the client, which implies full validation. - */ - case object Client extends AppendOrigin - - /** - * The log append come from the raft leader, which implies the offsets has been assigned - */ - case object RaftLeader extends AppendOrigin -} - -private[log] object LogValidator extends Logging { - - /** - * Update the offsets for this message set and do further validation on messages including: - * 1. Messages for compacted topics must have keys - * 2. When magic value >= 1, inner messages of a compressed message set must have monotonically increasing offsets - * starting from 0. - * 3. When magic value >= 1, validate and maybe overwrite timestamps of messages. - * 4. Declared count of records in DefaultRecordBatch must match number of valid records contained therein. - * - * This method will convert messages as necessary to the topic's configured message format version. If no format - * conversion or value overwriting is required for messages, this method will perform in-place operations to - * avoid expensive re-compression. - * - * Returns a ValidationAndOffsetAssignResult containing the validated message set, maximum timestamp, the offset - * of the shallow message with the max timestamp and a boolean indicating whether the message sizes may have changed. - */ - private[log] def validateMessagesAndAssignOffsets(records: MemoryRecords, - topicPartition: TopicPartition, - offsetCounter: LongRef, - time: Time, - now: Long, - sourceCompression: CompressionType, - targetCompression: CompressionType, - compactedTopic: Boolean, - magic: Byte, - timestampType: TimestampType, - timestampDiffMaxMs: Long, - partitionLeaderEpoch: Int, - origin: AppendOrigin, - interBrokerProtocolVersion: MetadataVersion, - brokerTopicStats: BrokerTopicStats, - requestLocal: RequestLocal): ValidationAndOffsetAssignResult = { - if (sourceCompression == CompressionType.NONE && targetCompression == CompressionType.NONE) { - // check the magic value - if (!records.hasMatchingMagic(magic)) - convertAndAssignOffsetsNonCompressed(records, topicPartition, offsetCounter, compactedTopic, time, now, timestampType, - timestampDiffMaxMs, magic, partitionLeaderEpoch, origin, brokerTopicStats) - else - // Do in-place validation, offset assignment and maybe set timestamp - assignOffsetsNonCompressed(records, topicPartition, offsetCounter, now, compactedTopic, timestampType, timestampDiffMaxMs, - partitionLeaderEpoch, origin, magic, brokerTopicStats) - } else { - validateMessagesAndAssignOffsetsCompressed(records, topicPartition, offsetCounter, time, now, sourceCompression, - targetCompression, compactedTopic, magic, timestampType, timestampDiffMaxMs, partitionLeaderEpoch, origin, - interBrokerProtocolVersion, brokerTopicStats, requestLocal) - } - } - - private def getFirstBatchAndMaybeValidateNoMoreBatches(records: MemoryRecords, sourceCompression: CompressionType): RecordBatch = { - val batchIterator = records.batches.iterator - - if (!batchIterator.hasNext) { - throw new InvalidRecordException("Record batch has no batches at all") - } - - val batch = batchIterator.next() - - // if the format is v2 and beyond, or if the messages are compressed, we should check there's only one batch. - if (batch.magic() >= RecordBatch.MAGIC_VALUE_V2 || sourceCompression != CompressionType.NONE) { - if (batchIterator.hasNext) { - throw new InvalidRecordException("Compressed outer record has more than one batch") - } - } - - batch - } - - private def validateBatch(topicPartition: TopicPartition, - firstBatch: RecordBatch, - batch: RecordBatch, - origin: AppendOrigin, - toMagic: Byte, - brokerTopicStats: BrokerTopicStats): Unit = { - // batch magic byte should have the same magic as the first batch - if (firstBatch.magic() != batch.magic()) { - brokerTopicStats.allTopicsStats.invalidMagicNumberRecordsPerSec.mark() - throw new InvalidRecordException(s"Batch magic ${batch.magic()} is not the same as the first batch'es magic byte ${firstBatch.magic()} in topic partition $topicPartition.") - } - - if (origin == AppendOrigin.Client) { - if (batch.magic >= RecordBatch.MAGIC_VALUE_V2) { - val countFromOffsets = batch.lastOffset - batch.baseOffset + 1 - if (countFromOffsets <= 0) { - brokerTopicStats.allTopicsStats.invalidOffsetOrSequenceRecordsPerSec.mark() - throw new InvalidRecordException(s"Batch has an invalid offset range: [${batch.baseOffset}, ${batch.lastOffset}] in topic partition $topicPartition.") - } - - // v2 and above messages always have a non-null count - val count = batch.countOrNull - if (count <= 0) { - brokerTopicStats.allTopicsStats.invalidOffsetOrSequenceRecordsPerSec.mark() - throw new InvalidRecordException(s"Invalid reported count for record batch: $count in topic partition $topicPartition.") - } - - if (countFromOffsets != batch.countOrNull) { - brokerTopicStats.allTopicsStats.invalidOffsetOrSequenceRecordsPerSec.mark() - throw new InvalidRecordException(s"Inconsistent batch offset range [${batch.baseOffset}, ${batch.lastOffset}] " + - s"and count of records $count in topic partition $topicPartition.") - } - } - - if (batch.isControlBatch) { - brokerTopicStats.allTopicsStats.invalidOffsetOrSequenceRecordsPerSec.mark() - throw new InvalidRecordException(s"Clients are not allowed to write control records in topic partition $topicPartition.") - } - - if (batch.hasProducerId && batch.baseSequence < 0) { - brokerTopicStats.allTopicsStats.invalidOffsetOrSequenceRecordsPerSec.mark() - throw new InvalidRecordException(s"Invalid sequence number ${batch.baseSequence} in record batch " + - s"with producerId ${batch.producerId} in topic partition $topicPartition.") - } - } - - if (batch.isTransactional && toMagic < RecordBatch.MAGIC_VALUE_V2) - throw new UnsupportedForMessageFormatException(s"Transactional records cannot be used with magic version $toMagic") - - if (batch.hasProducerId && toMagic < RecordBatch.MAGIC_VALUE_V2) - throw new UnsupportedForMessageFormatException(s"Idempotent records cannot be used with magic version $toMagic") - } - - private def validateRecord(batch: RecordBatch, topicPartition: TopicPartition, record: Record, batchIndex: Int, now: Long, - timestampType: TimestampType, timestampDiffMaxMs: Long, compactedTopic: Boolean, - brokerTopicStats: BrokerTopicStats): Option[ApiRecordError] = { - if (!record.hasMagic(batch.magic)) { - brokerTopicStats.allTopicsStats.invalidMagicNumberRecordsPerSec.mark() - return Some(ApiRecordError(Errors.INVALID_RECORD, new RecordError(batchIndex, - s"Record $record's magic does not match outer magic ${batch.magic} in topic partition $topicPartition."))) - } - - // verify the record-level CRC only if this is one of the deep entries of a compressed message - // set for magic v0 and v1. For non-compressed messages, there is no inner record for magic v0 and v1, - // so we depend on the batch-level CRC check in Log.analyzeAndValidateRecords(). For magic v2 and above, - // there is no record-level CRC to check. - if (batch.magic <= RecordBatch.MAGIC_VALUE_V1 && batch.isCompressed) { - try { - record.ensureValid() - } catch { - case e: InvalidRecordException => - brokerTopicStats.allTopicsStats.invalidMessageCrcRecordsPerSec.mark() - throw new CorruptRecordException(e.getMessage + s" in topic partition $topicPartition.") - } - } - - validateKey(record, batchIndex, topicPartition, compactedTopic, brokerTopicStats).orElse { - validateTimestamp(batch, record, batchIndex, now, timestampType, timestampDiffMaxMs) - } - } - - private def convertAndAssignOffsetsNonCompressed(records: MemoryRecords, - topicPartition: TopicPartition, - offsetCounter: LongRef, - compactedTopic: Boolean, - time: Time, - now: Long, - timestampType: TimestampType, - timestampDiffMaxMs: Long, - toMagicValue: Byte, - partitionLeaderEpoch: Int, - origin: AppendOrigin, - brokerTopicStats: BrokerTopicStats): ValidationAndOffsetAssignResult = { - val startNanos = time.nanoseconds - val sizeInBytesAfterConversion = AbstractRecords.estimateSizeInBytes(toMagicValue, offsetCounter.value, - CompressionType.NONE, records.records) - - val (producerId, producerEpoch, sequence, isTransactional) = { - val first = records.batches.asScala.head - (first.producerId, first.producerEpoch, first.baseSequence, first.isTransactional) - } - - // The current implementation of BufferSupplier is naive and works best when the buffer size - // cardinality is low, so don't use it here - val newBuffer = ByteBuffer.allocate(sizeInBytesAfterConversion) - val builder = MemoryRecords.builder(newBuffer, toMagicValue, CompressionType.NONE, timestampType, - offsetCounter.value, now, producerId, producerEpoch, sequence, isTransactional, partitionLeaderEpoch) - - val firstBatch = getFirstBatchAndMaybeValidateNoMoreBatches(records, CompressionType.NONE) - - records.batches.forEach { batch => - validateBatch(topicPartition, firstBatch, batch, origin, toMagicValue, brokerTopicStats) - - val recordErrors = new ArrayBuffer[ApiRecordError](0) - for ((record, batchIndex) <- batch.asScala.view.zipWithIndex) { - validateRecord(batch, topicPartition, record, batchIndex, now, timestampType, - timestampDiffMaxMs, compactedTopic, brokerTopicStats).foreach(recordError => recordErrors += recordError) - // we fail the batch if any record fails, so we stop appending if any record fails - if (recordErrors.isEmpty) - builder.appendWithOffset(offsetCounter.getAndIncrement(), record) - } - - processRecordErrors(recordErrors) - } - - val convertedRecords = builder.build() - - val info = builder.info - val recordConversionStats = new RecordConversionStats(builder.uncompressedBytesWritten, - builder.numRecords, time.nanoseconds - startNanos) - ValidationAndOffsetAssignResult( - validatedRecords = convertedRecords, - maxTimestamp = info.maxTimestamp, - shallowOffsetOfMaxTimestamp = info.shallowOffsetOfMaxTimestamp, - messageSizeMaybeChanged = true, - recordConversionStats = recordConversionStats) - } - - def assignOffsetsNonCompressed(records: MemoryRecords, - topicPartition: TopicPartition, - offsetCounter: LongRef, - now: Long, - compactedTopic: Boolean, - timestampType: TimestampType, - timestampDiffMaxMs: Long, - partitionLeaderEpoch: Int, - origin: AppendOrigin, - magic: Byte, - brokerTopicStats: BrokerTopicStats): ValidationAndOffsetAssignResult = { - var maxTimestamp = RecordBatch.NO_TIMESTAMP - var offsetOfMaxTimestamp = -1L - val initialOffset = offsetCounter.value - - val firstBatch = getFirstBatchAndMaybeValidateNoMoreBatches(records, CompressionType.NONE) - - records.batches.forEach { batch => - validateBatch(topicPartition, firstBatch, batch, origin, magic, brokerTopicStats) - - var maxBatchTimestamp = RecordBatch.NO_TIMESTAMP - var offsetOfMaxBatchTimestamp = -1L - - val recordErrors = new ArrayBuffer[ApiRecordError](0) - // This is a hot path and we want to avoid any unnecessary allocations. - // That said, there is no benefit in using `skipKeyValueIterator` for the uncompressed - // case since we don't do key/value copies in this path (we just slice the ByteBuffer) - var batchIndex = 0 - batch.forEach { record => - validateRecord(batch, topicPartition, record, batchIndex, now, timestampType, - timestampDiffMaxMs, compactedTopic, brokerTopicStats).foreach(recordError => recordErrors += recordError) - - val offset = offsetCounter.getAndIncrement() - if (batch.magic > RecordBatch.MAGIC_VALUE_V0 && record.timestamp > maxBatchTimestamp) { - maxBatchTimestamp = record.timestamp - offsetOfMaxBatchTimestamp = offset - } - batchIndex += 1 - } - - processRecordErrors(recordErrors) - - if (batch.magic > RecordBatch.MAGIC_VALUE_V0 && maxBatchTimestamp > maxTimestamp) { - maxTimestamp = maxBatchTimestamp - offsetOfMaxTimestamp = offsetOfMaxBatchTimestamp - } - - batch.setLastOffset(offsetCounter.value - 1) - - if (batch.magic >= RecordBatch.MAGIC_VALUE_V2) - batch.setPartitionLeaderEpoch(partitionLeaderEpoch) - - if (batch.magic > RecordBatch.MAGIC_VALUE_V0) { - if (timestampType == TimestampType.LOG_APPEND_TIME) - batch.setMaxTimestamp(TimestampType.LOG_APPEND_TIME, now) - else - batch.setMaxTimestamp(timestampType, maxBatchTimestamp) - } - } - - if (timestampType == TimestampType.LOG_APPEND_TIME) { - maxTimestamp = now - if (magic >= RecordBatch.MAGIC_VALUE_V2) - offsetOfMaxTimestamp = offsetCounter.value - 1 - else - offsetOfMaxTimestamp = initialOffset - } - - ValidationAndOffsetAssignResult( - validatedRecords = records, - maxTimestamp = maxTimestamp, - shallowOffsetOfMaxTimestamp = offsetOfMaxTimestamp, - messageSizeMaybeChanged = false, - recordConversionStats = RecordConversionStats.EMPTY) - } - - /** - * We cannot do in place assignment in one of the following situations: - * 1. Source and target compression codec are different - * 2. When the target magic is not equal to batches' magic, meaning format conversion is needed. - * 3. When the target magic is equal to V0, meaning absolute offsets need to be re-assigned. - */ - def validateMessagesAndAssignOffsetsCompressed(records: MemoryRecords, - topicPartition: TopicPartition, - offsetCounter: LongRef, - time: Time, - now: Long, - sourceCompression: CompressionType, - targetCompression: CompressionType, - compactedTopic: Boolean, - toMagic: Byte, - timestampType: TimestampType, - timestampDiffMaxMs: Long, - partitionLeaderEpoch: Int, - origin: AppendOrigin, - interBrokerProtocolVersion: MetadataVersion, - brokerTopicStats: BrokerTopicStats, - requestLocal: RequestLocal): ValidationAndOffsetAssignResult = { - - if (targetCompression == CompressionType.ZSTD && interBrokerProtocolVersion.isLessThan(IBP_2_1_IV0)) - throw new UnsupportedCompressionTypeException("Produce requests to inter.broker.protocol.version < 2.1 broker " + - "are not allowed to use ZStandard compression") - - def validateRecordCompression(batchIndex: Int, record: Record): Option[ApiRecordError] = { - if (sourceCompression != CompressionType.NONE && record.isCompressed) - Some(ApiRecordError(Errors.INVALID_RECORD, new RecordError(batchIndex, - s"Compressed outer record should not have an inner record with a compression attribute set: $record"))) - else None - } - - // No in place assignment situation 1 - var inPlaceAssignment = sourceCompression == targetCompression - - var maxTimestamp = RecordBatch.NO_TIMESTAMP - val expectedInnerOffset = new LongRef(0) - val validatedRecords = new mutable.ArrayBuffer[Record] - - var uncompressedSizeInBytes = 0 - - // Assume there's only one batch with compressed memory records; otherwise, return InvalidRecordException - // One exception though is that with format smaller than v2, if sourceCompression is noCompression, then each batch is actually - // a single record so we'd need to special handle it by creating a single wrapper batch that includes all the records - val firstBatch = getFirstBatchAndMaybeValidateNoMoreBatches(records, sourceCompression) - - // No in place assignment situation 2 and 3: we only need to check for the first batch because: - // 1. For most cases (compressed records, v2, for example), there's only one batch anyways. - // 2. For cases that there may be multiple batches, all batches' magic should be the same. - if (firstBatch.magic != toMagic || toMagic == RecordBatch.MAGIC_VALUE_V0) - inPlaceAssignment = false - - // Do not compress control records unless they are written compressed - if (sourceCompression == CompressionType.NONE && firstBatch.isControlBatch) - inPlaceAssignment = true - - records.batches.forEach { batch => - validateBatch(topicPartition, firstBatch, batch, origin, toMagic, brokerTopicStats) - uncompressedSizeInBytes += AbstractRecords.recordBatchHeaderSizeInBytes(toMagic, batch.compressionType()) - - // if we are on version 2 and beyond, and we know we are going for in place assignment, - // then we can optimize the iterator to skip key / value / headers since they would not be used at all - val recordsIterator = if (inPlaceAssignment && firstBatch.magic >= RecordBatch.MAGIC_VALUE_V2) - batch.skipKeyValueIterator(requestLocal.bufferSupplier) - else - batch.streamingIterator(requestLocal.bufferSupplier) - - try { - val recordErrors = new ArrayBuffer[ApiRecordError](0) - // this is a hot path and we want to avoid any unnecessary allocations. - var batchIndex = 0 - recordsIterator.forEachRemaining { record => - val expectedOffset = expectedInnerOffset.getAndIncrement() - val recordError = validateRecordCompression(batchIndex, record).orElse { - validateRecord(batch, topicPartition, record, batchIndex, now, - timestampType, timestampDiffMaxMs, compactedTopic, brokerTopicStats).orElse { - if (batch.magic > RecordBatch.MAGIC_VALUE_V0 && toMagic > RecordBatch.MAGIC_VALUE_V0) { - if (record.timestamp > maxTimestamp) - maxTimestamp = record.timestamp - - // Some older clients do not implement the V1 internal offsets correctly. - // Historically the broker handled this by rewriting the batches rather - // than rejecting the request. We must continue this handling here to avoid - // breaking these clients. - if (record.offset != expectedOffset) - inPlaceAssignment = false - } - None - } - } - - recordError match { - case Some(e) => recordErrors += e - case None => - uncompressedSizeInBytes += record.sizeInBytes() - validatedRecords += record - } - batchIndex += 1 - } - processRecordErrors(recordErrors) - } finally { - recordsIterator.close() - } - } - - if (!inPlaceAssignment) { - val (producerId, producerEpoch, sequence, isTransactional) = { - // note that we only reassign offsets for requests coming straight from a producer. For records with magic V2, - // there should be exactly one RecordBatch per request, so the following is all we need to do. For Records - // with older magic versions, there will never be a producer id, etc. - val first = records.batches.asScala.head - (first.producerId, first.producerEpoch, first.baseSequence, first.isTransactional) - } - buildRecordsAndAssignOffsets(toMagic, offsetCounter, time, timestampType, targetCompression, now, validatedRecords, - producerId, producerEpoch, sequence, isTransactional, partitionLeaderEpoch, uncompressedSizeInBytes) - } else { - // we can update the batch only and write the compressed payload as is; - // again we assume only one record batch within the compressed set - val batch = records.batches.iterator.next() - val lastOffset = offsetCounter.addAndGet(validatedRecords.size) - 1 - - batch.setLastOffset(lastOffset) - - if (timestampType == TimestampType.LOG_APPEND_TIME) - maxTimestamp = now - - if (toMagic >= RecordBatch.MAGIC_VALUE_V1) - batch.setMaxTimestamp(timestampType, maxTimestamp) - - if (toMagic >= RecordBatch.MAGIC_VALUE_V2) - batch.setPartitionLeaderEpoch(partitionLeaderEpoch) - - val recordConversionStats = new RecordConversionStats(uncompressedSizeInBytes, 0, 0) - ValidationAndOffsetAssignResult(validatedRecords = records, - maxTimestamp = maxTimestamp, - shallowOffsetOfMaxTimestamp = lastOffset, - messageSizeMaybeChanged = false, - recordConversionStats = recordConversionStats) - } - } - - private def buildRecordsAndAssignOffsets(magic: Byte, - offsetCounter: LongRef, - time: Time, - timestampType: TimestampType, - compressionType: CompressionType, - logAppendTime: Long, - validatedRecords: Seq[Record], - producerId: Long, - producerEpoch: Short, - baseSequence: Int, - isTransactional: Boolean, - partitionLeaderEpoch: Int, - uncompressedSizeInBytes: Int): ValidationAndOffsetAssignResult = { - val startNanos = time.nanoseconds - val estimatedSize = AbstractRecords.estimateSizeInBytes(magic, offsetCounter.value, compressionType, - validatedRecords.asJava) - // The current implementation of BufferSupplier is naive and works best when the buffer size - // cardinality is low, so don't use it here - val buffer = ByteBuffer.allocate(estimatedSize) - val builder = MemoryRecords.builder(buffer, magic, compressionType, timestampType, offsetCounter.value, - logAppendTime, producerId, producerEpoch, baseSequence, isTransactional, partitionLeaderEpoch) - - validatedRecords.foreach { record => - builder.appendWithOffset(offsetCounter.getAndIncrement(), record) - } - - val records = builder.build() - - val info = builder.info - - // This is not strictly correct, it represents the number of records where in-place assignment is not possible - // instead of the number of records that were converted. It will over-count cases where the source and target are - // message format V0 or if the inner offsets are not consecutive. This is OK since the impact is the same: we have - // to rebuild the records (including recompression if enabled). - val conversionCount = builder.numRecords - val recordConversionStats = new RecordConversionStats(uncompressedSizeInBytes + builder.uncompressedBytesWritten, - conversionCount, time.nanoseconds - startNanos) - - ValidationAndOffsetAssignResult( - validatedRecords = records, - maxTimestamp = info.maxTimestamp, - shallowOffsetOfMaxTimestamp = info.shallowOffsetOfMaxTimestamp, - messageSizeMaybeChanged = true, - recordConversionStats = recordConversionStats) - } - - private def validateKey(record: Record, - batchIndex: Int, - topicPartition: TopicPartition, - compactedTopic: Boolean, - brokerTopicStats: BrokerTopicStats): Option[ApiRecordError] = { - if (compactedTopic && !record.hasKey) { - brokerTopicStats.allTopicsStats.noKeyCompactedTopicRecordsPerSec.mark() - Some(ApiRecordError(Errors.INVALID_RECORD, new RecordError(batchIndex, - s"Compacted topic cannot accept message without key in topic partition $topicPartition."))) - } else None - } - - private def validateTimestamp(batch: RecordBatch, - record: Record, - batchIndex: Int, - now: Long, - timestampType: TimestampType, - timestampDiffMaxMs: Long): Option[ApiRecordError] = { - if (timestampType == TimestampType.CREATE_TIME - && record.timestamp != RecordBatch.NO_TIMESTAMP - && math.abs(record.timestamp - now) > timestampDiffMaxMs) - Some(ApiRecordError(Errors.INVALID_TIMESTAMP, new RecordError(batchIndex, - s"Timestamp ${record.timestamp} of message with offset ${record.offset} is " + - s"out of range. The timestamp should be within [${now - timestampDiffMaxMs}, " + - s"${now + timestampDiffMaxMs}]"))) - else if (batch.timestampType == TimestampType.LOG_APPEND_TIME) - Some(ApiRecordError(Errors.INVALID_TIMESTAMP, new RecordError(batchIndex, - s"Invalid timestamp type in message $record. Producer should not set timestamp " + - "type to LogAppendTime."))) - else None - } - - private def processRecordErrors(recordErrors: Seq[ApiRecordError]): Unit = { - if (recordErrors.nonEmpty) { - val errors = recordErrors.map(_.recordError) - if (recordErrors.exists(_.apiError == Errors.INVALID_TIMESTAMP)) { - throw new RecordValidationException(new InvalidTimestampException( - "One or more records have been rejected due to invalid timestamp"), errors) - } else { - throw new RecordValidationException(new InvalidRecordException( - "One or more records have been rejected due to " + errors.size + " record errors " + - "in total, and only showing the first three errors at most: " + errors.asJava.subList(0, math.min(errors.size, 3))), errors) - } - } - } - - case class ValidationAndOffsetAssignResult(validatedRecords: MemoryRecords, - maxTimestamp: Long, - shallowOffsetOfMaxTimestamp: Long, - messageSizeMaybeChanged: Boolean, - recordConversionStats: RecordConversionStats) - - private case class ApiRecordError(apiError: Errors, recordError: RecordError) -} diff --git a/core/src/main/scala/kafka/log/OffsetMap.scala b/core/src/main/scala/kafka/log/OffsetMap.scala deleted file mode 100755 index 22b5305203e90..0000000000000 --- a/core/src/main/scala/kafka/log/OffsetMap.scala +++ /dev/null @@ -1,201 +0,0 @@ -/** - * 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.log - -import java.util.Arrays -import java.security.MessageDigest -import java.nio.ByteBuffer -import kafka.utils._ -import org.apache.kafka.common.utils.Utils - -trait OffsetMap { - def slots: Int - def put(key: ByteBuffer, offset: Long): Unit - def get(key: ByteBuffer): Long - def updateLatestOffset(offset: Long): Unit - def clear(): Unit - def size: Int - def utilization: Double = size.toDouble / slots - def latestOffset: Long -} - -/** - * An hash table used for deduplicating the log. This hash table uses a cryptographicly secure hash of the key as a proxy for the key - * for comparisons and to save space on object overhead. Collisions are resolved by probing. This hash table does not support deletes. - * @param memory The amount of memory this map can use - * @param hashAlgorithm The hash algorithm instance to use: MD2, MD5, SHA-1, SHA-256, SHA-384, SHA-512 - */ -@nonthreadsafe -class SkimpyOffsetMap(val memory: Int, val hashAlgorithm: String = "MD5") extends OffsetMap { - private val bytes = ByteBuffer.allocate(memory) - - /* the hash algorithm instance to use, default is MD5 */ - private val digest = MessageDigest.getInstance(hashAlgorithm) - - /* the number of bytes for this hash algorithm */ - private val hashSize = digest.getDigestLength - - /* create some hash buffers to avoid reallocating each time */ - private val hash1 = new Array[Byte](hashSize) - private val hash2 = new Array[Byte](hashSize) - - /* number of entries put into the map */ - private var entries = 0 - - /* number of lookups on the map */ - private var lookups = 0L - - /* the number of probes for all lookups */ - private var probes = 0L - - /* the latest offset written into the map */ - private var lastOffset = -1L - - /** - * The number of bytes of space each entry uses (the number of bytes in the hash plus an 8 byte offset) - */ - val bytesPerEntry = hashSize + 8 - - /** - * The maximum number of entries this map can contain - */ - val slots: Int = memory / bytesPerEntry - - /** - * Associate this offset to the given key. - * @param key The key - * @param offset The offset - */ - override def put(key: ByteBuffer, offset: Long): Unit = { - require(entries < slots, "Attempt to add a new entry to a full offset map.") - lookups += 1 - hashInto(key, hash1) - // probe until we find the first empty slot - var attempt = 0 - var pos = positionOf(hash1, attempt) - while(!isEmpty(pos)) { - bytes.position(pos) - bytes.get(hash2) - if(Arrays.equals(hash1, hash2)) { - // we found an existing entry, overwrite it and return (size does not change) - bytes.putLong(offset) - lastOffset = offset - return - } - attempt += 1 - pos = positionOf(hash1, attempt) - } - // found an empty slot, update it--size grows by 1 - bytes.position(pos) - bytes.put(hash1) - bytes.putLong(offset) - lastOffset = offset - entries += 1 - } - - /** - * Check that there is no entry at the given position - */ - private def isEmpty(position: Int): Boolean = - bytes.getLong(position) == 0 && bytes.getLong(position + 8) == 0 && bytes.getLong(position + 16) == 0 - - /** - * Get the offset associated with this key. - * @param key The key - * @return The offset associated with this key or -1 if the key is not found - */ - override def get(key: ByteBuffer): Long = { - lookups += 1 - hashInto(key, hash1) - // search for the hash of this key by repeated probing until we find the hash we are looking for or we find an empty slot - var attempt = 0 - var pos = 0 - //we need to guard against attempt integer overflow if the map is full - //limit attempt to number of slots once positionOf(..) enters linear search mode - val maxAttempts = slots + hashSize - 4 - do { - if(attempt >= maxAttempts) - return -1L - pos = positionOf(hash1, attempt) - bytes.position(pos) - if(isEmpty(pos)) - return -1L - bytes.get(hash2) - attempt += 1 - } while(!Arrays.equals(hash1, hash2)) - bytes.getLong() - } - - /** - * Change the salt used for key hashing making all existing keys unfindable. - */ - override def clear(): Unit = { - this.entries = 0 - this.lookups = 0L - this.probes = 0L - this.lastOffset = -1L - Arrays.fill(bytes.array, bytes.arrayOffset, bytes.arrayOffset + bytes.limit(), 0.toByte) - } - - /** - * The number of entries put into the map (note that not all may remain) - */ - override def size: Int = entries - - /** - * The rate of collisions in the lookups - */ - def collisionRate: Double = - (this.probes - this.lookups) / this.lookups.toDouble - - /** - * The latest offset put into the map - */ - override def latestOffset: Long = lastOffset - - override def updateLatestOffset(offset: Long): Unit = { - lastOffset = offset - } - - /** - * Calculate the ith probe position. We first try reading successive integers from the hash itself - * then if all of those fail we degrade to linear probing. - * @param hash The hash of the key to find the position for - * @param attempt The ith probe - * @return The byte offset in the buffer at which the ith probing for the given hash would reside - */ - private def positionOf(hash: Array[Byte], attempt: Int): Int = { - val probe = CoreUtils.readInt(hash, math.min(attempt, hashSize - 4)) + math.max(0, attempt - hashSize + 4) - val slot = Utils.abs(probe) % slots - this.probes += 1 - slot * bytesPerEntry - } - - /** - * The offset at which we have stored the given key - * @param key The key to hash - * @param buffer The buffer to store the hash into - */ - private def hashInto(key: ByteBuffer, buffer: Array[Byte]): Unit = { - key.mark() - digest.update(key) - key.reset() - digest.digest(buffer, 0, hashSize) - } - -} diff --git a/core/src/main/scala/kafka/log/ProducerStateManager.scala b/core/src/main/scala/kafka/log/ProducerStateManager.scala index 7307bed0efa02..1b9c4593d0220 100644 --- a/core/src/main/scala/kafka/log/ProducerStateManager.scala +++ b/core/src/main/scala/kafka/log/ProducerStateManager.scala @@ -30,7 +30,7 @@ import org.apache.kafka.common.errors._ import org.apache.kafka.common.protocol.types._ import org.apache.kafka.common.record.{ControlRecordType, DefaultRecordBatch, EndTransactionMarker, RecordBatch} import org.apache.kafka.common.utils.{ByteUtils, Crc32C, Time, Utils} -import org.apache.kafka.server.log.internals.CompletedTxn +import org.apache.kafka.server.log.internals.{AppendOrigin, CompletedTxn} import scala.jdk.CollectionConverters._ import scala.collection.mutable.ListBuffer @@ -195,7 +195,7 @@ private[log] class ProducerAppendInfo(val topicPartition: TopicPartition, private def maybeValidateDataBatch(producerEpoch: Short, firstSeq: Int, offset: Long): Unit = { checkProducerEpoch(producerEpoch, offset) - if (origin == AppendOrigin.Client) { + if (origin == AppendOrigin.CLIENT) { checkSequence(producerEpoch, firstSeq, offset) } } @@ -205,7 +205,7 @@ private[log] class ProducerAppendInfo(val topicPartition: TopicPartition, val message = s"Epoch of producer $producerId at offset $offset in $topicPartition is $producerEpoch, " + s"which is smaller than the last seen epoch ${updatedEntry.producerEpoch}" - if (origin == AppendOrigin.Replication) { + if (origin == AppendOrigin.REPLICATION) { warn(message) } else { // Starting from 2.7, we replaced ProducerFenced error with InvalidProducerEpoch in the @@ -294,7 +294,7 @@ private[log] class ProducerAppendInfo(val topicPartition: TopicPartition, private def checkCoordinatorEpoch(endTxnMarker: EndTransactionMarker, offset: Long): Unit = { if (updatedEntry.coordinatorEpoch > endTxnMarker.coordinatorEpoch) { - if (origin == AppendOrigin.Replication) { + if (origin == AppendOrigin.REPLICATION) { info(s"Detected invalid coordinator epoch for producerId $producerId at " + s"offset $offset in partition $topicPartition: ${endTxnMarker.coordinatorEpoch} " + s"is older than previously known coordinator epoch ${updatedEntry.coordinatorEpoch}") diff --git a/core/src/main/scala/kafka/log/TimeIndex.scala b/core/src/main/scala/kafka/log/TimeIndex.scala deleted file mode 100644 index f5ffc7f6c5291..0000000000000 --- a/core/src/main/scala/kafka/log/TimeIndex.scala +++ /dev/null @@ -1,228 +0,0 @@ -/** - * 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.log - -import java.io.File -import java.nio.ByteBuffer -import kafka.utils.CoreUtils.inLock -import kafka.utils.Logging -import org.apache.kafka.common.errors.InvalidOffsetException -import org.apache.kafka.common.record.RecordBatch -import org.apache.kafka.server.log.internals.{AbstractIndex, CorruptIndexException, IndexSearchType, TimestampOffset} - -/** - * An index that maps from the timestamp to the logical offsets of the messages in a segment. This index might be - * sparse, i.e. it may not hold an entry for all the messages in the segment. - * - * The index is stored in a file that is preallocated to hold a fixed maximum amount of 12-byte time index entries. - * The file format is a series of time index entries. The physical format is a 8 bytes timestamp and a 4 bytes "relative" - * offset used in the [[OffsetIndex]]. A time index entry (TIMESTAMP, OFFSET) means that the biggest timestamp seen - * before OFFSET is TIMESTAMP. i.e. Any message whose timestamp is greater than TIMESTAMP must come after OFFSET. - * - * All external APIs translate from relative offsets to full offsets, so users of this class do not interact with the internal - * storage format. - * - * The timestamps in the same time index file are guaranteed to be monotonically increasing. - * - * The index supports timestamp lookup for a memory map of this file. The lookup is done using a binary search to find - * the offset of the message whose indexed timestamp is closest but smaller or equals to the target timestamp. - * - * Time index files can be opened in two ways: either as an empty, mutable index that allows appending or - * an immutable read-only index file that has previously been populated. The makeReadOnly method will turn a mutable file into an - * immutable one and truncate off any extra bytes. This is done when the index file is rolled over. - * - * No attempt is made to checksum the contents of this file, in the event of a crash it is rebuilt. - * - */ -// Avoid shadowing mutable file in AbstractIndex -class TimeIndex(_file: File, baseOffset: Long, maxIndexSize: Int = -1, writable: Boolean = true) - extends AbstractIndex(_file, baseOffset, maxIndexSize, writable) { - import TimeIndex._ - - @volatile private var _lastEntry = lastEntryFromIndexFile - - override def entrySize = 12 - - debug(s"Loaded index file ${file.getAbsolutePath} with maxEntries = $maxEntries, maxIndexSize = $maxIndexSize," + - s" entries = $entries, lastOffset = ${_lastEntry}, file position = ${mmap.position()}") - - // We override the full check to reserve the last time index entry slot for the on roll call. - override def isFull: Boolean = entries >= maxEntries - 1 - - private def timestamp(buffer: ByteBuffer, n: Int): Long = buffer.getLong(n * entrySize) - - private def relativeOffset(buffer: ByteBuffer, n: Int): Int = buffer.getInt(n * entrySize + 8) - - def lastEntry: TimestampOffset = _lastEntry - - /** - * Read the last entry from the index file. This operation involves disk access. - */ - private def lastEntryFromIndexFile: TimestampOffset = { - inLock(lock) { - entries match { - case 0 => new TimestampOffset(RecordBatch.NO_TIMESTAMP, baseOffset) - case s => parseEntry(mmap, s - 1) - } - } - } - - /** - * Get the nth timestamp mapping from the time index - * @param n The entry number in the time index - * @return The timestamp/offset pair at that entry - */ - def entry(n: Int): TimestampOffset = { - maybeLock(lock, { () => - if(n >= entries) - throw new IllegalArgumentException(s"Attempt to fetch the ${n}th entry from time index ${file.getAbsolutePath} " + - s"which has size $entries.") - parseEntry(mmap, n) - }) - } - - override def parseEntry(buffer: ByteBuffer, n: Int): TimestampOffset = { - new TimestampOffset(timestamp(buffer, n), baseOffset + relativeOffset(buffer, n)) - } - - /** - * Attempt to append a time index entry to the time index. - * The new entry is appended only if both the timestamp and offset are greater than the last appended timestamp and - * the last appended offset. - * - * @param timestamp The timestamp of the new time index entry - * @param offset The offset of the new time index entry - * @param skipFullCheck To skip checking whether the segment is full or not. We only skip the check when the segment - * gets rolled or the segment is closed. - */ - def maybeAppend(timestamp: Long, offset: Long, skipFullCheck: Boolean = false): Unit = { - inLock(lock) { - if (!skipFullCheck) - require(!isFull, "Attempt to append to a full time index (size = " + entries + ").") - // We do not throw exception when the offset equals to the offset of last entry. That means we are trying - // to insert the same time index entry as the last entry. - // If the timestamp index entry to be inserted is the same as the last entry, we simply ignore the insertion - // because that could happen in the following two scenarios: - // 1. A log segment is closed. - // 2. LogSegment.onBecomeInactiveSegment() is called when an active log segment is rolled. - if (entries != 0 && offset < lastEntry.offset) - throw new InvalidOffsetException(s"Attempt to append an offset ($offset) to slot $entries no larger than" + - s" the last offset appended (${lastEntry.offset}) to ${file.getAbsolutePath}.") - if (entries != 0 && timestamp < lastEntry.timestamp) - throw new IllegalStateException(s"Attempt to append a timestamp ($timestamp) to slot $entries no larger" + - s" than the last timestamp appended (${lastEntry.timestamp}) to ${file.getAbsolutePath}.") - // We only append to the time index when the timestamp is greater than the last inserted timestamp. - // If all the messages are in message format v0, the timestamp will always be NoTimestamp. In that case, the time - // index will be empty. - if (timestamp > lastEntry.timestamp) { - trace(s"Adding index entry $timestamp => $offset to ${file.getAbsolutePath}.") - mmap.putLong(timestamp) - mmap.putInt(relativeOffset(offset)) - incrementEntries() - _lastEntry = new TimestampOffset(timestamp, offset) - require(entries * entrySize == mmap.position(), s"$entries entries but file position in index is ${mmap.position()}.") - } - } - } - - /** - * Find the time index entry whose timestamp is less than or equal to the given timestamp. - * If the target timestamp is smaller than the least timestamp in the time index, (NoTimestamp, baseOffset) is - * returned. - * - * @param targetTimestamp The timestamp to look up. - * @return The time index entry found. - */ - def lookup(targetTimestamp: Long): TimestampOffset = { - maybeLock(lock, {() => - val idx = mmap.duplicate - val slot = largestLowerBoundSlotFor(idx, targetTimestamp, IndexSearchType.KEY) - if (slot == -1) - new TimestampOffset(RecordBatch.NO_TIMESTAMP, baseOffset) - else - parseEntry(idx, slot) - }) - } - - override def truncate(): Unit = truncateToEntries(0) - - /** - * Remove all entries from the index which have an offset greater than or equal to the given offset. - * Truncating to an offset larger than the largest in the index has no effect. - */ - override def truncateTo(offset: Long): Unit = { - inLock(lock) { - val idx = mmap.duplicate - val slot = largestLowerBoundSlotFor(idx, offset, IndexSearchType.VALUE) - - /* There are 3 cases for choosing the new size - * 1) if there is no entry in the index <= the offset, delete everything - * 2) if there is an entry for this exact offset, delete it and everything larger than it - * 3) if there is no entry for this offset, delete everything larger than the next smallest - */ - val newEntries = - if(slot < 0) - 0 - else if(relativeOffset(idx, slot) == offset - baseOffset) - slot - else - slot + 1 - truncateToEntries(newEntries) - } - } - - override def resize(newSize: Int): Boolean = { - inLock(lock) { - if (super.resize(newSize)) { - _lastEntry = lastEntryFromIndexFile - true - } else - false - } - } - - /** - * Truncates index to a known number of entries. - */ - private def truncateToEntries(entries: Int): Unit = { - inLock(lock) { - super.truncateToEntries0(entries) - _lastEntry = lastEntryFromIndexFile - debug(s"Truncated index ${file.getAbsolutePath} to $entries entries; position is now ${mmap.position()} and last entry is now ${_lastEntry}") - } - } - - override def sanityCheck(): Unit = { - val lastTimestamp = lastEntry.timestamp - val lastOffset = lastEntry.offset - if (entries != 0 && lastTimestamp < timestamp(mmap, 0)) - throw new CorruptIndexException(s"Corrupt time index found, time index file (${file.getAbsolutePath}) has " + - s"non-zero size but the last timestamp is $lastTimestamp which is less than the first timestamp " + - s"${timestamp(mmap, 0)}") - if (entries != 0 && lastOffset < baseOffset) - throw new CorruptIndexException(s"Corrupt time index found, time index file (${file.getAbsolutePath}) has " + - s"non-zero size but the last offset is $lastOffset which is less than the first offset $baseOffset") - if (length % entrySize != 0) - throw new CorruptIndexException(s"Time index file ${file.getAbsolutePath} is corrupt, found $length bytes " + - s"which is neither positive nor a multiple of $entrySize.") - } -} - -object TimeIndex extends Logging { - override val loggerName: String = classOf[TimeIndex].getName -} diff --git a/core/src/main/scala/kafka/log/UnifiedLog.scala b/core/src/main/scala/kafka/log/UnifiedLog.scala index e7a27c35f85da..c876840182b05 100644 --- a/core/src/main/scala/kafka/log/UnifiedLog.scala +++ b/core/src/main/scala/kafka/log/UnifiedLog.scala @@ -23,13 +23,12 @@ import java.io.{File, IOException} import java.nio.file.Files import java.util.Optional import java.util.concurrent.{ConcurrentHashMap, ConcurrentMap, TimeUnit} -import kafka.common.{LongRef, OffsetsOutOfOrderException, UnexpectedAppendOffsetException} -import kafka.log.AppendOrigin.RaftLeader +import kafka.common.{OffsetsOutOfOrderException, UnexpectedAppendOffsetException} import kafka.log.remote.RemoteLogManager import kafka.metrics.KafkaMetricsGroup -import kafka.server._ import kafka.server.checkpoints.LeaderEpochCheckpointFile import kafka.server.epoch.LeaderEpochFileCache +import kafka.server.{BrokerTopicMetrics, BrokerTopicStats, FetchDataInfo, FetchHighWatermark, FetchIsolation, FetchLogEnd, FetchTxnCommitted, LogOffsetMetadata, OffsetAndEpoch, PartitionMetadataFile, RequestLocal} import kafka.utils._ import org.apache.kafka.common.errors._ import org.apache.kafka.common.internals.Topic @@ -39,11 +38,11 @@ import org.apache.kafka.common.record._ import org.apache.kafka.common.requests.ListOffsetsRequest import org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH_OFFSET import org.apache.kafka.common.requests.ProduceResponse.RecordError -import org.apache.kafka.common.utils.{Time, Utils} +import org.apache.kafka.common.utils.{PrimitiveRef, Time, Utils} import org.apache.kafka.common.{InvalidRecordException, KafkaException, TopicPartition, Uuid} import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.server.common.MetadataVersion.IBP_0_10_0_IV0 -import org.apache.kafka.server.log.internals.{AbortedTxn, CompletedTxn} +import org.apache.kafka.server.log.internals.{AbortedTxn, AppendOrigin, CompletedTxn, LogDirFailureChannel, LogValidator} import org.apache.kafka.server.log.remote.metadata.storage.TopicBasedRemoteLogMetadataManagerConfig import org.apache.kafka.server.record.BrokerCompressionType @@ -238,7 +237,7 @@ case object SnapshotGenerated extends LogStartOffsetIncrementReason { */ @threadsafe class UnifiedLog(@volatile var logStartOffset: Long, - private val localLog: LocalLog, + private[log] val localLog: LocalLog, brokerTopicStats: BrokerTopicStats, val producerIdExpirationCheckIntervalMs: Int, @volatile var leaderEpochCache: Option[LeaderEpochFileCache], @@ -254,6 +253,7 @@ class UnifiedLog(@volatile var logStartOffset: Long, /* A lock that guards all modifications to the log */ private val lock = new Object + private val validatorMetricsRecorder = newValidatorMetricsRecorder(brokerTopicStats.allTopicsStats) /* The earliest offset which is part of an incomplete transaction. This is used to compute the * last stable offset (LSO) in ReplicaManager. Note that it is possible that the "true" first unstable offset @@ -552,15 +552,23 @@ class UnifiedLog(@volatile var logStartOffset: Long, ) } - private val tags = { - val maybeFutureTag = if (isFuture) Map("is-future" -> "true") else Map.empty[String, String] - Map("topic" -> topicPartition.topic, "partition" -> topicPartition.partition.toString) ++ maybeFutureTag - } - newGauge(LogMetricNames.NumLogSegments, () => numberOfSegments, tags) - newGauge(LogMetricNames.LogStartOffset, () => logStartOffset, tags) - newGauge(LogMetricNames.LogEndOffset, () => logEndOffset, tags) - newGauge(LogMetricNames.Size, () => size, tags) + private var metricNames: Map[String, Map[String, String]] = Map.empty + + newMetrics() + private[log] def newMetrics(): Unit = { + val tags = Map("topic" -> topicPartition.topic, "partition" -> topicPartition.partition.toString) ++ + (if (isFuture) Map("is-future" -> "true") else Map.empty) + newGauge(LogMetricNames.NumLogSegments, () => numberOfSegments, tags) + newGauge(LogMetricNames.LogStartOffset, () => logStartOffset, tags) + newGauge(LogMetricNames.LogEndOffset, () => logEndOffset, tags) + newGauge(LogMetricNames.Size, () => size, tags) + metricNames = Map(LogMetricNames.NumLogSegments -> tags, + LogMetricNames.LogStartOffset -> tags, + LogMetricNames.LogEndOffset -> tags, + LogMetricNames.Size -> tags) + + } val producerExpireCheck = scheduler.schedule(name = "PeriodicProducerExpirationCheck", fun = () => { lock synchronized { @@ -760,10 +768,10 @@ class UnifiedLog(@volatile var logStartOffset: Long, */ def appendAsLeader(records: MemoryRecords, leaderEpoch: Int, - origin: AppendOrigin = AppendOrigin.Client, + origin: AppendOrigin = AppendOrigin.CLIENT, interBrokerProtocolVersion: MetadataVersion = MetadataVersion.latest, requestLocal: RequestLocal = RequestLocal.NoCaching): LogAppendInfo = { - val validateAndAssignOffsets = origin != AppendOrigin.RaftLeader + val validateAndAssignOffsets = origin != AppendOrigin.RAFT_LEADER append(records, origin, interBrokerProtocolVersion, validateAndAssignOffsets, leaderEpoch, Some(requestLocal), ignoreRecordSize = false) } @@ -776,7 +784,7 @@ class UnifiedLog(@volatile var logStartOffset: Long, */ def appendAsFollower(records: MemoryRecords): LogAppendInfo = { append(records, - origin = AppendOrigin.Replication, + origin = AppendOrigin.REPLICATION, interBrokerProtocolVersion = MetadataVersion.latest, validateAndAssignOffsets = false, leaderEpoch = -1, @@ -829,15 +837,12 @@ class UnifiedLog(@volatile var logStartOffset: Long, localLog.checkIfMemoryMappedBufferClosed() if (validateAndAssignOffsets) { // assign offsets to the message set - val offset = new LongRef(localLog.logEndOffset) + val offset = PrimitiveRef.ofLong(localLog.logEndOffset) appendInfo.firstOffset = Some(LogOffsetMetadata(offset.value)) - val now = time.milliseconds val validateAndOffsetAssignResult = try { - LogValidator.validateMessagesAndAssignOffsets(validRecords, + val validator = new LogValidator(validRecords, topicPartition, - offset, time, - now, appendInfo.sourceCompression, appendInfo.targetCompression, config.compact, @@ -846,21 +851,26 @@ class UnifiedLog(@volatile var logStartOffset: Long, config.messageTimestampDifferenceMaxMs, leaderEpoch, origin, - interBrokerProtocolVersion, - brokerTopicStats, + interBrokerProtocolVersion + ) + validator.validateMessagesAndAssignOffsets(offset, + validatorMetricsRecorder, requestLocal.getOrElse(throw new IllegalArgumentException( - "requestLocal should be defined if assignOffsets is true"))) + "requestLocal should be defined if assignOffsets is true") + ).bufferSupplier + ) } catch { case e: IOException => throw new KafkaException(s"Error validating messages while appending to log $name", e) } + validRecords = validateAndOffsetAssignResult.validatedRecords - appendInfo.maxTimestamp = validateAndOffsetAssignResult.maxTimestamp - appendInfo.offsetOfMaxTimestamp = validateAndOffsetAssignResult.shallowOffsetOfMaxTimestamp + appendInfo.maxTimestamp = validateAndOffsetAssignResult.maxTimestampMs + appendInfo.offsetOfMaxTimestamp = validateAndOffsetAssignResult.shallowOffsetOfMaxTimestampMs appendInfo.lastOffset = offset.value - 1 appendInfo.recordConversionStats = validateAndOffsetAssignResult.recordConversionStats if (config.messageTimestampType == TimestampType.LOG_APPEND_TIME) - appendInfo.logAppendTime = now + appendInfo.logAppendTime = validateAndOffsetAssignResult.logAppendTimeMs // re-validate message sizes if there's a possibility that they have changed (due to re-compression or message // format conversion) @@ -1070,7 +1080,7 @@ class UnifiedLog(@volatile var logStartOffset: Long, if (batch.hasProducerId) { // if this is a client produce request, there will be up to 5 batches which could have been duplicated. // If we find a duplicate, we return the metadata of the appended batch to the client. - if (origin == AppendOrigin.Client) { + if (origin == AppendOrigin.CLIENT) { val maybeLastEntry = producerStateManager.lastEntry(batch.producerId) maybeLastEntry.flatMap(_.findDuplicateBatch(batch)).foreach { duplicate => @@ -1129,11 +1139,11 @@ class UnifiedLog(@volatile var logStartOffset: Long, var lastOffsetOfFirstBatch = -1L records.batches.forEach { batch => - if (origin == RaftLeader && batch.partitionLeaderEpoch != leaderEpoch) { + if (origin == AppendOrigin.RAFT_LEADER && batch.partitionLeaderEpoch != leaderEpoch) { throw new InvalidRecordException("Append from Raft leader did not set the batch epoch correctly") } // we only validate V2 and higher to avoid potential compatibility issues with older clients - if (batch.magic >= RecordBatch.MAGIC_VALUE_V2 && origin == AppendOrigin.Client && batch.baseOffset != 0) + if (batch.magic >= RecordBatch.MAGIC_VALUE_V2 && origin == AppendOrigin.CLIENT && batch.baseOffset != 0) throw new InvalidRecordException(s"The baseOffset of the record batch in the append to $topicPartition should " + s"be 0, but it is ${batch.baseOffset}") @@ -1793,10 +1803,10 @@ class UnifiedLog(@volatile var logStartOffset: Long, * remove deleted log metrics */ private[log] def removeLogMetrics(): Unit = { - removeMetric(LogMetricNames.NumLogSegments, tags) - removeMetric(LogMetricNames.LogStartOffset, tags) - removeMetric(LogMetricNames.LogEndOffset, tags) - removeMetric(LogMetricNames.Size, tags) + metricNames.foreach { + case (name, tags) => removeMetric(name, tags) + } + metricNames = Map.empty } /** @@ -1955,7 +1965,7 @@ object UnifiedLog extends Logging { batch, loadedProducers, firstOffsetMetadata = None, - origin = AppendOrigin.Replication) + origin = AppendOrigin.REPLICATION) maybeCompletedTxn.foreach(completedTxns += _) } } @@ -1981,8 +1991,8 @@ object UnifiedLog extends Logging { * @param dir The directory in which the log will reside * @param topicPartition The topic partition * @param logDirFailureChannel The LogDirFailureChannel to asynchronously handle log dir failure - * @param recordVersion The record version - * @param logPrefix The logging prefix + * @param recordVersion The record version + * @param logPrefix The logging prefix * @return The new LeaderEpochFileCache instance (if created), none otherwise */ def maybeCreateLeaderEpochCache(dir: File, @@ -2059,7 +2069,7 @@ object UnifiedLog extends Logging { * @param time The time instance used for checking the clock * @param reloadFromCleanShutdown True if the producer state is being built after a clean shutdown, * false otherwise. - * @param logPrefix The logging prefix + * @param logPrefix The logging prefix */ private[log] def rebuildProducerState(producerStateManager: ProducerStateManager, segments: LogSegments, @@ -2165,7 +2175,9 @@ object UnifiedLog extends Logging { parentDir: String, topicPartition: TopicPartition): Unit = { val snapshotsToDelete = segments.flatMap { segment => - producerStateManager.removeAndMarkSnapshotForDeletion(segment.baseOffset)} + producerStateManager.removeAndMarkSnapshotForDeletion(segment.baseOffset) + } + def deleteProducerSnapshots(): Unit = { LocalLog.maybeHandleIOException(logDirFailureChannel, parentDir, @@ -2185,6 +2197,27 @@ object UnifiedLog extends Logging { private[log] def createNewCleanedSegment(dir: File, logConfig: LogConfig, baseOffset: Long): LogSegment = { LocalLog.createNewCleanedSegment(dir, logConfig, baseOffset) } + + // Visible for benchmarking + def newValidatorMetricsRecorder(allTopicsStats: BrokerTopicMetrics): LogValidator.MetricsRecorder = { + new LogValidator.MetricsRecorder { + def recordInvalidMagic(): Unit = + allTopicsStats.invalidMagicNumberRecordsPerSec.mark() + + def recordInvalidOffset(): Unit = + allTopicsStats.invalidOffsetOrSequenceRecordsPerSec.mark() + + def recordInvalidSequence(): Unit = + allTopicsStats.invalidOffsetOrSequenceRecordsPerSec.mark() + + def recordInvalidChecksums(): Unit = + allTopicsStats.invalidMessageCrcRecordsPerSec.mark() + + def recordNoKeyCompactedTopic(): Unit = + allTopicsStats.noKeyCompactedTopicRecordsPerSec.mark() + } + } + } object LogMetricNames { diff --git a/core/src/main/scala/kafka/log/remote/RemoteIndexCache.scala b/core/src/main/scala/kafka/log/remote/RemoteIndexCache.scala index 4f80ee7b4f4bb..e6e69c61fc9fe 100644 --- a/core/src/main/scala/kafka/log/remote/RemoteIndexCache.scala +++ b/core/src/main/scala/kafka/log/remote/RemoteIndexCache.scala @@ -16,13 +16,13 @@ */ package kafka.log.remote -import kafka.log.{LazyIndex, _} +import kafka.log.UnifiedLog import kafka.log.remote.RemoteIndexCache.DirName import kafka.utils.{CoreUtils, Logging, ShutdownableThread} import org.apache.kafka.common.Uuid import org.apache.kafka.common.errors.CorruptRecordException import org.apache.kafka.common.utils.Utils -import org.apache.kafka.server.log.internals.{OffsetIndex, OffsetPosition, TransactionIndex} +import org.apache.kafka.server.log.internals.{LazyIndex, OffsetIndex, OffsetPosition, TimeIndex, TransactionIndex} import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType import org.apache.kafka.server.log.remote.storage.{RemoteLogSegmentMetadata, RemoteStorageManager} @@ -144,13 +144,13 @@ class RemoteIndexCache(maxSize: Int = 1024, remoteStorageManager: RemoteStorageM if (offsetIndexFile.exists() && timestampIndexFile.exists() && txnIndexFile.exists()) { val offsetIndex: LazyIndex[OffsetIndex] = { - val index = LazyIndex.forOffset(offsetIndexFile, offset, Int.MaxValue, writable = false) + val index = LazyIndex.forOffset(offsetIndexFile, offset, Int.MaxValue, false) index.get.sanityCheck() index } val timeIndex: LazyIndex[TimeIndex] = { - val index = LazyIndex.forTime(timestampIndexFile, offset, Int.MaxValue, writable = false) + val index = LazyIndex.forTime(timestampIndexFile, offset, Int.MaxValue, false) index.get.sanityCheck() index } @@ -241,7 +241,7 @@ class RemoteIndexCache(maxSize: Int = 1024, remoteStorageManager: RemoteStorageM val offsetIndex: LazyIndex[OffsetIndex] = loadIndexFile(fileName, UnifiedLog.IndexFileSuffix, rlsMetadata => remoteStorageManager.fetchIndex(rlsMetadata, IndexType.OFFSET), file => { - val index = LazyIndex.forOffset(file, startOffset, Int.MaxValue, writable = false) + val index = LazyIndex.forOffset(file, startOffset, Int.MaxValue, false) index.get.sanityCheck() index }) @@ -249,7 +249,7 @@ class RemoteIndexCache(maxSize: Int = 1024, remoteStorageManager: RemoteStorageM val timeIndex: LazyIndex[TimeIndex] = loadIndexFile(fileName, UnifiedLog.TimeIndexFileSuffix, rlsMetadata => remoteStorageManager.fetchIndex(rlsMetadata, IndexType.TIMESTAMP), file => { - val index = LazyIndex.forTime(file, startOffset, Int.MaxValue, writable = false) + val index = LazyIndex.forTime(file, startOffset, Int.MaxValue, false) index.get.sanityCheck() index }) diff --git a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala index 2aa2feca352ec..7056f5db0e6c0 100644 --- a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala +++ b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala @@ -16,9 +16,9 @@ */ package kafka.raft -import kafka.log.{AppendOrigin, Defaults, LogConfig, LogOffsetSnapshot, ProducerStateManagerConfig, SnapshotGenerated, UnifiedLog} +import kafka.log.{Defaults, LogConfig, LogOffsetSnapshot, ProducerStateManagerConfig, SnapshotGenerated, UnifiedLog} import kafka.server.KafkaConfig.{MetadataLogSegmentBytesProp, MetadataLogSegmentMinBytesProp} -import kafka.server.{BrokerTopicStats, FetchHighWatermark, FetchLogEnd, KafkaConfig, LogDirFailureChannel, RequestLocal} +import kafka.server.{BrokerTopicStats, FetchHighWatermark, FetchLogEnd, KafkaConfig, RequestLocal} import kafka.utils.{CoreUtils, Logging, Scheduler} import org.apache.kafka.common.config.AbstractConfig import org.apache.kafka.common.errors.InvalidConfigurationException @@ -26,6 +26,7 @@ import org.apache.kafka.common.record.{ControlRecordUtils, MemoryRecords, Record import org.apache.kafka.common.utils.{BufferSupplier, Time} import org.apache.kafka.common.{KafkaException, TopicPartition, Uuid} import org.apache.kafka.raft.{Isolation, KafkaRaftClient, LogAppendInfo, LogFetchInfo, LogOffsetMetadata, OffsetAndEpoch, OffsetMetadata, ReplicatedLog, ValidOffsetAndEpoch} +import org.apache.kafka.server.log.internals.{AppendOrigin, LogDirFailureChannel} import org.apache.kafka.snapshot.{FileRawSnapshotReader, FileRawSnapshotWriter, RawSnapshotReader, RawSnapshotWriter, SnapshotPath, Snapshots} import java.io.File @@ -79,7 +80,7 @@ final class KafkaMetadataLog private ( handleAndConvertLogAppendInfo( log.appendAsLeader(records.asInstanceOf[MemoryRecords], leaderEpoch = epoch, - origin = AppendOrigin.RaftLeader, + origin = AppendOrigin.RAFT_LEADER, requestLocal = RequestLocal.NoCaching ) ) diff --git a/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala b/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala index 1c4c7c377104f..090dd6c03e233 100644 --- a/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala +++ b/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala @@ -55,7 +55,7 @@ class BrokerLifecycleManager( val config: KafkaConfig, val time: Time, val threadNamePrefix: Option[String], - val zkBrokerEpoch: Option[Long] = None + val zkBrokerEpochSupplier: Option[() => Long] = None ) extends Logging { val logContext = new LogContext(s"[BrokerLifecycleManager id=${config.nodeId}] ") @@ -270,8 +270,8 @@ class BrokerLifecycleManager( _clusterId = clusterId _advertisedListeners = advertisedListeners.duplicate() _supportedFeatures = new util.HashMap[String, VersionRange](supportedFeatures) - if (zkBrokerEpoch.isDefined) { - // ZK brokers don't block on registration during startup + if (zkBrokerEpochSupplier.isEmpty) { + // Only KRaft brokers block on registration during startup eventQueue.scheduleDeferred("initialRegistrationTimeout", new DeadlineFunction(time.nanoseconds() + initialTimeoutNs), new RegistrationTimeoutEvent()) @@ -292,7 +292,7 @@ class BrokerLifecycleManager( } val data = new BrokerRegistrationRequestData(). setBrokerId(nodeId). - setMigratingZkBrokerEpoch(zkBrokerEpoch.getOrElse(-1)). + setMigratingZkBrokerEpoch(zkBrokerEpochSupplier.map(_.apply()).getOrElse(-1)). setClusterId(_clusterId). setFeatures(features). setIncarnationId(incarnationId). diff --git a/core/src/main/scala/kafka/server/BrokerServer.scala b/core/src/main/scala/kafka/server/BrokerServer.scala index 2b7561c31f3c7..cbe700358f6f6 100644 --- a/core/src/main/scala/kafka/server/BrokerServer.scala +++ b/core/src/main/scala/kafka/server/BrokerServer.scala @@ -43,6 +43,7 @@ import org.apache.kafka.raft import org.apache.kafka.raft.{RaftClient, RaftConfig} import org.apache.kafka.server.authorizer.Authorizer import org.apache.kafka.server.common.ApiMessageAndVersion +import org.apache.kafka.server.log.internals.LogDirFailureChannel import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig import org.apache.kafka.server.metrics.KafkaYammerMetrics import org.apache.kafka.snapshot.SnapshotWriter diff --git a/core/src/main/scala/kafka/server/BrokerToControllerChannelManager.scala b/core/src/main/scala/kafka/server/BrokerToControllerChannelManager.scala index 73a77d339bd06..b1c2728756097 100644 --- a/core/src/main/scala/kafka/server/BrokerToControllerChannelManager.scala +++ b/core/src/main/scala/kafka/server/BrokerToControllerChannelManager.scala @@ -38,11 +38,13 @@ import scala.collection.Seq import scala.compat.java8.OptionConverters._ import scala.jdk.CollectionConverters._ -case class ControllerInformation(node: Option[Node], - listenerName: ListenerName, - securityProtocol: SecurityProtocol, - saslMechanism: String, - isZkController: Boolean) +case class ControllerInformation( + node: Option[Node], + listenerName: ListenerName, + securityProtocol: SecurityProtocol, + saslMechanism: String, + isZkController: Boolean +) trait ControllerNodeProvider { def getControllerInfo(): ControllerInformation @@ -190,7 +192,7 @@ class BrokerToControllerChannelManagerImpl( } private[server] def newRequestThread = { - def networkClient(controllerInfo: ControllerInformation) = { + def buildNetworkClient(controllerInfo: ControllerInformation) = { val channelBuilder = ChannelBuilders.clientChannelBuilder( controllerInfo.securityProtocol, JaasContext.Type.SERVER, @@ -239,8 +241,11 @@ class BrokerToControllerChannelManagerImpl( case Some(name) => s"$name:BrokerToControllerChannelManager broker=${config.brokerId} name=$channelName" } + val controllerInformation = controllerNodeProvider.getControllerInfo() new BrokerToControllerRequestThread( - networkClient, + buildNetworkClient(controllerInformation), + controllerInformation.isZkController, + buildNetworkClient, manualMetadataUpdater, controllerNodeProvider, config, @@ -290,6 +295,8 @@ case class BrokerToControllerQueueItem( ) class BrokerToControllerRequestThread( + initialNetworkClient: KafkaClient, + var isNetworkClientForZkController: Boolean, networkClientFactory: ControllerInformation => KafkaClient, metadataUpdater: ManualMetadataUpdater, controllerNodeProvider: ControllerNodeProvider, @@ -297,22 +304,24 @@ class BrokerToControllerRequestThread( time: Time, threadName: String, retryTimeoutMs: Long -) extends InterBrokerSendThread(threadName, null, Math.min(Int.MaxValue, Math.min(config.controllerSocketTimeoutMs, retryTimeoutMs)).toInt, time, isInterruptible = false) { - - var isZkController = false - private def maybeResetNetworkClient(controllerInformation: ControllerInformation, - initialize: Boolean = false): Unit = { - if (initialize || isZkController != controllerInformation.isZkController) { - if (!initialize) { - debug("Controller changed to " + (if (isZkController) "kraft" else "zk") + " mode. " + - "Resetting network client") - } +) extends InterBrokerSendThread( + threadName, + initialNetworkClient, + Math.min(Int.MaxValue, Math.min(config.controllerSocketTimeoutMs, retryTimeoutMs)).toInt, + time, + isInterruptible = false +) { + + private def maybeResetNetworkClient(controllerInformation: ControllerInformation): Unit = { + if (isNetworkClientForZkController != controllerInformation.isZkController) { + debug("Controller changed to " + (if (isNetworkClientForZkController) "kraft" else "zk") + " mode. " + + "Resetting network client") // Close existing network client. if (networkClient != null) { networkClient.initiateClose() networkClient.close() } - isZkController = controllerInformation.isZkController + isNetworkClientForZkController = controllerInformation.isZkController updateControllerAddress(controllerInformation.node.orNull) controllerInformation.node.foreach(n => metadataUpdater.setNodes(Seq(n).asJava)) networkClient = networkClientFactory(controllerInformation) @@ -321,7 +330,6 @@ class BrokerToControllerRequestThread( private val requestQueue = new LinkedBlockingDeque[BrokerToControllerQueueItem]() private val activeController = new AtomicReference[Node](null) - maybeResetNetworkClient(controllerNodeProvider.getControllerInfo(), initialize = true) // Used for testing @volatile diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 776196455ad09..662b61a5fd39a 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -23,7 +23,6 @@ import kafka.common.OffsetAndMetadata import kafka.controller.ReplicaAssignment import kafka.coordinator.group._ import kafka.coordinator.transaction.{InitProducerIdResult, TransactionCoordinator} -import kafka.log.AppendOrigin import kafka.network.RequestChannel import kafka.server.QuotaFactory.{QuotaManagers, UnboundedQuota} import kafka.server.metadata.ConfigRepository @@ -71,6 +70,7 @@ import org.apache.kafka.common.{Node, TopicIdPartition, TopicPartition, Uuid} import org.apache.kafka.server.authorizer._ import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.server.common.MetadataVersion.{IBP_0_11_0_IV0, IBP_2_3_IV0} +import org.apache.kafka.server.log.internals.AppendOrigin import org.apache.kafka.server.record.BrokerCompressionType import java.lang.{Long => JLong} @@ -669,7 +669,7 @@ class KafkaApis(val requestChannel: RequestChannel, timeout = produceRequest.timeout.toLong, requiredAcks = produceRequest.acks, internalTopicsAllowed = internalTopicsAllowed, - origin = AppendOrigin.Client, + origin = AppendOrigin.CLIENT, entriesPerPartition = authorizedRequestInfo, requestLocal = requestLocal, responseCallback = sendResponseCallback, @@ -2333,7 +2333,7 @@ class KafkaApis(val requestChannel: RequestChannel, timeout = config.requestTimeoutMs.toLong, requiredAcks = -1, internalTopicsAllowed = true, - origin = AppendOrigin.Coordinator, + origin = AppendOrigin.COORDINATOR, entriesPerPartition = controlRecords, requestLocal = requestLocal, responseCallback = maybeSendResponseCallback(producerId, marker.transactionResult)) diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 0b904d768d914..6423f6390ee6f 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -56,6 +56,7 @@ import org.apache.kafka.server.authorizer.Authorizer import org.apache.kafka.server.common.{ApiMessageAndVersion, MetadataVersion} import org.apache.kafka.server.common.MetadataVersion._ import org.apache.kafka.server.metrics.KafkaYammerMetrics +import org.apache.kafka.server.log.internals.LogDirFailureChannel import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig import org.apache.zookeeper.client.ZKClientConfig @@ -151,7 +152,7 @@ class KafkaServer( var kafkaScheduler: KafkaScheduler = _ - var kraftControllerNodes: Seq[Node] = Seq.empty + var kraftControllerNodes: Seq[Node] = _ @volatile var metadataCache: ZkMetadataCache = _ var quotaManagers: QuotaFactory.QuotaManagers = _ @@ -277,6 +278,8 @@ class KafkaServer( if (config.migrationEnabled) { kraftControllerNodes = RaftConfig.voterConnectionsToNodes( RaftConfig.parseVoterConnections(config.quorumVoters)).asScala + } else { + kraftControllerNodes = Seq.empty } metadataCache = MetadataCache.zkMetadataCache( config.brokerId, @@ -357,7 +360,7 @@ class KafkaServer( lifecycleManager = new BrokerLifecycleManager(config, time, threadNamePrefix, - zkBrokerEpoch = Some(brokerEpoch)) + zkBrokerEpochSupplier = Some(() => kafkaController.brokerEpoch)) // Now that the broker is successfully registered, checkpoint its metadata val zkMetaProperties = ZkMetaProperties(clusterId, config.brokerId) @@ -707,9 +710,9 @@ class KafkaServer( // 1. Find the controller and establish a connection to it. // If the controller id or the broker registration are missing, we sleep and retry (if there are remaining retries) - metadataCache.getControllerId.filter(_.isInstanceOf[ZkCachedControllerId]).map(_.id) match { - case Some(controllerId) => - metadataCache.getAliveBrokerNode(controllerId, config.interBrokerListenerName) match { + metadataCache.getControllerId match { + case Some(controllerId: ZkCachedControllerId) => + metadataCache.getAliveBrokerNode(controllerId.id, config.interBrokerListenerName) match { case Some(broker) => // if this is the first attempt, if the controller has changed or if an exception was thrown in a previous // attempt, connect to the most recent controller @@ -726,8 +729,8 @@ class KafkaServer( case None => info(s"Broker registration for controller $controllerId is not available in the metadata cache") } - case None => - info("No controller present in the metadata cache") + case Some(_: KRaftCachedControllerId) | None => + info("No zk controller present in the metadata cache") } // 2. issue a controlled shutdown to the controller diff --git a/core/src/main/scala/kafka/server/LogDirFailureChannel.scala b/core/src/main/scala/kafka/server/LogDirFailureChannel.scala deleted file mode 100644 index 71ba9ac305f7a..0000000000000 --- a/core/src/main/scala/kafka/server/LogDirFailureChannel.scala +++ /dev/null @@ -1,62 +0,0 @@ -/** - * 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.server - -import java.io.IOException -import java.util.concurrent.{ArrayBlockingQueue, ConcurrentHashMap} - -import kafka.utils.Logging - -/* - * LogDirFailureChannel allows an external thread to block waiting for new offline log dirs. - * - * There should be a single instance of LogDirFailureChannel accessible by any class that does disk-IO operation. - * If IOException is encountered while accessing a log directory, the corresponding class can add the log directory name - * to the LogDirFailureChannel using maybeAddOfflineLogDir(). Each log directory will be added only once. After a log - * directory is added for the first time, a thread which is blocked waiting for new offline log directories - * can take the name of the new offline log directory out of the LogDirFailureChannel and handle the log failure properly. - * An offline log directory will stay offline until the broker is restarted. - * - */ -class LogDirFailureChannel(logDirNum: Int) extends Logging { - - private val offlineLogDirs = new ConcurrentHashMap[String, String] - private val offlineLogDirQueue = new ArrayBlockingQueue[String](logDirNum) - - def hasOfflineLogDir(logDir: String): Boolean = { - offlineLogDirs.containsKey(logDir) - } - - /* - * If the given logDir is not already offline, add it to the - * set of offline log dirs and enqueue it to the logDirFailureEvent queue - */ - def maybeAddOfflineLogDir(logDir: String, msg: => String, e: IOException): Unit = { - error(msg, e) - if (offlineLogDirs.putIfAbsent(logDir, logDir) == null) - offlineLogDirQueue.add(logDir) - } - - /* - * Get the next offline log dir from logDirFailureEvent queue. - * The method will wait if necessary until a new offline log directory becomes available - */ - def takeNextOfflineLogDir(): String = offlineLogDirQueue.take() - -} diff --git a/core/src/main/scala/kafka/server/MetadataCache.scala b/core/src/main/scala/kafka/server/MetadataCache.scala index e0501ef1ebef9..342b23cec4bf0 100755 --- a/core/src/main/scala/kafka/server/MetadataCache.scala +++ b/core/src/main/scala/kafka/server/MetadataCache.scala @@ -32,6 +32,10 @@ case class FinalizedFeaturesAndEpoch(features: Map[String, Short], epoch: Long) } } +/** + * Used to represent the controller id cached in the metadata cache of the broker. This trait is + * extended to represent if the controller is KRaft controller or Zk controller. + */ sealed trait CachedControllerId { val id: Int } @@ -118,10 +122,9 @@ object MetadataCache { def zkMetadataCache(brokerId: Int, metadataVersion: MetadataVersion, brokerFeatures: BrokerFeatures = BrokerFeatures.createEmpty(), - kraftControllerNodes: collection.Seq[Node] = null) + kraftControllerNodes: collection.Seq[Node] = collection.Seq.empty[Node]) : ZkMetadataCache = { - new ZkMetadataCache(brokerId, metadataVersion, brokerFeatures, - Option(kraftControllerNodes).getOrElse(collection.Seq.empty[Node])) + new ZkMetadataCache(brokerId, metadataVersion, brokerFeatures, kraftControllerNodes) } def kRaftMetadataCache(brokerId: Int): KRaftMetadataCache = { diff --git a/core/src/main/scala/kafka/server/PartitionMetadataFile.scala b/core/src/main/scala/kafka/server/PartitionMetadataFile.scala index f88a4cc90752a..d0d4552ade5ac 100644 --- a/core/src/main/scala/kafka/server/PartitionMetadataFile.scala +++ b/core/src/main/scala/kafka/server/PartitionMetadataFile.scala @@ -26,7 +26,7 @@ import kafka.utils.Logging import org.apache.kafka.common.Uuid import org.apache.kafka.common.errors.{InconsistentTopicIdException, KafkaStorageException} import org.apache.kafka.common.utils.Utils - +import org.apache.kafka.server.log.internals.LogDirFailureChannel object PartitionMetadataFile { diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 7b56b6d673249..c88cc68869388 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -24,7 +24,6 @@ import java.util.concurrent.locks.Lock import com.yammer.metrics.core.Meter import kafka.api._ import kafka.cluster.{BrokerEndPoint, Partition} -import kafka.common.RecordValidationException import kafka.controller.{KafkaController, StateChangeLogger} import kafka.log._ import kafka.log.remote.RemoteLogManager @@ -62,6 +61,7 @@ import org.apache.kafka.common.utils.Time import org.apache.kafka.image.{LocalReplicaChanges, MetadataImage, TopicsDelta} import org.apache.kafka.metadata.LeaderConstants.NO_LEADER import org.apache.kafka.server.common.MetadataVersion._ +import org.apache.kafka.server.log.internals.{AppendOrigin, LogDirFailureChannel, RecordValidationException} import java.nio.file.{Files, Paths} import java.util @@ -984,7 +984,7 @@ class ReplicaManager(val config: KafkaConfig, val logStartOffset = processFailedRecord(topicPartition, rve.invalidException) val recordErrors = rve.recordErrors (topicPartition, LogAppendResult(LogAppendInfo.unknownLogAppendInfoWithAdditionalInfo( - logStartOffset, recordErrors, rve.invalidException.getMessage), Some(rve.invalidException))) + logStartOffset, recordErrors.asScala, rve.invalidException.getMessage), Some(rve.invalidException))) case t: Throwable => val logStartOffset = processFailedRecord(topicPartition, t) (topicPartition, LogAppendResult(LogAppendInfo.unknownLogAppendInfoWithLogStartOffset(logStartOffset), Some(t))) diff --git a/core/src/main/scala/kafka/server/checkpoints/CheckpointFileWithFailureHandler.scala b/core/src/main/scala/kafka/server/checkpoints/CheckpointFileWithFailureHandler.scala index 7021c6742caea..0e669249bddca 100644 --- a/core/src/main/scala/kafka/server/checkpoints/CheckpointFileWithFailureHandler.scala +++ b/core/src/main/scala/kafka/server/checkpoints/CheckpointFileWithFailureHandler.scala @@ -16,9 +16,9 @@ */ package kafka.server.checkpoints -import kafka.server.LogDirFailureChannel import org.apache.kafka.common.errors.KafkaStorageException import org.apache.kafka.server.common.CheckpointFile +import org.apache.kafka.server.log.internals.LogDirFailureChannel import CheckpointFile.EntryFormatter import java.io._ diff --git a/core/src/main/scala/kafka/server/checkpoints/LeaderEpochCheckpointFile.scala b/core/src/main/scala/kafka/server/checkpoints/LeaderEpochCheckpointFile.scala index c772b82dece98..93ef93d0bd3b7 100644 --- a/core/src/main/scala/kafka/server/checkpoints/LeaderEpochCheckpointFile.scala +++ b/core/src/main/scala/kafka/server/checkpoints/LeaderEpochCheckpointFile.scala @@ -16,9 +16,9 @@ */ package kafka.server.checkpoints -import kafka.server.LogDirFailureChannel import kafka.server.epoch.EpochEntry import org.apache.kafka.server.common.CheckpointFile.EntryFormatter +import org.apache.kafka.server.log.internals.LogDirFailureChannel import java.io._ import java.util.Optional diff --git a/core/src/main/scala/kafka/server/checkpoints/OffsetCheckpointFile.scala b/core/src/main/scala/kafka/server/checkpoints/OffsetCheckpointFile.scala index f7b83eae6af68..483a186c8ddba 100644 --- a/core/src/main/scala/kafka/server/checkpoints/OffsetCheckpointFile.scala +++ b/core/src/main/scala/kafka/server/checkpoints/OffsetCheckpointFile.scala @@ -16,10 +16,10 @@ */ package kafka.server.checkpoints -import kafka.server.LogDirFailureChannel import kafka.server.epoch.EpochEntry import org.apache.kafka.common.TopicPartition import org.apache.kafka.server.common.CheckpointFile.EntryFormatter +import org.apache.kafka.server.log.internals.LogDirFailureChannel import java.io._ import java.util.Optional diff --git a/core/src/main/scala/kafka/tools/DumpLogSegments.scala b/core/src/main/scala/kafka/tools/DumpLogSegments.scala index a4914182588c0..3302bfe91b485 100755 --- a/core/src/main/scala/kafka/tools/DumpLogSegments.scala +++ b/core/src/main/scala/kafka/tools/DumpLogSegments.scala @@ -32,7 +32,7 @@ import org.apache.kafka.common.record._ import org.apache.kafka.common.utils.Utils import org.apache.kafka.metadata.MetadataRecordSerde import org.apache.kafka.metadata.bootstrap.BootstrapDirectory -import org.apache.kafka.server.log.internals.{OffsetIndex, TransactionIndex} +import org.apache.kafka.server.log.internals.{OffsetIndex, TimeIndex, TransactionIndex} import org.apache.kafka.snapshot.Snapshots import scala.jdk.CollectionConverters._ @@ -172,7 +172,7 @@ object DumpLogSegments { val fileRecords = FileRecords.open(logFile, false) val indexFile = new File(file.getAbsoluteFile.getParent, file.getName.split("\\.")(0) + UnifiedLog.IndexFileSuffix) val index = new OffsetIndex(indexFile, startOffset, -1, false) - val timeIndex = new TimeIndex(file, baseOffset = startOffset, writable = false) + val timeIndex = new TimeIndex(file, startOffset, -1, false) try { //Check that index passes sanityCheck, this is the check that determines if indexes will be rebuilt on startup or not. diff --git a/core/src/main/scala/kafka/utils/CoreUtils.scala b/core/src/main/scala/kafka/utils/CoreUtils.scala index 96a71c2851838..40beb6ba04320 100755 --- a/core/src/main/scala/kafka/utils/CoreUtils.scala +++ b/core/src/main/scala/kafka/utils/CoreUtils.scala @@ -212,16 +212,6 @@ object CoreUtils { s.substring(0, s.length - oldSuffix.length) + newSuffix } - /** - * Read a big-endian integer from a byte array - */ - def readInt(bytes: Array[Byte], offset: Int): Int = { - ((bytes(offset) & 0xFF) << 24) | - ((bytes(offset + 1) & 0xFF) << 16) | - ((bytes(offset + 2) & 0xFF) << 8) | - (bytes(offset + 3) & 0xFF) - } - /** * Execute the given function inside the lock */ diff --git a/core/src/test/scala/integration/kafka/api/TransactionsExpirationTest.scala b/core/src/test/scala/integration/kafka/api/TransactionsExpirationTest.scala index 79fc67e4c5013..ee5a7aebd7a95 100644 --- a/core/src/test/scala/integration/kafka/api/TransactionsExpirationTest.scala +++ b/core/src/test/scala/integration/kafka/api/TransactionsExpirationTest.scala @@ -129,9 +129,11 @@ class TransactionsExpirationTest extends KafkaServerTestHarness { producer.flush() // Ensure producer IDs are added. - val pState = producerState - assertEquals(1, pState.size) - val oldProducerId = pState(0).producerId + var pState : List[ProducerState] = null + TestUtils.waitUntilTrue(() => { pState = producerState; pState.nonEmpty}, "Producer IDs for topic1 did not propagate quickly") + assertEquals(1, pState.size, "Unexpected producer to topic1") + val oldProducerId = pState.head.producerId + val oldProducerEpoch = pState.head.producerEpoch producer.abortTransaction() @@ -150,12 +152,16 @@ class TransactionsExpirationTest extends KafkaServerTestHarness { producer.commitTransaction() // Producer IDs should repopulate. - val pState2 = producerState - assertEquals(1, pState2.size) - val newProducerId = pState2(0).producerId - - // Producer IDs should be the same. + var pState2 : List[ProducerState] = null + TestUtils.waitUntilTrue(() => {pState2 = producerState; pState2.nonEmpty}, "Producer IDs for topic1 did not propagate quickly") + assertEquals(1, pState2.size, "Unexpected producer to topic1") + val newProducerId = pState2.head.producerId + val newProducerEpoch = pState2.head.producerEpoch + + // Because the transaction IDs outlive the producer IDs, creating a producer with the same transactional id + // soon after the first will re-use the same producerId, while bumping the epoch to indicate that they are distinct. assertEquals(oldProducerId, newProducerId) + assertEquals(oldProducerEpoch + 1, newProducerEpoch) consumer.subscribe(List(topic1).asJava) @@ -205,9 +211,9 @@ class TransactionsExpirationTest extends KafkaServerTestHarness { serverProps.put(KafkaConfig.AutoLeaderRebalanceEnableProp, false.toString) serverProps.put(KafkaConfig.GroupInitialRebalanceDelayMsProp, "0") serverProps.put(KafkaConfig.TransactionsAbortTimedOutTransactionCleanupIntervalMsProp, "200") - serverProps.put(KafkaConfig.TransactionalIdExpirationMsProp, "1000") + serverProps.put(KafkaConfig.TransactionalIdExpirationMsProp, "10000") serverProps.put(KafkaConfig.TransactionsRemoveExpiredTransactionalIdCleanupIntervalMsProp, "500") - serverProps.put(KafkaConfig.ProducerIdExpirationMsProp, "500") + serverProps.put(KafkaConfig.ProducerIdExpirationMsProp, "5000") serverProps.put(KafkaConfig.ProducerIdExpirationCheckIntervalMsProp, "500") serverProps } diff --git a/core/src/test/scala/kafka/server/BrokerToControllerRequestThreadTest.scala b/core/src/test/scala/kafka/server/BrokerToControllerRequestThreadTest.scala index eea5c7517a0b5..d62bf74780aa0 100644 --- a/core/src/test/scala/kafka/server/BrokerToControllerRequestThreadTest.scala +++ b/core/src/test/scala/kafka/server/BrokerToControllerRequestThreadTest.scala @@ -57,8 +57,9 @@ class BrokerToControllerRequestThreadTest { when(controllerNodeProvider.getControllerInfo()).thenReturn(emptyControllerInfo) val retryTimeoutMs = 30000 - val testRequestThread = new BrokerToControllerRequestThread(_ => mockClient, new ManualMetadataUpdater(), controllerNodeProvider, - config, time, "", retryTimeoutMs) + val testRequestThread = new BrokerToControllerRequestThread( + mockClient, isNetworkClientForZkController = true, _ => mockClient, new ManualMetadataUpdater(), + controllerNodeProvider, config, time, "", retryTimeoutMs) testRequestThread.started = true val completionHandler = new TestControllerRequestCompletionHandler(None) @@ -94,8 +95,9 @@ class BrokerToControllerRequestThreadTest { when(controllerNodeProvider.getControllerInfo()).thenReturn(controllerInfo(Some(activeController))) val expectedResponse = RequestTestUtils.metadataUpdateWith(2, Collections.singletonMap("a", 2)) - val testRequestThread = new BrokerToControllerRequestThread(_ => mockClient, new ManualMetadataUpdater(), controllerNodeProvider, - config, time, "", retryTimeoutMs = Long.MaxValue) + val testRequestThread = new BrokerToControllerRequestThread( + mockClient, isNetworkClientForZkController = true, _ => mockClient, new ManualMetadataUpdater(), + controllerNodeProvider, config, time, "", retryTimeoutMs = Long.MaxValue) testRequestThread.started = true mockClient.prepareResponse(expectedResponse) @@ -134,10 +136,11 @@ class BrokerToControllerRequestThreadTest { val newController = new Node(newControllerId, "host2", 1234) when(controllerNodeProvider.getControllerInfo()).thenReturn( - emptyControllerInfo, controllerInfo(Some(oldController)), controllerInfo(Some(newController))) + controllerInfo(Some(oldController)), controllerInfo(Some(newController))) val expectedResponse = RequestTestUtils.metadataUpdateWith(3, Collections.singletonMap("a", 2)) - val testRequestThread = new BrokerToControllerRequestThread(_ => mockClient, new ManualMetadataUpdater(), + val testRequestThread = new BrokerToControllerRequestThread( + mockClient, isNetworkClientForZkController = true, _ => mockClient, new ManualMetadataUpdater(), controllerNodeProvider, config, time, "", retryTimeoutMs = Long.MaxValue) testRequestThread.started = true @@ -182,14 +185,15 @@ class BrokerToControllerRequestThreadTest { val newController = new Node(newControllerId, "host2", port) when(controllerNodeProvider.getControllerInfo()).thenReturn( - emptyControllerInfo, controllerInfo(Some(oldController)), controllerInfo(Some(newController))) + controllerInfo(Some(oldController)), controllerInfo(Some(newController))) val responseWithNotControllerError = RequestTestUtils.metadataUpdateWith("cluster1", 2, Collections.singletonMap("a", Errors.NOT_CONTROLLER), Collections.singletonMap("a", 2)) val expectedResponse = RequestTestUtils.metadataUpdateWith(3, Collections.singletonMap("a", 2)) - val testRequestThread = new BrokerToControllerRequestThread(_ => mockClient, new ManualMetadataUpdater(), controllerNodeProvider, - config, time, "", retryTimeoutMs = Long.MaxValue) + val testRequestThread = new BrokerToControllerRequestThread( + mockClient, isNetworkClientForZkController = true,_ => mockClient, new ManualMetadataUpdater(), + controllerNodeProvider, config, time, "", retryTimeoutMs = Long.MaxValue) testRequestThread.started = true val completionHandler = new TestControllerRequestCompletionHandler(Some(expectedResponse)) @@ -243,7 +247,6 @@ class BrokerToControllerRequestThreadTest { val newController = new Node(newControllerId, "host2", port) when(controllerNodeProvider.getControllerInfo()).thenReturn( - emptyControllerInfo, // call to create network client. controllerInfo(Some(oldController)), controllerInfo(Some(newController)) ) @@ -255,8 +258,9 @@ class BrokerToControllerRequestThreadTest { // response for retry request after receiving NOT_CONTROLLER error val expectedResponse = RequestTestUtils.metadataUpdateWith(3, Collections.singletonMap("a", 2)) - val testRequestThread = new BrokerToControllerRequestThread(_ => mockClient, new ManualMetadataUpdater(), controllerNodeProvider, - config, time, "", retryTimeoutMs = Long.MaxValue) + val testRequestThread = new BrokerToControllerRequestThread( + mockClient, isNetworkClientForZkController = true, _ => mockClient, new ManualMetadataUpdater(), + controllerNodeProvider, config, time, "", retryTimeoutMs = Long.MaxValue) testRequestThread.started = true val completionHandler = new TestControllerRequestCompletionHandler(Some(expectedResponse)) @@ -311,14 +315,15 @@ class BrokerToControllerRequestThreadTest { val controllerNodeProvider = mock(classOf[ControllerNodeProvider]) val controller = new Node(controllerId, "host1", 1234) - when(controllerNodeProvider.getControllerInfo()).thenReturn(emptyControllerInfo, controllerInfo(Some(controller))) + when(controllerNodeProvider.getControllerInfo()).thenReturn(controllerInfo(Some(controller))) val retryTimeoutMs = 30000 val responseWithNotControllerError = RequestTestUtils.metadataUpdateWith("cluster1", 2, Collections.singletonMap("a", Errors.NOT_CONTROLLER), Collections.singletonMap("a", 2)) - val testRequestThread = new BrokerToControllerRequestThread(_ => mockClient, new ManualMetadataUpdater(), controllerNodeProvider, - config, time, "", retryTimeoutMs) + val testRequestThread = new BrokerToControllerRequestThread( + mockClient, isNetworkClientForZkController = true,_ => mockClient, new ManualMetadataUpdater(), + controllerNodeProvider, config, time, "", retryTimeoutMs) testRequestThread.started = true val completionHandler = new TestControllerRequestCompletionHandler() @@ -375,8 +380,9 @@ class BrokerToControllerRequestThreadTest { mockClient.prepareUnsupportedVersionResponse(request => request.apiKey == ApiKeys.METADATA) - val testRequestThread = new BrokerToControllerRequestThread(_ => mockClient, new ManualMetadataUpdater(), controllerNodeProvider, - config, time, "", retryTimeoutMs = Long.MaxValue) + val testRequestThread = new BrokerToControllerRequestThread( + mockClient, isNetworkClientForZkController = true, _ => mockClient, new ManualMetadataUpdater(), + controllerNodeProvider, config, time, "", retryTimeoutMs = Long.MaxValue) testRequestThread.started = true testRequestThread.enqueue(queueItem) @@ -412,8 +418,9 @@ class BrokerToControllerRequestThreadTest { mockClient.createPendingAuthenticationError(activeController, 50) - val testRequestThread = new BrokerToControllerRequestThread(_ => mockClient, new ManualMetadataUpdater(), controllerNodeProvider, - config, time, "", retryTimeoutMs = Long.MaxValue) + val testRequestThread = new BrokerToControllerRequestThread( + mockClient, isNetworkClientForZkController = true, _ => mockClient, new ManualMetadataUpdater(), + controllerNodeProvider, config, time, "", retryTimeoutMs = Long.MaxValue) testRequestThread.started = true testRequestThread.enqueue(queueItem) @@ -433,8 +440,9 @@ class BrokerToControllerRequestThreadTest { val controllerNodeProvider = mock(classOf[ControllerNodeProvider]) when(controllerNodeProvider.getControllerInfo()).thenReturn(emptyControllerInfo) - val testRequestThread = new BrokerToControllerRequestThread(_ => mockClient, new ManualMetadataUpdater(), controllerNodeProvider, - config, time, "", retryTimeoutMs = Long.MaxValue) + val testRequestThread = new BrokerToControllerRequestThread( + mockClient, isNetworkClientForZkController = true, _ => mockClient, new ManualMetadataUpdater(), + controllerNodeProvider, config, time, "", retryTimeoutMs = Long.MaxValue) val completionHandler = new TestControllerRequestCompletionHandler(None) val queueItem = BrokerToControllerQueueItem( diff --git a/core/src/test/scala/kafka/server/LocalLeaderEndPointTest.scala b/core/src/test/scala/kafka/server/LocalLeaderEndPointTest.scala index ba2d2c43c219b..08734b2e97abf 100644 --- a/core/src/test/scala/kafka/server/LocalLeaderEndPointTest.scala +++ b/core/src/test/scala/kafka/server/LocalLeaderEndPointTest.scala @@ -18,7 +18,6 @@ package kafka.server import kafka.cluster.BrokerEndPoint -import kafka.log.AppendOrigin import kafka.server.checkpoints.LazyOffsetCheckpoints import kafka.utils.{MockScheduler, MockTime, TestUtils} import org.apache.kafka.common.{Node, TopicPartition, Uuid} @@ -28,6 +27,7 @@ import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.record.{CompressionType, MemoryRecords, SimpleRecord} import org.apache.kafka.common.requests.LeaderAndIsrRequest import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse +import org.apache.kafka.server.log.internals.{AppendOrigin, LogDirFailureChannel} import org.junit.jupiter.api.{BeforeEach, Test} import org.junit.jupiter.api.Assertions._ import org.mockito.Mockito.mock @@ -158,7 +158,7 @@ class LocalLeaderEndPointTest { private def appendRecords(replicaManager: ReplicaManager, partition: TopicPartition, records: MemoryRecords, - origin: AppendOrigin = AppendOrigin.Client, + origin: AppendOrigin = AppendOrigin.CLIENT, requiredAcks: Short = -1): CallbackResult[PartitionResponse] = { val result = new CallbackResult[PartitionResponse]() def appendCallback(responses: Map[TopicPartition, PartitionResponse]): Unit = { @@ -189,4 +189,4 @@ class LocalLeaderEndPointTest { new SimpleRecord("third message".getBytes()), ) } -} \ No newline at end of file +} diff --git a/core/src/test/scala/other/kafka/StressTestLog.scala b/core/src/test/scala/other/kafka/StressTestLog.scala index de5a65a41ec5c..5492a15febabf 100755 --- a/core/src/test/scala/other/kafka/StressTestLog.scala +++ b/core/src/test/scala/other/kafka/StressTestLog.scala @@ -21,11 +21,12 @@ import java.util.Properties import java.util.concurrent.atomic._ import kafka.log._ -import kafka.server.{BrokerTopicStats, FetchLogEnd, LogDirFailureChannel} +import kafka.server.{BrokerTopicStats, FetchLogEnd} import kafka.utils._ import org.apache.kafka.clients.consumer.OffsetOutOfRangeException import org.apache.kafka.common.record.FileRecords import org.apache.kafka.common.utils.Utils +import org.apache.kafka.server.log.internals.LogDirFailureChannel /** * A stress test that instantiates a log and then runs continual appends against it from one thread and continual reads against it diff --git a/core/src/test/scala/other/kafka/TestLinearWriteSpeed.scala b/core/src/test/scala/other/kafka/TestLinearWriteSpeed.scala index 026afd0ddd222..ab681e690afd6 100755 --- a/core/src/test/scala/other/kafka/TestLinearWriteSpeed.scala +++ b/core/src/test/scala/other/kafka/TestLinearWriteSpeed.scala @@ -25,10 +25,11 @@ import java.util.{Properties, Random} import joptsimple._ import kafka.log._ -import kafka.server.{BrokerTopicStats, LogDirFailureChannel} +import kafka.server.BrokerTopicStats import kafka.utils._ import org.apache.kafka.common.record._ import org.apache.kafka.common.utils.{Time, Utils} +import org.apache.kafka.server.log.internals.LogDirFailureChannel import scala.math._ diff --git a/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala b/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala index d627ce54a0342..22f2e75cb5817 100644 --- a/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala @@ -35,6 +35,7 @@ import org.apache.kafka.common.requests.FetchRequest import org.apache.kafka.common.utils.Utils import org.apache.kafka.common.{TopicPartition, Uuid} import org.apache.kafka.server.common.MetadataVersion +import org.apache.kafka.server.log.internals.{AppendOrigin, LogDirFailureChannel} import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertTrue} import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} import org.mockito.ArgumentMatchers @@ -369,7 +370,7 @@ class PartitionLockTest extends Logging { (0 until numRecords).foreach { _ => val batch = TestUtils.records(records = List(new SimpleRecord("k1".getBytes, "v1".getBytes), new SimpleRecord("k2".getBytes, "v2".getBytes))) - partition.appendRecordsToLeader(batch, origin = AppendOrigin.Client, requiredAcks = 0, requestLocal) + partition.appendRecordsToLeader(batch, origin = AppendOrigin.CLIENT, requiredAcks = 0, requestLocal) } } diff --git a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala index 82ffee42d035a..2c52888bdbed7 100644 --- a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala @@ -54,6 +54,7 @@ import org.apache.kafka.common.replica.ClientMetadata.DefaultClientMetadata import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol} import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.server.common.MetadataVersion.IBP_2_6_IV0 +import org.apache.kafka.server.log.internals.{AppendOrigin, LogDirFailureChannel} import org.apache.kafka.server.metrics.KafkaYammerMetrics import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.ValueSource @@ -740,8 +741,8 @@ class PartitionTest extends AbstractPartitionTest { val requestLocal = RequestLocal.withThreadConfinedCaching // after makeLeader(() call, partition should know about all the replicas // append records with initial leader epoch - partition.appendRecordsToLeader(batch1, origin = AppendOrigin.Client, requiredAcks = 0, requestLocal) - partition.appendRecordsToLeader(batch2, origin = AppendOrigin.Client, requiredAcks = 0, requestLocal) + partition.appendRecordsToLeader(batch1, origin = AppendOrigin.CLIENT, requiredAcks = 0, requestLocal) + partition.appendRecordsToLeader(batch2, origin = AppendOrigin.CLIENT, requiredAcks = 0, requestLocal) assertEquals(partition.localLogOrException.logStartOffset, partition.localLogOrException.highWatermark, "Expected leader's HW not move") @@ -949,7 +950,7 @@ class PartitionTest extends AbstractPartitionTest { new SimpleRecord("k2".getBytes, "v2".getBytes), new SimpleRecord("k3".getBytes, "v3".getBytes)), baseOffset = 0L) - partition.appendRecordsToLeader(records, origin = AppendOrigin.Client, requiredAcks = 0, RequestLocal.withThreadConfinedCaching) + partition.appendRecordsToLeader(records, origin = AppendOrigin.CLIENT, requiredAcks = 0, RequestLocal.withThreadConfinedCaching) def fetchOffset(isolationLevel: Option[IsolationLevel], timestamp: Long): TimestampAndOffset = { val res = partition.fetchOffsetForTimestamp(timestamp, @@ -1075,9 +1076,9 @@ class PartitionTest extends AbstractPartitionTest { // after makeLeader(() call, partition should know about all the replicas // append records with initial leader epoch - val lastOffsetOfFirstBatch = partition.appendRecordsToLeader(batch1, origin = AppendOrigin.Client, + val lastOffsetOfFirstBatch = partition.appendRecordsToLeader(batch1, origin = AppendOrigin.CLIENT, requiredAcks = 0, requestLocal).lastOffset - partition.appendRecordsToLeader(batch2, origin = AppendOrigin.Client, requiredAcks = 0, requestLocal) + partition.appendRecordsToLeader(batch2, origin = AppendOrigin.CLIENT, requiredAcks = 0, requestLocal) assertEquals(partition.localLogOrException.logStartOffset, partition.log.get.highWatermark, "Expected leader's HW not move") // let the follower in ISR move leader's HW to move further but below LEO @@ -1109,7 +1110,7 @@ class PartitionTest extends AbstractPartitionTest { val currentLeaderEpochStartOffset = partition.localLogOrException.logEndOffset // append records with the latest leader epoch - partition.appendRecordsToLeader(batch3, origin = AppendOrigin.Client, requiredAcks = 0, requestLocal) + partition.appendRecordsToLeader(batch3, origin = AppendOrigin.CLIENT, requiredAcks = 0, requestLocal) // fetch from follower not in ISR from log start offset should not add this follower to ISR fetchFollower(partition, replicaId = follower1, fetchOffset = 0) diff --git a/core/src/test/scala/unit/kafka/coordinator/AbstractCoordinatorConcurrencyTest.scala b/core/src/test/scala/unit/kafka/coordinator/AbstractCoordinatorConcurrencyTest.scala index d741508231fdf..326ca5101ce0d 100644 --- a/core/src/test/scala/unit/kafka/coordinator/AbstractCoordinatorConcurrencyTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/AbstractCoordinatorConcurrencyTest.scala @@ -21,9 +21,8 @@ import java.util.concurrent.{ConcurrentHashMap, Executors} import java.util.{Collections, Random} import java.util.concurrent.atomic.AtomicInteger import java.util.concurrent.locks.Lock - import kafka.coordinator.AbstractCoordinatorConcurrencyTest._ -import kafka.log.{AppendOrigin, LogConfig, UnifiedLog} +import kafka.log.{LogConfig, UnifiedLog} import kafka.server._ import kafka.utils._ import kafka.utils.timer.MockTimer @@ -32,6 +31,7 @@ import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.record.{MemoryRecords, RecordBatch, RecordConversionStats} import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse +import org.apache.kafka.server.log.internals.AppendOrigin import org.junit.jupiter.api.{AfterEach, BeforeEach} import org.mockito.Mockito.{mock, withSettings, CALLS_REAL_METHODS} diff --git a/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala b/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala index be1f90bc7354d..5ee68ef3505ee 100644 --- a/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala @@ -31,13 +31,13 @@ import org.apache.kafka.common.requests.{JoinGroupRequest, OffsetCommitRequest, import java.util.concurrent.TimeUnit import java.util.concurrent.locks.ReentrantLock import kafka.cluster.Partition -import kafka.log.AppendOrigin import kafka.zk.KafkaZkClient import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.Subscription import org.apache.kafka.clients.consumer.internals.ConsumerProtocol import org.apache.kafka.common.internals.Topic import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.message.LeaveGroupRequestData.MemberIdentity +import org.apache.kafka.server.log.internals.AppendOrigin import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} import org.junit.jupiter.params.ParameterizedTest @@ -3854,7 +3854,7 @@ class GroupCoordinatorTest { when(replicaManager.appendRecords(anyLong, anyShort(), internalTopicsAllowed = ArgumentMatchers.eq(true), - origin = ArgumentMatchers.eq(AppendOrigin.Coordinator), + origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR), any[Map[TopicPartition, MemoryRecords]], capturedArgument.capture(), any[Option[ReentrantLock]], @@ -3888,7 +3888,7 @@ class GroupCoordinatorTest { when(replicaManager.appendRecords(anyLong, anyShort(), internalTopicsAllowed = ArgumentMatchers.eq(true), - origin = ArgumentMatchers.eq(AppendOrigin.Coordinator), + origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR), any[Map[TopicPartition, MemoryRecords]], capturedArgument.capture(), any[Option[ReentrantLock]], @@ -4032,7 +4032,7 @@ class GroupCoordinatorTest { when(replicaManager.appendRecords(anyLong, anyShort(), internalTopicsAllowed = ArgumentMatchers.eq(true), - origin = ArgumentMatchers.eq(AppendOrigin.Coordinator), + origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR), any[Map[TopicPartition, MemoryRecords]], capturedArgument.capture(), any[Option[ReentrantLock]], @@ -4065,7 +4065,7 @@ class GroupCoordinatorTest { when(replicaManager.appendRecords(anyLong, anyShort(), internalTopicsAllowed = ArgumentMatchers.eq(true), - origin = ArgumentMatchers.eq(AppendOrigin.Coordinator), + origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR), any[Map[TopicPartition, MemoryRecords]], capturedArgument.capture(), any[Option[ReentrantLock]], diff --git a/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala b/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala index 16cdca0591d63..94be08c671fef 100644 --- a/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala @@ -21,12 +21,12 @@ import java.lang.management.ManagementFactory import java.nio.ByteBuffer import java.util.concurrent.locks.ReentrantLock import java.util.{Collections, Optional} - import com.yammer.metrics.core.Gauge + import javax.management.ObjectName import kafka.cluster.Partition import kafka.common.OffsetAndMetadata -import kafka.log.{AppendOrigin, LogAppendInfo, UnifiedLog} +import kafka.log.{LogAppendInfo, UnifiedLog} import kafka.server.{FetchDataInfo, FetchLogEnd, HostedPartition, KafkaConfig, LogOffsetMetadata, ReplicaManager, RequestLocal} import kafka.utils.{KafkaScheduler, MockTime, TestUtils} import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor @@ -42,6 +42,7 @@ import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse import org.apache.kafka.common.utils.Utils import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.server.common.MetadataVersion._ +import org.apache.kafka.server.log.internals.AppendOrigin import org.apache.kafka.server.metrics.KafkaYammerMetrics import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} @@ -1172,7 +1173,7 @@ class GroupMetadataManagerTest { verify(replicaManager).appendRecords(anyLong(), anyShort(), internalTopicsAllowed = ArgumentMatchers.eq(true), - origin = ArgumentMatchers.eq(AppendOrigin.Coordinator), + origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR), any(), any(), any[Option[ReentrantLock]], @@ -1208,7 +1209,7 @@ class GroupMetadataManagerTest { verify(replicaManager).appendRecords(anyLong(), anyShort(), internalTopicsAllowed = ArgumentMatchers.eq(true), - origin = ArgumentMatchers.eq(AppendOrigin.Coordinator), + origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR), any(), any(), any[Option[ReentrantLock]], @@ -1282,7 +1283,7 @@ class GroupMetadataManagerTest { verify(replicaManager).appendRecords(anyLong(), anyShort(), internalTopicsAllowed = ArgumentMatchers.eq(true), - origin = ArgumentMatchers.eq(AppendOrigin.Coordinator), + origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR), any(), any(), any[Option[ReentrantLock]], @@ -1322,7 +1323,7 @@ class GroupMetadataManagerTest { verify(replicaManager).appendRecords(anyLong(), anyShort(), internalTopicsAllowed = ArgumentMatchers.eq(true), - origin = ArgumentMatchers.eq(AppendOrigin.Coordinator), + origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR), any[Map[TopicPartition, MemoryRecords]], capturedResponseCallback.capture(), any[Option[ReentrantLock]], @@ -1380,7 +1381,7 @@ class GroupMetadataManagerTest { verify(replicaManager).appendRecords(anyLong(), anyShort(), internalTopicsAllowed = ArgumentMatchers.eq(true), - origin = ArgumentMatchers.eq(AppendOrigin.Coordinator), + origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR), any[Map[TopicPartition, MemoryRecords]], any(), any[Option[ReentrantLock]], @@ -1428,7 +1429,7 @@ class GroupMetadataManagerTest { verify(replicaManager).appendRecords(anyLong(), anyShort(), internalTopicsAllowed = ArgumentMatchers.eq(true), - origin = ArgumentMatchers.eq(AppendOrigin.Coordinator), + origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR), any[Map[TopicPartition, MemoryRecords]], any(), any[Option[ReentrantLock]], @@ -1561,7 +1562,7 @@ class GroupMetadataManagerTest { verify(replicaManager).appendRecords(anyLong(), anyShort(), internalTopicsAllowed = ArgumentMatchers.eq(true), - origin = ArgumentMatchers.eq(AppendOrigin.Coordinator), + origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR), any[Map[TopicPartition, MemoryRecords]], any(), any[Option[ReentrantLock]], @@ -1640,7 +1641,7 @@ class GroupMetadataManagerTest { time.sleep(2) when(partition.appendRecordsToLeader(any[MemoryRecords], - origin = ArgumentMatchers.eq(AppendOrigin.Coordinator), requiredAcks = anyInt(), + origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR), requiredAcks = anyInt(), any())).thenReturn(LogAppendInfo.UnknownLogAppendInfo) groupMetadataManager.cleanupGroupMetadata() @@ -1655,7 +1656,7 @@ class GroupMetadataManagerTest { verify(replicaManager).appendRecords(anyLong(), anyShort(), internalTopicsAllowed = ArgumentMatchers.eq(true), - origin = ArgumentMatchers.eq(AppendOrigin.Coordinator), + origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR), any(), any(), any[Option[ReentrantLock]], @@ -1681,7 +1682,7 @@ class GroupMetadataManagerTest { when(replicaManager.getMagic(any())).thenReturn(Some(RecordBatch.CURRENT_MAGIC_VALUE)) mockGetPartition() when(partition.appendRecordsToLeader(recordsCapture.capture(), - origin = ArgumentMatchers.eq(AppendOrigin.Coordinator), requiredAcks = anyInt(), + origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR), requiredAcks = anyInt(), any())).thenReturn(LogAppendInfo.UnknownLogAppendInfo) groupMetadataManager.cleanupGroupMetadata() @@ -1724,7 +1725,7 @@ class GroupMetadataManagerTest { when(replicaManager.getMagic(any())).thenReturn(Some(RecordBatch.CURRENT_MAGIC_VALUE)) mockGetPartition() when(partition.appendRecordsToLeader(recordsCapture.capture(), - origin = ArgumentMatchers.eq(AppendOrigin.Coordinator), requiredAcks = anyInt(), + origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR), requiredAcks = anyInt(), any())).thenReturn(LogAppendInfo.UnknownLogAppendInfo) groupMetadataManager.cleanupGroupMetadata() @@ -1792,7 +1793,7 @@ class GroupMetadataManagerTest { val recordsCapture: ArgumentCaptor[MemoryRecords] = ArgumentCaptor.forClass(classOf[MemoryRecords]) when(partition.appendRecordsToLeader(recordsCapture.capture(), - origin = ArgumentMatchers.eq(AppendOrigin.Coordinator), requiredAcks = anyInt(), + origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR), requiredAcks = anyInt(), any())).thenReturn(LogAppendInfo.UnknownLogAppendInfo) groupMetadataManager.cleanupGroupMetadata() @@ -1886,7 +1887,7 @@ class GroupMetadataManagerTest { // expect the offset tombstone when(partition.appendRecordsToLeader(any[MemoryRecords], - origin = ArgumentMatchers.eq(AppendOrigin.Coordinator), requiredAcks = anyInt(), + origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR), requiredAcks = anyInt(), any())).thenReturn(LogAppendInfo.UnknownLogAppendInfo) groupMetadataManager.cleanupGroupMetadata() @@ -1907,7 +1908,7 @@ class GroupMetadataManagerTest { // expect the offset tombstone when(partition.appendRecordsToLeader(any[MemoryRecords], - origin = ArgumentMatchers.eq(AppendOrigin.Coordinator), requiredAcks = anyInt(), + origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR), requiredAcks = anyInt(), any())).thenReturn(LogAppendInfo.UnknownLogAppendInfo) groupMetadataManager.cleanupGroupMetadata() @@ -1947,7 +1948,7 @@ class GroupMetadataManagerTest { // expect the offset tombstone when(partition.appendRecordsToLeader(any[MemoryRecords], - origin = ArgumentMatchers.eq(AppendOrigin.Coordinator), requiredAcks = anyInt(), + origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR), requiredAcks = anyInt(), any())).thenReturn(LogAppendInfo.UnknownLogAppendInfo) groupMetadataManager.cleanupGroupMetadata() @@ -2020,7 +2021,7 @@ class GroupMetadataManagerTest { // expect the offset tombstone when(partition.appendRecordsToLeader(any[MemoryRecords], - origin = ArgumentMatchers.eq(AppendOrigin.Coordinator), requiredAcks = anyInt(), + origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR), requiredAcks = anyInt(), any())).thenReturn(LogAppendInfo.UnknownLogAppendInfo) groupMetadataManager.cleanupGroupMetadata() @@ -2144,13 +2145,13 @@ class GroupMetadataManagerTest { // expect the offset tombstone when(partition.appendRecordsToLeader(any[MemoryRecords], - origin = ArgumentMatchers.eq(AppendOrigin.Coordinator), requiredAcks = anyInt(), + origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR), requiredAcks = anyInt(), any())).thenReturn(LogAppendInfo.UnknownLogAppendInfo) groupMetadataManager.cleanupGroupMetadata() verify(partition).appendRecordsToLeader(any[MemoryRecords], - origin = ArgumentMatchers.eq(AppendOrigin.Coordinator), requiredAcks = anyInt(), + origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR), requiredAcks = anyInt(), any()) verify(replicaManager, times(2)).onlinePartition(groupTopicPartition) @@ -2454,7 +2455,7 @@ class GroupMetadataManagerTest { verify(replicaManager).appendRecords(anyLong(), anyShort(), internalTopicsAllowed = ArgumentMatchers.eq(true), - origin = ArgumentMatchers.eq(AppendOrigin.Coordinator), + origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR), any[Map[TopicPartition, MemoryRecords]], capturedArgument.capture(), any[Option[ReentrantLock]], @@ -2469,7 +2470,7 @@ class GroupMetadataManagerTest { when(replicaManager.appendRecords(anyLong(), anyShort(), internalTopicsAllowed = ArgumentMatchers.eq(true), - origin = ArgumentMatchers.eq(AppendOrigin.Coordinator), + origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR), capturedRecords.capture(), capturedCallback.capture(), any[Option[ReentrantLock]], diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala index 6a56b768c47ea..59a4ef9241b1f 100644 --- a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala @@ -20,9 +20,8 @@ import java.lang.management.ManagementFactory import java.nio.ByteBuffer import java.util.concurrent.CountDownLatch import java.util.concurrent.locks.ReentrantLock - import javax.management.ObjectName -import kafka.log.{AppendOrigin, Defaults, LogConfig, UnifiedLog} +import kafka.log.{Defaults, LogConfig, UnifiedLog} import kafka.server.{FetchDataInfo, FetchLogEnd, LogOffsetMetadata, ReplicaManager, RequestLocal} import kafka.utils.{MockScheduler, Pool, TestUtils} import kafka.zk.KafkaZkClient @@ -34,6 +33,7 @@ import org.apache.kafka.common.record._ import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse import org.apache.kafka.common.requests.TransactionResult import org.apache.kafka.common.utils.MockTime +import org.apache.kafka.server.log.internals.AppendOrigin import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} import org.mockito.{ArgumentCaptor, ArgumentMatchers} @@ -648,7 +648,7 @@ class TransactionStateManagerTest { anyLong(), ArgumentMatchers.eq((-1).toShort), ArgumentMatchers.eq(true), - ArgumentMatchers.eq(AppendOrigin.Coordinator), + ArgumentMatchers.eq(AppendOrigin.COORDINATOR), any(), any(), any[Option[ReentrantLock]], @@ -691,7 +691,7 @@ class TransactionStateManagerTest { anyLong(), ArgumentMatchers.eq((-1).toShort), ArgumentMatchers.eq(true), - ArgumentMatchers.eq(AppendOrigin.Coordinator), + ArgumentMatchers.eq(AppendOrigin.COORDINATOR), any(), any(), any[Option[ReentrantLock]], @@ -731,7 +731,7 @@ class TransactionStateManagerTest { anyLong(), ArgumentMatchers.eq((-1).toShort), ArgumentMatchers.eq(true), - ArgumentMatchers.eq(AppendOrigin.Coordinator), + ArgumentMatchers.eq(AppendOrigin.COORDINATOR), any(), any(), any[Option[ReentrantLock]], @@ -886,7 +886,7 @@ class TransactionStateManagerTest { anyLong(), ArgumentMatchers.eq((-1).toShort), ArgumentMatchers.eq(true), - ArgumentMatchers.eq(AppendOrigin.Coordinator), + ArgumentMatchers.eq(AppendOrigin.COORDINATOR), recordsCapture.capture(), callbackCapture.capture(), any[Option[ReentrantLock]], @@ -1036,7 +1036,7 @@ class TransactionStateManagerTest { when(replicaManager.appendRecords(anyLong(), anyShort(), internalTopicsAllowed = ArgumentMatchers.eq(true), - origin = ArgumentMatchers.eq(AppendOrigin.Coordinator), + origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR), any[Map[TopicPartition, MemoryRecords]], capturedArgument.capture(), any[Option[ReentrantLock]], diff --git a/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala index 4df20c5a90de8..d6f97f36e00ef 100644 --- a/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala @@ -19,12 +19,13 @@ package kafka.log import java.io.File import java.nio.file.Files import java.util.Properties -import kafka.server.{BrokerTopicStats, LogDirFailureChannel} +import kafka.server.BrokerTopicStats import kafka.utils.{MockTime, Pool, TestUtils} import kafka.utils.Implicits._ import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.record.{CompressionType, MemoryRecords, RecordBatch} import org.apache.kafka.common.utils.Utils +import org.apache.kafka.server.log.internals.LogDirFailureChannel import org.junit.jupiter.api.{AfterEach, Tag} import scala.collection.Seq diff --git a/core/src/test/scala/unit/kafka/log/BrokerCompressionTest.scala b/core/src/test/scala/unit/kafka/log/BrokerCompressionTest.scala index c27f5a9586b1f..b2daa24962dba 100755 --- a/core/src/test/scala/unit/kafka/log/BrokerCompressionTest.scala +++ b/core/src/test/scala/unit/kafka/log/BrokerCompressionTest.scala @@ -17,10 +17,11 @@ package kafka.log -import kafka.server.{BrokerTopicStats, FetchLogEnd, LogDirFailureChannel} +import kafka.server.{BrokerTopicStats, FetchLogEnd} import kafka.utils._ import org.apache.kafka.common.record.{CompressionType, MemoryRecords, RecordBatch, SimpleRecord} import org.apache.kafka.common.utils.Utils +import org.apache.kafka.server.log.internals.LogDirFailureChannel import org.apache.kafka.server.record.BrokerCompressionType import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api._ diff --git a/core/src/test/scala/unit/kafka/log/LocalLogTest.scala b/core/src/test/scala/unit/kafka/log/LocalLogTest.scala index c816ac5e009b2..deee5685bc58a 100644 --- a/core/src/test/scala/unit/kafka/log/LocalLogTest.scala +++ b/core/src/test/scala/unit/kafka/log/LocalLogTest.scala @@ -22,12 +22,13 @@ import java.nio.channels.ClosedChannelException import java.nio.charset.StandardCharsets import java.util.regex.Pattern import java.util.Collections -import kafka.server.{FetchDataInfo, KafkaConfig, LogDirFailureChannel, LogOffsetMetadata} +import kafka.server.{FetchDataInfo, KafkaConfig, LogOffsetMetadata} import kafka.utils.{MockTime, Scheduler, TestUtils} import org.apache.kafka.common.{KafkaException, TopicPartition} import org.apache.kafka.common.errors.KafkaStorageException import org.apache.kafka.common.record.{CompressionType, MemoryRecords, Record, SimpleRecord} import org.apache.kafka.common.utils.{Time, Utils} +import org.apache.kafka.server.log.internals.LogDirFailureChannel import org.junit.jupiter.api.Assertions.{assertFalse, _} import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala index 2b4d212b6052d..a2176d290c2f9 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala @@ -20,12 +20,12 @@ package kafka.log import java.io.File import java.nio.file.Files import java.util.Properties - -import kafka.server.{BrokerTopicStats, LogDirFailureChannel} +import kafka.server.{BrokerTopicStats} import kafka.utils._ import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.record._ import org.apache.kafka.common.utils.Utils +import org.apache.kafka.server.log.internals.{AppendOrigin, LogDirFailureChannel} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, Test} @@ -678,7 +678,7 @@ class LogCleanerManagerTest extends Logging { log.appendAsLeader(MemoryRecords.withEndTransactionMarker(time.milliseconds(), producerId, producerEpoch, new EndTransactionMarker(ControlRecordType.ABORT, 15)), leaderEpoch = 0, - origin = AppendOrigin.Coordinator) + origin = AppendOrigin.COORDINATOR) log.roll() log.updateHighWatermark(4L) diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala old mode 100755 new mode 100644 index 071e8b8fd1c34..194335849eae1 --- a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala @@ -24,13 +24,13 @@ import java.nio.file.Paths import java.util.Properties import java.util.concurrent.{CountDownLatch, TimeUnit} import kafka.common._ -import kafka.server.{BrokerTopicStats, KafkaConfig, LogDirFailureChannel} +import kafka.server.{BrokerTopicStats, KafkaConfig} import kafka.utils._ import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.errors.CorruptRecordException import org.apache.kafka.common.record._ import org.apache.kafka.common.utils.Utils -import org.apache.kafka.server.log.internals.AbortedTxn +import org.apache.kafka.server.log.internals.{AbortedTxn, AppendOrigin, LogDirFailureChannel, OffsetMap} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, Test} @@ -303,11 +303,11 @@ class LogCleanerTest { val appendProducer2 = appendTransactionalAsLeader(log, producerId2, producerEpoch) def abort(producerId: Long): Unit = { - log.appendAsLeader(abortMarker(producerId, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.Replication) + log.appendAsLeader(abortMarker(producerId, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.REPLICATION) } def commit(producerId: Long): Unit = { - log.appendAsLeader(commitMarker(producerId, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.Replication) + log.appendAsLeader(commitMarker(producerId, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.REPLICATION) } // Append some transaction data (offset range in parenthesis) @@ -393,10 +393,10 @@ class LogCleanerTest { appendProducer1(Seq(1, 2)) appendProducer2(Seq(2, 3)) appendProducer1(Seq(3, 4)) - log.appendAsLeader(abortMarker(pid1, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.Coordinator) - log.appendAsLeader(commitMarker(pid2, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.Coordinator) + log.appendAsLeader(abortMarker(pid1, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.COORDINATOR) + log.appendAsLeader(commitMarker(pid2, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.COORDINATOR) appendProducer1(Seq(2)) - log.appendAsLeader(commitMarker(pid1, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.Coordinator) + log.appendAsLeader(commitMarker(pid1, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.COORDINATOR) val abortedTransactions = log.collectAbortedTransactions(log.logStartOffset, log.logEndOffset) @@ -434,11 +434,11 @@ class LogCleanerTest { appendProducer2(Seq(5, 6)) appendProducer3(Seq(6, 7)) appendProducer1(Seq(7, 8)) - log.appendAsLeader(abortMarker(pid2, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.Coordinator) + log.appendAsLeader(abortMarker(pid2, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.COORDINATOR) appendProducer3(Seq(8, 9)) - log.appendAsLeader(commitMarker(pid3, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.Coordinator) + log.appendAsLeader(commitMarker(pid3, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.COORDINATOR) appendProducer1(Seq(9, 10)) - log.appendAsLeader(abortMarker(pid1, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.Coordinator) + log.appendAsLeader(abortMarker(pid1, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.COORDINATOR) // we have only cleaned the records in the first segment val dirtyOffset = cleaner.clean(LogToClean(new TopicPartition("test", 0), log, 0L, log.activeSegment.baseOffset))._1 @@ -469,9 +469,9 @@ class LogCleanerTest { appendProducer(Seq(1)) appendProducer(Seq(2, 3)) - log.appendAsLeader(commitMarker(producerId, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.Coordinator) + log.appendAsLeader(commitMarker(producerId, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.COORDINATOR) appendProducer(Seq(2)) - log.appendAsLeader(commitMarker(producerId, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.Coordinator) + log.appendAsLeader(commitMarker(producerId, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.COORDINATOR) log.roll() // cannot remove the marker in this pass because there are still valid records @@ -480,7 +480,7 @@ class LogCleanerTest { assertEquals(List(0, 2, 3, 4, 5), offsetsInLog(log)) appendProducer(Seq(1, 3)) - log.appendAsLeader(commitMarker(producerId, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.Coordinator) + log.appendAsLeader(commitMarker(producerId, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.COORDINATOR) log.roll() // the first cleaning preserves the commit marker (at offset 3) since there were still records for the transaction @@ -517,10 +517,10 @@ class LogCleanerTest { val appendProducer = appendTransactionalAsLeader(log, producerId, producerEpoch) appendProducer(Seq(1)) - log.appendAsLeader(abortMarker(producerId, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.Coordinator) + log.appendAsLeader(abortMarker(producerId, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.COORDINATOR) appendProducer(Seq(2)) appendProducer(Seq(2)) - log.appendAsLeader(commitMarker(producerId, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.Coordinator) + log.appendAsLeader(commitMarker(producerId, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.COORDINATOR) log.roll() cleaner.doClean(LogToClean(tp, log, 0L, log.activeSegment.baseOffset), currentTime = largeTimestamp) @@ -552,7 +552,7 @@ class LogCleanerTest { // [{Producer1: 2, 3}], [{Producer2: 2, 3}, {Producer2: Commit}] producer2(Seq(2, 3)) // offsets 2, 3 log.appendAsLeader(commitMarker(2L, producerEpoch), leaderEpoch = 0, - origin = AppendOrigin.Coordinator) // offset 4 + origin = AppendOrigin.COORDINATOR) // offset 4 log.roll() // [{Producer1: 2, 3}], [{Producer2: 2, 3}, {Producer2: Commit}], [{2}, {3}, {Producer1: Commit}] @@ -560,7 +560,7 @@ class LogCleanerTest { log.appendAsLeader(record(2, 2), leaderEpoch = 0) // offset 5 log.appendAsLeader(record(3, 3), leaderEpoch = 0) // offset 6 log.appendAsLeader(commitMarker(1L, producerEpoch), leaderEpoch = 0, - origin = AppendOrigin.Coordinator) // offset 7 + origin = AppendOrigin.COORDINATOR) // offset 7 log.roll() // first time through the records are removed @@ -582,7 +582,7 @@ class LogCleanerTest { // {1}, {3}, {4}, {5}, {6}, {7}, {8}, {9} ==> Offsets producer2(Seq(1)) // offset 8 log.appendAsLeader(commitMarker(2L, producerEpoch), leaderEpoch = 0, - origin = AppendOrigin.Coordinator) // offset 9 + origin = AppendOrigin.COORDINATOR) // offset 9 log.roll() // Expected State: [{Producer1: EmptyBatch}, {Producer2: Commit}, {2}, {3}, {Producer1: Commit}, {Producer2: 1}, {Producer2: Commit}] @@ -612,7 +612,7 @@ class LogCleanerTest { // [{Producer1: Commit}, {2}, {3}] log.appendAsLeader(commitMarker(1L, producerEpoch), leaderEpoch = 0, - origin = AppendOrigin.Coordinator) // offset 1 + origin = AppendOrigin.COORDINATOR) // offset 1 log.appendAsLeader(record(2, 2), leaderEpoch = 0) // offset 2 log.appendAsLeader(record(3, 3), leaderEpoch = 0) // offset 3 log.roll() @@ -648,7 +648,7 @@ class LogCleanerTest { appendTransaction(Seq(1)) log.roll() - log.appendAsLeader(commitMarker(producerId, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.Coordinator) + log.appendAsLeader(commitMarker(producerId, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.COORDINATOR) log.roll() // Both the record and the marker should remain after cleaning @@ -671,7 +671,7 @@ class LogCleanerTest { appendTransaction(Seq(1)) log.roll() - log.appendAsLeader(abortMarker(producerId, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.Coordinator) + log.appendAsLeader(abortMarker(producerId, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.COORDINATOR) log.roll() // Both the batch and the marker should remain after cleaning. The batch is retained @@ -701,9 +701,9 @@ class LogCleanerTest { appendProducer(Seq(1)) appendProducer(Seq(2, 3)) - log.appendAsLeader(abortMarker(producerId, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.Coordinator) + log.appendAsLeader(abortMarker(producerId, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.COORDINATOR) appendProducer(Seq(3)) - log.appendAsLeader(commitMarker(producerId, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.Coordinator) + log.appendAsLeader(commitMarker(producerId, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.COORDINATOR) log.roll() // Aborted records are removed, but the abort marker is still preserved. @@ -731,14 +731,14 @@ class LogCleanerTest { val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val appendFirstTransaction = appendTransactionalAsLeader(log, producerId, producerEpoch, - origin = AppendOrigin.Replication) + origin = AppendOrigin.REPLICATION) appendFirstTransaction(Seq(1)) - log.appendAsLeader(commitMarker(producerId, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.Coordinator) + log.appendAsLeader(commitMarker(producerId, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.COORDINATOR) val appendSecondTransaction = appendTransactionalAsLeader(log, producerId, producerEpoch, - origin = AppendOrigin.Replication) + origin = AppendOrigin.REPLICATION) appendSecondTransaction(Seq(2)) - log.appendAsLeader(commitMarker(producerId, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.Coordinator) + log.appendAsLeader(commitMarker(producerId, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.COORDINATOR) log.appendAsLeader(record(1, 1), leaderEpoch = 0) log.appendAsLeader(record(2, 1), leaderEpoch = 0) @@ -771,7 +771,7 @@ class LogCleanerTest { val appendProducer = appendTransactionalAsLeader(log, producerId, producerEpoch) appendProducer(Seq(2, 3)) // batch last offset is 1 - log.appendAsLeader(abortMarker(producerId, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.Coordinator) + log.appendAsLeader(abortMarker(producerId, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.COORDINATOR) log.roll() def assertAbortedTransactionIndexed(): Unit = { @@ -1012,7 +1012,7 @@ class LogCleanerTest { appendProducer(Seq(1)) appendProducer(Seq(2, 3)) - log.appendAsLeader(abortMarker(producerId, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.Coordinator) + log.appendAsLeader(abortMarker(producerId, producerEpoch), leaderEpoch = 0, origin = AppendOrigin.COORDINATOR) log.roll() cleaner.clean(LogToClean(new TopicPartition("test", 0), log, 0L, log.activeSegment.baseOffset)) @@ -1046,7 +1046,7 @@ class LogCleanerTest { val producerId1 = 1L val appendProducer = appendTransactionalAsLeader(log, producerId1, producerEpoch, leaderEpoch) appendProducer(Seq(1)) - log.appendAsLeader(commitMarker(producerId1, producerEpoch), leaderEpoch, origin = AppendOrigin.Coordinator) + log.appendAsLeader(commitMarker(producerId1, producerEpoch), leaderEpoch, origin = AppendOrigin.COORDINATOR) // Now we append one transaction with a key which conflicts with the COMMIT marker appended above def commitRecordKey(): ByteBuffer = { @@ -1065,8 +1065,8 @@ class LogCleanerTest { 0, new SimpleRecord(time.milliseconds(), commitRecordKey(), ByteBuffer.wrap("foo".getBytes)) ) - log.appendAsLeader(records, leaderEpoch, origin = AppendOrigin.Client) - log.appendAsLeader(commitMarker(producerId2, producerEpoch), leaderEpoch, origin = AppendOrigin.Coordinator) + log.appendAsLeader(records, leaderEpoch, origin = AppendOrigin.CLIENT) + log.appendAsLeader(commitMarker(producerId2, producerEpoch), leaderEpoch, origin = AppendOrigin.COORDINATOR) log.roll() assertEquals(List(0, 1, 2, 3), offsetsInLog(log)) @@ -2013,7 +2013,7 @@ class LogCleanerTest { producerId: Long, producerEpoch: Short, leaderEpoch: Int = 0, - origin: AppendOrigin = AppendOrigin.Client + origin: AppendOrigin = AppendOrigin.CLIENT ): Seq[Int] => LogAppendInfo = { appendIdempotentAsLeader( log, @@ -2031,7 +2031,7 @@ class LogCleanerTest { producerEpoch: Short, isTransactional: Boolean = false, leaderEpoch: Int = 0, - origin: AppendOrigin = AppendOrigin.Client + origin: AppendOrigin = AppendOrigin.CLIENT ): Seq[Int] => LogAppendInfo = { var sequence = 0 keys: Seq[Int] => { diff --git a/core/src/test/scala/unit/kafka/log/LogConcurrencyTest.scala b/core/src/test/scala/unit/kafka/log/LogConcurrencyTest.scala index 010c9f7960da9..14a23f68e09f4 100644 --- a/core/src/test/scala/unit/kafka/log/LogConcurrencyTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogConcurrencyTest.scala @@ -20,10 +20,11 @@ package kafka.log import java.util.Properties import java.util.concurrent.{Callable, Executors} -import kafka.server.{BrokerTopicStats, FetchHighWatermark, LogDirFailureChannel} +import kafka.server.{BrokerTopicStats, FetchHighWatermark} import kafka.utils.{KafkaScheduler, TestUtils} import org.apache.kafka.common.record.SimpleRecord import org.apache.kafka.common.utils.{Time, Utils} +import org.apache.kafka.server.log.internals.LogDirFailureChannel import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} diff --git a/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala b/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala index f877860a8324c..e59d43e0cf716 100644 --- a/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala @@ -22,7 +22,7 @@ import java.nio.ByteBuffer import java.nio.file.{Files, NoSuchFileException, Paths} import java.util.Properties import kafka.server.epoch.{EpochEntry, LeaderEpochFileCache} -import kafka.server.{BrokerTopicStats, FetchDataInfo, KafkaConfig, LogDirFailureChannel} +import kafka.server.{BrokerTopicStats, FetchDataInfo, KafkaConfig} import kafka.server.metadata.MockConfigRepository import kafka.utils.{CoreUtils, MockTime, Scheduler, TestUtils} import org.apache.kafka.common.TopicPartition @@ -31,7 +31,7 @@ import org.apache.kafka.common.record.{CompressionType, ControlRecordType, Defau import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.server.common.MetadataVersion.IBP_0_11_0_IV0 -import org.apache.kafka.server.log.internals.{AbortedTxn, OffsetIndex} +import org.apache.kafka.server.log.internals.{AbortedTxn, LogDirFailureChannel, OffsetIndex} import org.junit.jupiter.api.Assertions.{assertDoesNotThrow, assertEquals, assertFalse, assertNotEquals, assertThrows, assertTrue} import org.junit.jupiter.api.function.Executable import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} diff --git a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala index 2d70eba7c435c..3c17878ae799d 100755 --- a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala @@ -21,7 +21,7 @@ import com.yammer.metrics.core.{Gauge, MetricName} import kafka.log.remote.RemoteIndexCache import kafka.server.checkpoints.OffsetCheckpointFile import kafka.server.metadata.{ConfigRepository, MockConfigRepository} -import kafka.server.{BrokerTopicStats, FetchDataInfo, FetchLogEnd, LogDirFailureChannel} +import kafka.server.{BrokerTopicStats, FetchDataInfo, FetchLogEnd} import kafka.utils._ import org.apache.directory.api.util.FileUtils import org.apache.kafka.common.errors.OffsetOutOfRangeException @@ -37,6 +37,7 @@ import java.io._ import java.nio.file.Files import java.util.concurrent.{ConcurrentHashMap, ConcurrentMap, Future} import java.util.{Collections, Properties} +import org.apache.kafka.server.log.internals.LogDirFailureChannel import org.apache.kafka.server.metrics.KafkaYammerMetrics import scala.collection.{Map, mutable} @@ -919,6 +920,8 @@ class LogManagerTest { // Replace the current log with the future one and verify that only one set of metrics are present logManager.replaceCurrentWithFutureLog(tp) verifyMetrics(1) + // the future log is gone, so we have to make sure the metrics gets gone also. + assertEquals(0, logMetrics.count(m => m.getMBeanName.contains("is-future"))) // Trigger the deletion of the former current directory and verify that one set of metrics is still present time.sleep(logConfig.fileDeleteDelayMs + 1) diff --git a/core/src/test/scala/unit/kafka/log/LogTestUtils.scala b/core/src/test/scala/unit/kafka/log/LogTestUtils.scala index a293ed9eac928..b614e11c7e703 100644 --- a/core/src/test/scala/unit/kafka/log/LogTestUtils.scala +++ b/core/src/test/scala/unit/kafka/log/LogTestUtils.scala @@ -22,17 +22,18 @@ import kafka.log.remote.RemoteLogManager import java.io.File import java.util.Properties import kafka.server.checkpoints.LeaderEpochCheckpointFile -import kafka.server.{BrokerTopicStats, FetchDataInfo, FetchIsolation, FetchLogEnd, LogDirFailureChannel} +import kafka.server.{BrokerTopicStats, FetchDataInfo, FetchIsolation, FetchLogEnd} import kafka.utils.{Scheduler, TestUtils} import org.apache.kafka.common.Uuid import org.apache.kafka.common.record.{CompressionType, ControlRecordType, EndTransactionMarker, FileRecords, MemoryRecords, RecordBatch, SimpleRecord} import org.apache.kafka.common.utils.{Time, Utils} +import org.apache.kafka.server.log.internals.LogDirFailureChannel import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse} import java.nio.file.Files import java.util.concurrent.{ConcurrentHashMap, ConcurrentMap} import kafka.log -import org.apache.kafka.server.log.internals.{AbortedTxn, TransactionIndex} +import org.apache.kafka.server.log.internals.{AbortedTxn, AppendOrigin, LazyIndex, TransactionIndex} import scala.collection.Iterable import scala.jdk.CollectionConverters._ @@ -46,8 +47,8 @@ object LogTestUtils { indexIntervalBytes: Int = 10, time: Time = Time.SYSTEM): LogSegment = { val ms = FileRecords.open(UnifiedLog.logFile(logDir, offset)) - val idx = LazyIndex.forOffset(UnifiedLog.offsetIndexFile(logDir, offset), offset, maxIndexSize = 1000) - val timeIdx = LazyIndex.forTime(UnifiedLog.timeIndexFile(logDir, offset), offset, maxIndexSize = 1500) + val idx = LazyIndex.forOffset(UnifiedLog.offsetIndexFile(logDir, offset), offset, 1000, true) + val timeIdx = LazyIndex.forTime(UnifiedLog.timeIndexFile(logDir, offset), offset, 1500, true) val txnIndex = new TransactionIndex(offset, UnifiedLog.transactionIndexFile(logDir, offset)) new LogSegment(ms, idx, timeIdx, txnIndex, offset, indexIntervalBytes, 0, time) @@ -215,7 +216,7 @@ object LogTestUtils { leaderEpoch: Int = 0): LogAppendInfo = { val records = endTxnRecords(controlType, producerId, producerEpoch, coordinatorEpoch = coordinatorEpoch, timestamp = timestamp) - log.appendAsLeader(records, origin = AppendOrigin.Coordinator, leaderEpoch = leaderEpoch) + log.appendAsLeader(records, origin = AppendOrigin.COORDINATOR, leaderEpoch = leaderEpoch) } private def endTxnRecords(controlRecordType: ControlRecordType, diff --git a/core/src/test/scala/unit/kafka/log/LogValidatorTest.scala b/core/src/test/scala/unit/kafka/log/LogValidatorTest.scala index ac8718d366283..aa5843b7c0338 100644 --- a/core/src/test/scala/unit/kafka/log/LogValidatorTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogValidatorTest.scala @@ -18,16 +18,16 @@ package kafka.log import java.nio.ByteBuffer import java.util.concurrent.TimeUnit - -import kafka.common.{LongRef, RecordValidationException} -import kafka.log.LogValidator.ValidationAndOffsetAssignResult import kafka.server.{BrokerTopicStats, RequestLocal} +import kafka.utils.MockTime import kafka.utils.TestUtils.meterCount import org.apache.kafka.common.errors.{InvalidTimestampException, UnsupportedCompressionTypeException, UnsupportedForMessageFormatException} import org.apache.kafka.common.record._ -import org.apache.kafka.common.utils.Time +import org.apache.kafka.common.utils.{PrimitiveRef, Time} import org.apache.kafka.common.{InvalidRecordException, TopicPartition} import org.apache.kafka.server.common.MetadataVersion +import org.apache.kafka.server.log.internals.LogValidator.ValidationResult +import org.apache.kafka.server.log.internals.{AppendOrigin, LogValidator, RecordValidationException} import org.apache.kafka.server.metrics.KafkaYammerMetrics import org.apache.kafka.test.TestUtils import org.junit.jupiter.api.Assertions._ @@ -39,8 +39,8 @@ class LogValidatorTest { val time = Time.SYSTEM val topicPartition = new TopicPartition("topic", 0) - val brokerTopicStats = new BrokerTopicStats val metricsKeySet = KafkaYammerMetrics.defaultRegistry.allMetrics.keySet.asScala + val metricsRecorder = UnifiedLog.newValidatorMetricsRecorder(new BrokerTopicStats().allTopicsStats) @Test def testOnlyOneBatch(): Unit = { @@ -112,23 +112,22 @@ class LogValidatorTest { private def validateMessages(records: MemoryRecords, magic: Byte, sourceCompressionType: CompressionType, - targetCompressionType: CompressionType): ValidationAndOffsetAssignResult = { - LogValidator.validateMessagesAndAssignOffsets(records, + targetCompressionType: CompressionType): ValidationResult = { + val mockTime = new MockTime(0L, 0L) + new LogValidator(records, topicPartition, - new LongRef(0L), - time, - now = 0L, + mockTime, sourceCompressionType, targetCompressionType, - compactedTopic = false, + false, magic, TimestampType.CREATE_TIME, 1000L, RecordBatch.NO_PRODUCER_EPOCH, - origin = AppendOrigin.Client, - MetadataVersion.IBP_2_3_IV1, - brokerTopicStats, - RequestLocal.withThreadConfinedCaching) + AppendOrigin.CLIENT, + MetadataVersion.IBP_2_3_IV1 + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(0L), metricsRecorder, RequestLocal.withThreadConfinedCaching.bufferSupplier) } @Test @@ -142,34 +141,39 @@ class LogValidatorTest { } private def checkLogAppendTimeNonCompressed(magic: Byte): Unit = { - val now = System.currentTimeMillis() + val mockTime = new MockTime // The timestamps should be overwritten val records = createRecords(magicValue = magic, timestamp = 1234L, codec = CompressionType.NONE) - val validatedResults = LogValidator.validateMessagesAndAssignOffsets(records, + val offsetCounter = PrimitiveRef.ofLong(0) + val validatedResults = new LogValidator(records, topicPartition, - offsetCounter = new LongRef(0), - time= time, - now = now, - sourceCompression = CompressionType.NONE, - targetCompression = CompressionType.NONE, - compactedTopic = false, - magic = magic, - timestampType = TimestampType.LOG_APPEND_TIME, - timestampDiffMaxMs = 1000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching) + mockTime, + CompressionType.NONE, + CompressionType.NONE, + false, + magic, + TimestampType.LOG_APPEND_TIME, + 1000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + offsetCounter, + metricsRecorder, + RequestLocal.withThreadConfinedCaching.bufferSupplier + ) + + assertEquals(offsetCounter.value, records.records.asScala.size) val validatedRecords = validatedResults.validatedRecords assertEquals(records.records.asScala.size, validatedRecords.records.asScala.size, "message set size should not change") + val now = mockTime.milliseconds validatedRecords.batches.forEach(batch => validateLogAppendTime(now, 1234L, batch)) - assertEquals(now, validatedResults.maxTimestamp, s"Max timestamp should be $now") + assertEquals(now, validatedResults.maxTimestampMs, s"Max timestamp should be $now") assertFalse(validatedResults.messageSizeMaybeChanged, "Message size should not have been changed") // we index from last offset in version 2 instead of base offset val expectedMaxTimestampOffset = if (magic >= RecordBatch.MAGIC_VALUE_V2) 2 else 0 - assertEquals(expectedMaxTimestampOffset, validatedResults.shallowOffsetOfMaxTimestamp, + assertEquals(expectedMaxTimestampOffset, validatedResults.shallowOffsetOfMaxTimestampMs, s"The offset of max timestamp should be $expectedMaxTimestampOffset") verifyRecordConversionStats(validatedResults.recordConversionStats, numConvertedRecords = 0, records, compressed = false) @@ -181,36 +185,38 @@ class LogValidatorTest { } private def checkLogAppendTimeWithRecompression(targetMagic: Byte): Unit = { - val now = System.currentTimeMillis() + val mockTime = new MockTime // The timestamps should be overwritten val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V0, codec = CompressionType.GZIP) - val validatedResults = LogValidator.validateMessagesAndAssignOffsets( + val validatedResults = new LogValidator( records, topicPartition, - offsetCounter = new LongRef(0), - time = time, - now = now, - sourceCompression = CompressionType.GZIP, - targetCompression = CompressionType.GZIP, - compactedTopic = false, - magic = targetMagic, - timestampType = TimestampType.LOG_APPEND_TIME, - timestampDiffMaxMs = 1000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching) - val validatedRecords = validatedResults.validatedRecords + mockTime, + CompressionType.GZIP, + CompressionType.GZIP, + false, + targetMagic, + TimestampType.LOG_APPEND_TIME, + 1000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(0), + metricsRecorder, + RequestLocal.withThreadConfinedCaching.bufferSupplier + ) + val validatedRecords = validatedResults.validatedRecords assertEquals(records.records.asScala.size, validatedRecords.records.asScala.size, "message set size should not change") + val now = mockTime.milliseconds() validatedRecords.batches.forEach(batch => validateLogAppendTime(now, -1, batch)) assertTrue(validatedRecords.batches.iterator.next().isValid, "MessageSet should still valid") - assertEquals(now, validatedResults.maxTimestamp, + assertEquals(now, validatedResults.maxTimestampMs, s"Max timestamp should be $now") - assertEquals(records.records.asScala.size - 1, validatedResults.shallowOffsetOfMaxTimestamp, + assertEquals(records.records.asScala.size - 1, validatedResults.shallowOffsetOfMaxTimestampMs, s"The offset of max timestamp should be ${records.records.asScala.size - 1}") assertTrue(validatedResults.messageSizeMaybeChanged, "Message size may have been changed") @@ -230,36 +236,38 @@ class LogValidatorTest { } private def checkLogAppendTimeWithoutRecompression(magic: Byte): Unit = { - val now = System.currentTimeMillis() + val mockTime = new MockTime // The timestamps should be overwritten val records = createRecords(magicValue = magic, timestamp = 1234L, codec = CompressionType.GZIP) - val validatedResults = LogValidator.validateMessagesAndAssignOffsets( + val validatedResults = new LogValidator( records, topicPartition, - offsetCounter = new LongRef(0), - time = time, - now = now, - sourceCompression = CompressionType.GZIP, - targetCompression = CompressionType.GZIP, - compactedTopic = false, - magic = magic, - timestampType = TimestampType.LOG_APPEND_TIME, - timestampDiffMaxMs = 1000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching) + mockTime, + CompressionType.GZIP, + CompressionType.GZIP, + false, + magic, + TimestampType.LOG_APPEND_TIME, + 1000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(0), + metricsRecorder, + RequestLocal.withThreadConfinedCaching.bufferSupplier + ) val validatedRecords = validatedResults.validatedRecords assertEquals(records.records.asScala.size, validatedRecords.records.asScala.size, "message set size should not change") + val now = mockTime.milliseconds() validatedRecords.batches.forEach(batch => validateLogAppendTime(now, 1234L, batch)) assertTrue(validatedRecords.batches.iterator.next().isValid, "MessageSet should still valid") - assertEquals(now, validatedResults.maxTimestamp, + assertEquals(now, validatedResults.maxTimestampMs, s"Max timestamp should be $now") - assertEquals(records.records.asScala.size - 1, validatedResults.shallowOffsetOfMaxTimestamp, + assertEquals(records.records.asScala.size - 1, validatedResults.shallowOffsetOfMaxTimestampMs, s"The offset of max timestamp should be ${records.records.asScala.size - 1}") assertFalse(validatedResults.messageSizeMaybeChanged, "Message size should not have been changed") @@ -296,23 +304,24 @@ class LogValidatorTest { val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V2, timestamp = 1234L, codec = CompressionType.NONE) records.buffer.putInt(DefaultRecordBatch.RECORDS_COUNT_OFFSET, count) records.buffer.putInt(DefaultRecordBatch.LAST_OFFSET_DELTA_OFFSET, lastOffsetDelta) - LogValidator.validateMessagesAndAssignOffsets( + new LogValidator( records, topicPartition, - offsetCounter = new LongRef(0), - time = time, - now = time.milliseconds(), - sourceCompression = CompressionType.GZIP, - targetCompression = CompressionType.GZIP, - compactedTopic = false, - magic = RecordBatch.MAGIC_VALUE_V2, - timestampType = TimestampType.LOG_APPEND_TIME, - timestampDiffMaxMs = 1000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching) + time, + CompressionType.GZIP, + CompressionType.GZIP, + false, + RecordBatch.MAGIC_VALUE_V2, + TimestampType.LOG_APPEND_TIME, + 1000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(0), + metricsRecorder, + RequestLocal.withThreadConfinedCaching.bufferSupplier + ) } @Test @@ -342,22 +351,25 @@ class LogValidatorTest { new SimpleRecord(timestampSeq(1), "there".getBytes), new SimpleRecord(timestampSeq(2), "beautiful".getBytes)) - val validatingResults = LogValidator.validateMessagesAndAssignOffsets(records, + val offsetCounter = PrimitiveRef.ofLong(0); + val validatingResults = new LogValidator(records, topicPartition, - offsetCounter = new LongRef(0), - time = time, - now = System.currentTimeMillis(), - sourceCompression = CompressionType.NONE, - targetCompression = CompressionType.NONE, - compactedTopic = false, - magic = magic, - timestampType = TimestampType.CREATE_TIME, - timestampDiffMaxMs = 1000L, - partitionLeaderEpoch = partitionLeaderEpoch, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching) + time, + CompressionType.NONE, + CompressionType.NONE, + false, + magic, + TimestampType.CREATE_TIME, + 1000L, + partitionLeaderEpoch, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + offsetCounter, + metricsRecorder, + RequestLocal.withThreadConfinedCaching.bufferSupplier + ) + val validatedRecords = validatingResults.validatedRecords var i = 0 @@ -377,9 +389,11 @@ class LogValidatorTest { i += 1 } } - assertEquals(now + 1, validatingResults.maxTimestamp, + + assertEquals(i, offsetCounter.value); + assertEquals(now + 1, validatingResults.maxTimestampMs, s"Max timestamp should be ${now + 1}") - assertEquals(1, validatingResults.shallowOffsetOfMaxTimestamp, + assertEquals(1, validatingResults.shallowOffsetOfMaxTimestampMs, s"Offset of max timestamp should be 1") assertFalse(validatingResults.messageSizeMaybeChanged, "Message size should not have been changed") @@ -415,22 +429,23 @@ class LogValidatorTest { new SimpleRecord(timestampSeq(1), "there".getBytes), new SimpleRecord(timestampSeq(2), "beautiful".getBytes)) - val validatingResults = LogValidator.validateMessagesAndAssignOffsets(records, + val validatingResults = new LogValidator(records, topicPartition, - offsetCounter = new LongRef(0), - time = time, - now = System.currentTimeMillis(), - sourceCompression = CompressionType.NONE, - targetCompression = CompressionType.GZIP, - compactedTopic = false, - magic = magic, - timestampType = TimestampType.CREATE_TIME, - timestampDiffMaxMs = 1000L, - partitionLeaderEpoch = partitionLeaderEpoch, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching) + time, + CompressionType.NONE, + CompressionType.GZIP, + false, + magic, + TimestampType.CREATE_TIME, + 1000L, + partitionLeaderEpoch, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(0), + metricsRecorder, + RequestLocal.withThreadConfinedCaching.bufferSupplier + ) val validatedRecords = validatingResults.validatedRecords var i = 0 @@ -449,9 +464,9 @@ class LogValidatorTest { i += 1 } } - assertEquals(now + 1, validatingResults.maxTimestamp, + assertEquals(now + 1, validatingResults.maxTimestampMs, s"Max timestamp should be ${now + 1}") - assertEquals(2, validatingResults.shallowOffsetOfMaxTimestamp, + assertEquals(2, validatingResults.shallowOffsetOfMaxTimestampMs, "Offset of max timestamp should be 2") assertTrue(validatingResults.messageSizeMaybeChanged, "Message size should have been changed") @@ -472,22 +487,23 @@ class LogValidatorTest { private def checkCreateTimeUpConversionFromV0(toMagic: Byte): Unit = { val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V0, codec = CompressionType.GZIP) - val validatedResults = LogValidator.validateMessagesAndAssignOffsets(records, + val validatedResults = new LogValidator(records, topicPartition, - offsetCounter = new LongRef(0), - time = time, - now = System.currentTimeMillis(), - sourceCompression = CompressionType.GZIP, - targetCompression = CompressionType.GZIP, - magic = toMagic, - compactedTopic = false, - timestampType = TimestampType.CREATE_TIME, - timestampDiffMaxMs = 1000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching) + time, + CompressionType.GZIP, + CompressionType.GZIP, + false, + toMagic, + TimestampType.CREATE_TIME, + 1000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(0), + metricsRecorder, + RequestLocal.withThreadConfinedCaching.bufferSupplier + ) val validatedRecords = validatedResults.validatedRecords for (batch <- validatedRecords.batches.asScala) { @@ -499,9 +515,9 @@ class LogValidatorTest { assertEquals(RecordBatch.NO_PRODUCER_ID, batch.producerId) assertEquals(RecordBatch.NO_SEQUENCE, batch.baseSequence) } - assertEquals(validatedResults.maxTimestamp, RecordBatch.NO_TIMESTAMP, + assertEquals(validatedResults.maxTimestampMs, RecordBatch.NO_TIMESTAMP, s"Max timestamp should be ${RecordBatch.NO_TIMESTAMP}") - assertEquals(validatedRecords.records.asScala.size - 1, validatedResults.shallowOffsetOfMaxTimestamp, + assertEquals(validatedRecords.records.asScala.size - 1, validatedResults.shallowOffsetOfMaxTimestampMs, s"Offset of max timestamp should be ${validatedRecords.records.asScala.size - 1}") assertTrue(validatedResults.messageSizeMaybeChanged, "Message size should have been changed") @@ -518,22 +534,23 @@ class LogValidatorTest { def testCreateTimeUpConversionV1ToV2(): Unit = { val timestamp = System.currentTimeMillis() val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V1, codec = CompressionType.GZIP, timestamp = timestamp) - val validatedResults = LogValidator.validateMessagesAndAssignOffsets(records, + val validatedResults = new LogValidator(records, topicPartition, - offsetCounter = new LongRef(0), - time = time, - now = timestamp, - sourceCompression = CompressionType.GZIP, - targetCompression = CompressionType.GZIP, - magic = RecordBatch.MAGIC_VALUE_V2, - compactedTopic = false, - timestampType = TimestampType.CREATE_TIME, - timestampDiffMaxMs = 1000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching) + time, + CompressionType.GZIP, + CompressionType.GZIP, + false, + RecordBatch.MAGIC_VALUE_V2, + TimestampType.CREATE_TIME, + 1000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.latest, + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(0), + metricsRecorder, + RequestLocal.withThreadConfinedCaching.bufferSupplier + ) val validatedRecords = validatedResults.validatedRecords for (batch <- validatedRecords.batches.asScala) { @@ -545,8 +562,8 @@ class LogValidatorTest { assertEquals(RecordBatch.NO_PRODUCER_ID, batch.producerId) assertEquals(RecordBatch.NO_SEQUENCE, batch.baseSequence) } - assertEquals(timestamp, validatedResults.maxTimestamp) - assertEquals(validatedRecords.records.asScala.size - 1, validatedResults.shallowOffsetOfMaxTimestamp, + assertEquals(timestamp, validatedResults.maxTimestampMs) + assertEquals(validatedRecords.records.asScala.size - 1, validatedResults.shallowOffsetOfMaxTimestampMs, s"Offset of max timestamp should be ${validatedRecords.records.asScala.size - 1}") assertTrue(validatedResults.messageSizeMaybeChanged, "Message size should have been changed") @@ -576,22 +593,23 @@ class LogValidatorTest { new SimpleRecord(timestampSeq(1), "there".getBytes), new SimpleRecord(timestampSeq(2), "beautiful".getBytes)) - val validatedResults = LogValidator.validateMessagesAndAssignOffsets(records, + val validatedResults = new LogValidator(records, topicPartition, - offsetCounter = new LongRef(0), - time = time, - now = System.currentTimeMillis(), - sourceCompression = CompressionType.GZIP, - targetCompression = CompressionType.GZIP, - magic = magic, - compactedTopic = false, - timestampType = TimestampType.CREATE_TIME, - timestampDiffMaxMs = 1000L, - partitionLeaderEpoch = partitionLeaderEpoch, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching) + time, + CompressionType.GZIP, + CompressionType.GZIP, + false, + magic, + TimestampType.CREATE_TIME, + 1000L, + partitionLeaderEpoch, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(0), + metricsRecorder, + RequestLocal.withThreadConfinedCaching.bufferSupplier + ) val validatedRecords = validatedResults.validatedRecords var i = 0 @@ -610,8 +628,8 @@ class LogValidatorTest { i += 1 } } - assertEquals(now + 1, validatedResults.maxTimestamp, s"Max timestamp should be ${now + 1}") - assertEquals(validatedRecords.records.asScala.size - 1, validatedResults.shallowOffsetOfMaxTimestamp, + assertEquals(now + 1, validatedResults.maxTimestampMs, s"Max timestamp should be ${now + 1}") + assertEquals(validatedRecords.records.asScala.size - 1, validatedResults.shallowOffsetOfMaxTimestampMs, s"Offset of max timestamp should be ${validatedRecords.records.asScala.size - 1}") assertFalse(validatedResults.messageSizeMaybeChanged, "Message size should not have been changed") @@ -629,23 +647,24 @@ class LogValidatorTest { val now = System.currentTimeMillis() val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V1, timestamp = now - 1001L, codec = CompressionType.NONE) - assertThrows(classOf[RecordValidationException], () => LogValidator.validateMessagesAndAssignOffsets( + assertThrows(classOf[RecordValidationException], () => new LogValidator( records, topicPartition, - offsetCounter = new LongRef(0), - time = time, - now = System.currentTimeMillis(), - sourceCompression = CompressionType.NONE, - targetCompression = CompressionType.NONE, - compactedTopic = false, - magic = RecordBatch.MAGIC_VALUE_V1, - timestampType = TimestampType.CREATE_TIME, - timestampDiffMaxMs = 1000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching)) + time, + CompressionType.NONE, + CompressionType.NONE, + false, + RecordBatch.MAGIC_VALUE_V1, + TimestampType.CREATE_TIME, + 1000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(0), + metricsRecorder, + RequestLocal.withThreadConfinedCaching.bufferSupplier + )) } @Test @@ -653,23 +672,24 @@ class LogValidatorTest { val now = System.currentTimeMillis() val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V2, timestamp = now - 1001L, codec = CompressionType.NONE) - assertThrows(classOf[RecordValidationException], () => LogValidator.validateMessagesAndAssignOffsets( + assertThrows(classOf[RecordValidationException], () => new LogValidator( records, topicPartition, - offsetCounter = new LongRef(0), - time = time, - now = System.currentTimeMillis(), - sourceCompression = CompressionType.NONE, - targetCompression = CompressionType.NONE, - compactedTopic = false, - magic = RecordBatch.MAGIC_VALUE_V2, - timestampType = TimestampType.CREATE_TIME, - timestampDiffMaxMs = 1000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching)) + time, + CompressionType.NONE, + CompressionType.NONE, + false, + RecordBatch.MAGIC_VALUE_V2, + TimestampType.CREATE_TIME, + 1000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(0), + metricsRecorder, + RequestLocal.withThreadConfinedCaching.bufferSupplier + )) } @Test @@ -677,23 +697,24 @@ class LogValidatorTest { val now = System.currentTimeMillis() val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V1, timestamp = now - 1001L, codec = CompressionType.GZIP) - assertThrows(classOf[RecordValidationException], () => LogValidator.validateMessagesAndAssignOffsets( + assertThrows(classOf[RecordValidationException], () => new LogValidator( records, topicPartition, - offsetCounter = new LongRef(0), - time = time, - now = System.currentTimeMillis(), - sourceCompression = CompressionType.GZIP, - targetCompression = CompressionType.GZIP, - magic = RecordBatch.MAGIC_VALUE_V1, - compactedTopic = false, - timestampType = TimestampType.CREATE_TIME, - timestampDiffMaxMs = 1000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching)) + time, + CompressionType.GZIP, + CompressionType.GZIP, + false, + RecordBatch.MAGIC_VALUE_V1, + TimestampType.CREATE_TIME, + 1000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(0), + metricsRecorder, + RequestLocal.withThreadConfinedCaching.bufferSupplier + )) } @Test @@ -701,23 +722,24 @@ class LogValidatorTest { val now = System.currentTimeMillis() val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V2, timestamp = now - 1001L, codec = CompressionType.GZIP) - assertThrows(classOf[RecordValidationException], () => LogValidator.validateMessagesAndAssignOffsets( + assertThrows(classOf[RecordValidationException], () => new LogValidator( records, topicPartition, - offsetCounter = new LongRef(0), - time = time, - now = System.currentTimeMillis(), - sourceCompression = CompressionType.GZIP, - targetCompression = CompressionType.GZIP, - magic = RecordBatch.MAGIC_VALUE_V1, - compactedTopic = false, - timestampType = TimestampType.CREATE_TIME, - timestampDiffMaxMs = 1000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching)) + time, + CompressionType.GZIP, + CompressionType.GZIP, + false, + RecordBatch.MAGIC_VALUE_V1, + TimestampType.CREATE_TIME, + 1000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(0), + metricsRecorder, + RequestLocal.withThreadConfinedCaching.bufferSupplier + )) } @Test @@ -725,22 +747,21 @@ class LogValidatorTest { val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V0, codec = CompressionType.NONE) val offset = 1234567 checkOffsets(records, 0) - checkOffsets(LogValidator.validateMessagesAndAssignOffsets(records, + checkOffsets(new LogValidator(records, topicPartition, - offsetCounter = new LongRef(offset), - time = time, - now = System.currentTimeMillis(), - sourceCompression = CompressionType.NONE, - targetCompression = CompressionType.NONE, - magic = RecordBatch.MAGIC_VALUE_V0, - compactedTopic = false, - timestampType = TimestampType.CREATE_TIME, - timestampDiffMaxMs = 1000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching).validatedRecords, offset) + time, + CompressionType.NONE, + CompressionType.NONE, + false, + RecordBatch.MAGIC_VALUE_V0, + TimestampType.CREATE_TIME, + 1000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.latest, + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching.bufferSupplier + ).validatedRecords, offset) } @Test @@ -748,22 +769,21 @@ class LogValidatorTest { val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V0, codec = CompressionType.GZIP) val offset = 1234567 checkOffsets(records, 0) - checkOffsets(LogValidator.validateMessagesAndAssignOffsets(records, + checkOffsets(new LogValidator(records, topicPartition, - offsetCounter = new LongRef(offset), - time = time, - now = System.currentTimeMillis(), - sourceCompression = CompressionType.GZIP, - targetCompression = CompressionType.GZIP, - compactedTopic = false, - magic = RecordBatch.MAGIC_VALUE_V0, - timestampType = TimestampType.CREATE_TIME, - timestampDiffMaxMs = 1000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching).validatedRecords, offset) + time, + CompressionType.GZIP, + CompressionType.GZIP, + false, + RecordBatch.MAGIC_VALUE_V0, + TimestampType.CREATE_TIME, + 1000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching.bufferSupplier + ).validatedRecords, offset) } @Test @@ -772,22 +792,21 @@ class LogValidatorTest { val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V1, timestamp = now, codec = CompressionType.NONE) val offset = 1234567 checkOffsets(records, 0) - val messageWithOffset = LogValidator.validateMessagesAndAssignOffsets(records, + val messageWithOffset = new LogValidator(records, topicPartition, - offsetCounter = new LongRef(offset), - time = time, - now = System.currentTimeMillis(), - sourceCompression = CompressionType.NONE, - targetCompression = CompressionType.NONE, - compactedTopic = false, - magic = RecordBatch.MAGIC_VALUE_V1, - timestampType = TimestampType.CREATE_TIME, - timestampDiffMaxMs = 5000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching).validatedRecords + time, + CompressionType.NONE, + CompressionType.NONE, + false, + RecordBatch.MAGIC_VALUE_V1, + TimestampType.CREATE_TIME, + 5000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching.bufferSupplier + ).validatedRecords checkOffsets(messageWithOffset, offset) } @@ -797,22 +816,21 @@ class LogValidatorTest { val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V2, timestamp = now, codec = CompressionType.NONE) val offset = 1234567 checkOffsets(records, 0) - val messageWithOffset = LogValidator.validateMessagesAndAssignOffsets(records, + val messageWithOffset = new LogValidator(records, topicPartition, - offsetCounter = new LongRef(offset), - time = time, - now = System.currentTimeMillis(), - sourceCompression = CompressionType.NONE, - targetCompression = CompressionType.NONE, - compactedTopic = false, - magic = RecordBatch.MAGIC_VALUE_V2, - timestampType = TimestampType.CREATE_TIME, - timestampDiffMaxMs = 5000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching).validatedRecords + time, + CompressionType.NONE, + CompressionType.NONE, + false, + RecordBatch.MAGIC_VALUE_V2, + TimestampType.CREATE_TIME, + 5000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching.bufferSupplier + ).validatedRecords checkOffsets(messageWithOffset, offset) } @@ -822,23 +840,22 @@ class LogValidatorTest { val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V1, timestamp = now, codec = CompressionType.GZIP) val offset = 1234567 checkOffsets(records, 0) - val compressedMessagesWithOffset = LogValidator.validateMessagesAndAssignOffsets( + val compressedMessagesWithOffset = new LogValidator( records, topicPartition, - offsetCounter = new LongRef(offset), - time = time, - now = System.currentTimeMillis(), - sourceCompression = CompressionType.GZIP, - targetCompression = CompressionType.GZIP, - compactedTopic = false, - magic = RecordBatch.MAGIC_VALUE_V1, - timestampType = TimestampType.CREATE_TIME, - timestampDiffMaxMs = 5000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching).validatedRecords + time, + CompressionType.GZIP, + CompressionType.GZIP, + false, + RecordBatch.MAGIC_VALUE_V1, + TimestampType.CREATE_TIME, + 5000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching.bufferSupplier + ).validatedRecords checkOffsets(compressedMessagesWithOffset, offset) } @@ -848,23 +865,22 @@ class LogValidatorTest { val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V2, timestamp = now, codec = CompressionType.GZIP) val offset = 1234567 checkOffsets(records, 0) - val compressedMessagesWithOffset = LogValidator.validateMessagesAndAssignOffsets( + val compressedMessagesWithOffset = new LogValidator( records, topicPartition, - offsetCounter = new LongRef(offset), - time = time, - now = System.currentTimeMillis(), - sourceCompression = CompressionType.GZIP, - targetCompression = CompressionType.GZIP, - compactedTopic = false, - magic = RecordBatch.MAGIC_VALUE_V2, - timestampType = TimestampType.CREATE_TIME, - timestampDiffMaxMs = 5000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching).validatedRecords + time, + CompressionType.GZIP, + CompressionType.GZIP, + false, + RecordBatch.MAGIC_VALUE_V2, + TimestampType.CREATE_TIME, + 5000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching.bufferSupplier + ).validatedRecords checkOffsets(compressedMessagesWithOffset, offset) } @@ -873,22 +889,21 @@ class LogValidatorTest { val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V0, codec = CompressionType.NONE) checkOffsets(records, 0) val offset = 1234567 - val validatedResults = LogValidator.validateMessagesAndAssignOffsets(records, + val validatedResults = new LogValidator(records, topicPartition, - offsetCounter = new LongRef(offset), - time = time, - now = System.currentTimeMillis(), - sourceCompression = CompressionType.NONE, - targetCompression = CompressionType.NONE, - compactedTopic = false, - magic = RecordBatch.MAGIC_VALUE_V1, - timestampType = TimestampType.LOG_APPEND_TIME, - timestampDiffMaxMs = 1000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching) + time, + CompressionType.NONE, + CompressionType.NONE, + false, + RecordBatch.MAGIC_VALUE_V1, + TimestampType.LOG_APPEND_TIME, + 1000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching.bufferSupplier + ) checkOffsets(validatedResults.validatedRecords, offset) verifyRecordConversionStats(validatedResults.recordConversionStats, numConvertedRecords = 3, records, compressed = false) @@ -899,22 +914,21 @@ class LogValidatorTest { val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V0, codec = CompressionType.NONE) checkOffsets(records, 0) val offset = 1234567 - val validatedResults = LogValidator.validateMessagesAndAssignOffsets(records, + val validatedResults = new LogValidator(records, topicPartition, - offsetCounter = new LongRef(offset), - time = time, - now = System.currentTimeMillis(), - sourceCompression = CompressionType.NONE, - targetCompression = CompressionType.NONE, - compactedTopic = false, - magic = RecordBatch.MAGIC_VALUE_V2, - timestampType = TimestampType.LOG_APPEND_TIME, - timestampDiffMaxMs = 1000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching) + time, + CompressionType.NONE, + CompressionType.NONE, + false, + RecordBatch.MAGIC_VALUE_V2, + TimestampType.LOG_APPEND_TIME, + 1000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching.bufferSupplier + ) checkOffsets(validatedResults.validatedRecords, offset) verifyRecordConversionStats(validatedResults.recordConversionStats, numConvertedRecords = 3, records, compressed = false) @@ -925,22 +939,21 @@ class LogValidatorTest { val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V0, codec = CompressionType.GZIP) val offset = 1234567 checkOffsets(records, 0) - val validatedResults = LogValidator.validateMessagesAndAssignOffsets(records, + val validatedResults = new LogValidator(records, topicPartition, - offsetCounter = new LongRef(offset), - time = time, - now = System.currentTimeMillis(), - sourceCompression = CompressionType.GZIP, - targetCompression = CompressionType.GZIP, - compactedTopic = false, - magic = RecordBatch.MAGIC_VALUE_V1, - timestampType = TimestampType.LOG_APPEND_TIME, - timestampDiffMaxMs = 1000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching) + time, + CompressionType.GZIP, + CompressionType.GZIP, + false, + RecordBatch.MAGIC_VALUE_V1, + TimestampType.LOG_APPEND_TIME, + 1000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching.bufferSupplier + ) checkOffsets(validatedResults.validatedRecords, offset) verifyRecordConversionStats(validatedResults.recordConversionStats, numConvertedRecords = 3, records, compressed = true) @@ -951,22 +964,21 @@ class LogValidatorTest { val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V0, codec = CompressionType.GZIP) val offset = 1234567 checkOffsets(records, 0) - val validatedResults = LogValidator.validateMessagesAndAssignOffsets(records, + val validatedResults = new LogValidator(records, topicPartition, - offsetCounter = new LongRef(offset), - time = time, - now = System.currentTimeMillis(), - sourceCompression = CompressionType.GZIP, - targetCompression = CompressionType.GZIP, - compactedTopic = false, - magic = RecordBatch.MAGIC_VALUE_V2, - timestampType = TimestampType.LOG_APPEND_TIME, - timestampDiffMaxMs = 1000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching) + time, + CompressionType.GZIP, + CompressionType.GZIP, + false, + RecordBatch.MAGIC_VALUE_V2, + TimestampType.LOG_APPEND_TIME, + 1000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching.bufferSupplier + ) checkOffsets(validatedResults.validatedRecords, offset) verifyRecordConversionStats(validatedResults.recordConversionStats, numConvertedRecords = 3, records, compressed = true) @@ -977,22 +989,21 @@ class LogValidatorTest { val offset = 1234567 val endTxnMarker = new EndTransactionMarker(ControlRecordType.COMMIT, 0) val records = MemoryRecords.withEndTransactionMarker(23423L, 5, endTxnMarker) - assertThrows(classOf[InvalidRecordException], () => LogValidator.validateMessagesAndAssignOffsets(records, + assertThrows(classOf[InvalidRecordException], () => new LogValidator(records, topicPartition, - offsetCounter = new LongRef(offset), - time = time, - now = System.currentTimeMillis(), - sourceCompression = CompressionType.NONE, - targetCompression = CompressionType.NONE, - compactedTopic = false, - magic = RecordBatch.CURRENT_MAGIC_VALUE, - timestampType = TimestampType.CREATE_TIME, - timestampDiffMaxMs = 5000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching)) + time, + CompressionType.NONE, + CompressionType.NONE, + false, + RecordBatch.CURRENT_MAGIC_VALUE, + TimestampType.CREATE_TIME, + 5000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching.bufferSupplier + )) } @Test @@ -1000,22 +1011,21 @@ class LogValidatorTest { val offset = 1234567 val endTxnMarker = new EndTransactionMarker(ControlRecordType.COMMIT, 0) val records = MemoryRecords.withEndTransactionMarker(23423L, 5, endTxnMarker) - val result = LogValidator.validateMessagesAndAssignOffsets(records, + val result = new LogValidator(records, topicPartition, - offsetCounter = new LongRef(offset), - time = time, - now = System.currentTimeMillis(), - sourceCompression = CompressionType.NONE, - targetCompression = CompressionType.SNAPPY, - compactedTopic = false, - magic = RecordBatch.CURRENT_MAGIC_VALUE, - timestampType = TimestampType.CREATE_TIME, - timestampDiffMaxMs = 5000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Coordinator, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching) + time, + CompressionType.NONE, + CompressionType.SNAPPY, + false, + RecordBatch.CURRENT_MAGIC_VALUE, + TimestampType.CREATE_TIME, + 5000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.COORDINATOR, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching.bufferSupplier + ) val batches = TestUtils.toList(result.validatedRecords.batches) assertEquals(1, batches.size) val batch = batches.get(0) @@ -1028,22 +1038,21 @@ class LogValidatorTest { val now = System.currentTimeMillis() val records = createRecords(RecordBatch.MAGIC_VALUE_V1, now, codec = CompressionType.NONE) checkOffsets(records, 0) - checkOffsets(LogValidator.validateMessagesAndAssignOffsets(records, + checkOffsets(new LogValidator(records, topicPartition, - offsetCounter = new LongRef(offset), - time = time, - now = System.currentTimeMillis(), - sourceCompression = CompressionType.NONE, - targetCompression = CompressionType.NONE, - compactedTopic = false, - magic = RecordBatch.MAGIC_VALUE_V0, - timestampType = TimestampType.CREATE_TIME, - timestampDiffMaxMs = 5000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching).validatedRecords, offset) + time, + CompressionType.NONE, + CompressionType.NONE, + false, + RecordBatch.MAGIC_VALUE_V0, + TimestampType.CREATE_TIME, + 5000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching.bufferSupplier + ).validatedRecords, offset) } @Test @@ -1052,22 +1061,21 @@ class LogValidatorTest { val now = System.currentTimeMillis() val records = createRecords(RecordBatch.MAGIC_VALUE_V1, now, CompressionType.GZIP) checkOffsets(records, 0) - checkOffsets(LogValidator.validateMessagesAndAssignOffsets(records, + checkOffsets(new LogValidator(records, topicPartition, - offsetCounter = new LongRef(offset), - time = time, - now = System.currentTimeMillis(), - sourceCompression = CompressionType.GZIP, - targetCompression = CompressionType.GZIP, - compactedTopic = false, - magic = RecordBatch.MAGIC_VALUE_V0, - timestampType = TimestampType.CREATE_TIME, - timestampDiffMaxMs = 5000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching).validatedRecords, offset) + time, + CompressionType.GZIP, + CompressionType.GZIP, + false, + RecordBatch.MAGIC_VALUE_V0, + TimestampType.CREATE_TIME, + 5000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching.bufferSupplier + ).validatedRecords, offset) } @Test @@ -1075,22 +1083,21 @@ class LogValidatorTest { val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V1, codec = CompressionType.NONE) checkOffsets(records, 0) val offset = 1234567 - checkOffsets(LogValidator.validateMessagesAndAssignOffsets(records, + checkOffsets(new LogValidator(records, topicPartition, - offsetCounter = new LongRef(offset), - time = time, - now = System.currentTimeMillis(), - sourceCompression = CompressionType.NONE, - targetCompression = CompressionType.NONE, - compactedTopic = false, - magic = RecordBatch.MAGIC_VALUE_V2, - timestampType = TimestampType.LOG_APPEND_TIME, - timestampDiffMaxMs = 1000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching).validatedRecords, offset) + time, + CompressionType.NONE, + CompressionType.NONE, + false, + RecordBatch.MAGIC_VALUE_V2, + TimestampType.LOG_APPEND_TIME, + 1000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching.bufferSupplier + ).validatedRecords, offset) } @Test @@ -1098,22 +1105,21 @@ class LogValidatorTest { val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V1, codec = CompressionType.GZIP) val offset = 1234567 checkOffsets(records, 0) - checkOffsets(LogValidator.validateMessagesAndAssignOffsets(records, + checkOffsets(new LogValidator(records, topicPartition, - offsetCounter = new LongRef(offset), - time = time, - now = System.currentTimeMillis(), - sourceCompression = CompressionType.GZIP, - targetCompression = CompressionType.GZIP, - compactedTopic = false, - magic = RecordBatch.MAGIC_VALUE_V2, - timestampType = TimestampType.LOG_APPEND_TIME, - timestampDiffMaxMs = 1000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching).validatedRecords, offset) + time, + CompressionType.GZIP, + CompressionType.GZIP, + false, + RecordBatch.MAGIC_VALUE_V2, + TimestampType.LOG_APPEND_TIME, + 1000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching.bufferSupplier + ).validatedRecords, offset) } @Test @@ -1122,22 +1128,21 @@ class LogValidatorTest { val now = System.currentTimeMillis() val records = createRecords(RecordBatch.MAGIC_VALUE_V2, now, codec = CompressionType.NONE) checkOffsets(records, 0) - checkOffsets(LogValidator.validateMessagesAndAssignOffsets(records, + checkOffsets(new LogValidator(records, topicPartition, - offsetCounter = new LongRef(offset), - time = time, - now = System.currentTimeMillis(), - sourceCompression = CompressionType.NONE, - targetCompression = CompressionType.NONE, - compactedTopic = false, - magic = RecordBatch.MAGIC_VALUE_V1, - timestampType = TimestampType.CREATE_TIME, - timestampDiffMaxMs = 5000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching).validatedRecords, offset) + time, + CompressionType.NONE, + CompressionType.NONE, + false, + RecordBatch.MAGIC_VALUE_V1, + TimestampType.CREATE_TIME, + 5000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching.bufferSupplier + ).validatedRecords, offset) } @Test @@ -1146,22 +1151,21 @@ class LogValidatorTest { val now = System.currentTimeMillis() val records = createRecords(RecordBatch.MAGIC_VALUE_V2, now, CompressionType.GZIP) checkOffsets(records, 0) - checkOffsets(LogValidator.validateMessagesAndAssignOffsets(records, + checkOffsets(new LogValidator(records, topicPartition, - offsetCounter = new LongRef(offset), - time = time, - now = System.currentTimeMillis(), - sourceCompression = CompressionType.GZIP, - targetCompression = CompressionType.GZIP, - compactedTopic = false, - magic = RecordBatch.MAGIC_VALUE_V1, - timestampType = TimestampType.CREATE_TIME, - timestampDiffMaxMs = 5000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching).validatedRecords, offset) + time, + CompressionType.GZIP, + CompressionType.GZIP, + false, + RecordBatch.MAGIC_VALUE_V1, + TimestampType.CREATE_TIME, + 5000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching.bufferSupplier + ).validatedRecords, offset) } @Test @@ -1172,22 +1176,21 @@ class LogValidatorTest { val sequence = 0 val records = MemoryRecords.withTransactionalRecords(CompressionType.NONE, producerId, producerEpoch, sequence, new SimpleRecord("hello".getBytes), new SimpleRecord("there".getBytes), new SimpleRecord("beautiful".getBytes)) - assertThrows(classOf[UnsupportedForMessageFormatException], () => LogValidator.validateMessagesAndAssignOffsets(records, + assertThrows(classOf[UnsupportedForMessageFormatException], () => new LogValidator(records, topicPartition, - offsetCounter = new LongRef(offset), - time = time, - now = System.currentTimeMillis(), - sourceCompression = CompressionType.GZIP, - targetCompression = CompressionType.GZIP, - compactedTopic = false, - magic = RecordBatch.MAGIC_VALUE_V1, - timestampType = TimestampType.CREATE_TIME, - timestampDiffMaxMs = 5000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching)) + time, + CompressionType.GZIP, + CompressionType.GZIP, + false, + RecordBatch.MAGIC_VALUE_V1, + TimestampType.CREATE_TIME, + 5000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching.bufferSupplier + )) } @Test @@ -1198,22 +1201,21 @@ class LogValidatorTest { val sequence = 0 val records = MemoryRecords.withIdempotentRecords(CompressionType.NONE, producerId, producerEpoch, sequence, new SimpleRecord("hello".getBytes), new SimpleRecord("there".getBytes), new SimpleRecord("beautiful".getBytes)) - assertThrows(classOf[UnsupportedForMessageFormatException], () => LogValidator.validateMessagesAndAssignOffsets(records, + assertThrows(classOf[UnsupportedForMessageFormatException], () => new LogValidator(records, topicPartition, - offsetCounter = new LongRef(offset), - time = time, - now = System.currentTimeMillis(), - sourceCompression = CompressionType.GZIP, - targetCompression = CompressionType.GZIP, - compactedTopic = false, - magic = RecordBatch.MAGIC_VALUE_V1, - timestampType = TimestampType.CREATE_TIME, - timestampDiffMaxMs = 5000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching)) + time, + CompressionType.GZIP, + CompressionType.GZIP, + false, + RecordBatch.MAGIC_VALUE_V1, + TimestampType.CREATE_TIME, + 5000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching.bufferSupplier + )) } @Test @@ -1222,22 +1224,21 @@ class LogValidatorTest { val now = System.currentTimeMillis() val records = createRecords(RecordBatch.MAGIC_VALUE_V2, now, codec = CompressionType.NONE) checkOffsets(records, 0) - checkOffsets(LogValidator.validateMessagesAndAssignOffsets(records, + checkOffsets(new LogValidator(records, topicPartition, - offsetCounter = new LongRef(offset), - time = time, - now = System.currentTimeMillis(), - sourceCompression = CompressionType.NONE, - targetCompression = CompressionType.NONE, - compactedTopic = false, - magic = RecordBatch.MAGIC_VALUE_V0, - timestampType = TimestampType.CREATE_TIME, - timestampDiffMaxMs = 5000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching).validatedRecords, offset) + time, + CompressionType.NONE, + CompressionType.NONE, + false, + RecordBatch.MAGIC_VALUE_V0, + TimestampType.CREATE_TIME, + 5000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching.bufferSupplier + ).validatedRecords, offset) } @Test @@ -1246,45 +1247,42 @@ class LogValidatorTest { val now = System.currentTimeMillis() val records = createRecords(RecordBatch.MAGIC_VALUE_V2, now, CompressionType.GZIP) checkOffsets(records, 0) - checkOffsets(LogValidator.validateMessagesAndAssignOffsets(records, + checkOffsets(new LogValidator(records, topicPartition, - offsetCounter = new LongRef(offset), - time = time, - now = System.currentTimeMillis(), - sourceCompression = CompressionType.GZIP, - targetCompression = CompressionType.GZIP, - compactedTopic = false, - magic = RecordBatch.MAGIC_VALUE_V0, - timestampType = TimestampType.CREATE_TIME, - timestampDiffMaxMs = 5000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching).validatedRecords, offset) + time, + CompressionType.GZIP, + CompressionType.GZIP, + false, + RecordBatch.MAGIC_VALUE_V0, + TimestampType.CREATE_TIME, + 5000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching.bufferSupplier + ).validatedRecords, offset) } @Test def testNonIncreasingOffsetRecordBatchHasMetricsLogged(): Unit = { val records = createNonIncreasingOffsetRecords(RecordBatch.MAGIC_VALUE_V2) records.batches().asScala.head.setLastOffset(2) - assertThrows(classOf[InvalidRecordException], () => LogValidator.validateMessagesAndAssignOffsets(records, + assertThrows(classOf[InvalidRecordException], () => new LogValidator(records, topicPartition, - offsetCounter = new LongRef(0L), - time = time, - now = System.currentTimeMillis(), - sourceCompression = CompressionType.GZIP, - targetCompression = CompressionType.GZIP, - compactedTopic = false, - magic = RecordBatch.MAGIC_VALUE_V0, - timestampType = TimestampType.CREATE_TIME, - timestampDiffMaxMs = 5000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching) - ) + time, + CompressionType.GZIP, + CompressionType.GZIP, + false, + RecordBatch.MAGIC_VALUE_V0, + TimestampType.CREATE_TIME, + 5000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(0L), metricsRecorder, RequestLocal.withThreadConfinedCaching.bufferSupplier + )) assertEquals(metricsKeySet.count(_.getMBeanName.endsWith(s"${BrokerTopicStats.InvalidOffsetOrSequenceRecordsPerSec}")), 1) assertTrue(meterCount(s"${BrokerTopicStats.InvalidOffsetOrSequenceRecordsPerSec}") > 0) } @@ -1296,25 +1294,23 @@ class LogValidatorTest { @Test def testZStdCompressedWithUnavailableIBPVersion(): Unit = { - val now = System.currentTimeMillis() // The timestamps should be overwritten val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V2, timestamp = 1234L, codec = CompressionType.NONE) - assertThrows(classOf[UnsupportedCompressionTypeException], () => LogValidator.validateMessagesAndAssignOffsets(records, + assertThrows(classOf[UnsupportedCompressionTypeException], () => new LogValidator(records, topicPartition, - offsetCounter = new LongRef(0), - time= time, - now = now, - sourceCompression = CompressionType.NONE, - targetCompression = CompressionType.ZSTD, - compactedTopic = false, - magic = RecordBatch.MAGIC_VALUE_V2, - timestampType = TimestampType.LOG_APPEND_TIME, - timestampDiffMaxMs = 1000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.IBP_2_0_IV1, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching)) + time, + CompressionType.NONE, + CompressionType.ZSTD, + false, + RecordBatch.MAGIC_VALUE_V2, + TimestampType.LOG_APPEND_TIME, + 1000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.IBP_2_0_IV1 + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(0L), metricsRecorder, RequestLocal.withThreadConfinedCaching.bufferSupplier + )) } @Test @@ -1333,27 +1329,26 @@ class LogValidatorTest { val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V2, timestamp = now - 1001L, codec = CompressionType.GZIP) val e = assertThrows(classOf[RecordValidationException], - () => LogValidator.validateMessagesAndAssignOffsets( + () => new LogValidator( records, topicPartition, - offsetCounter = new LongRef(0), - time = time, - now = System.currentTimeMillis(), - sourceCompression = CompressionType.GZIP, - targetCompression = CompressionType.GZIP, - magic = RecordBatch.MAGIC_VALUE_V1, - compactedTopic = false, - timestampType = TimestampType.CREATE_TIME, - timestampDiffMaxMs = 1000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching) + time, + CompressionType.GZIP, + CompressionType.GZIP, + false, + RecordBatch.MAGIC_VALUE_V1, + TimestampType.CREATE_TIME, + 1000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(0L), metricsRecorder, RequestLocal.withThreadConfinedCaching.bufferSupplier + ) ) assertTrue(e.invalidException.isInstanceOf[InvalidTimestampException]) - assertTrue(e.recordErrors.nonEmpty) + assertFalse(e.recordErrors.isEmpty) assertEquals(e.recordErrors.size, 3) } @@ -1366,10 +1361,10 @@ class LogValidatorTest { ) assertTrue(e.invalidException.isInstanceOf[InvalidRecordException]) - assertTrue(e.recordErrors.nonEmpty) + assertFalse(e.recordErrors.isEmpty) // recordsWithInvalidInnerMagic creates 20 records assertEquals(e.recordErrors.size, 20) - e.recordErrors.foreach(assertNotNull(_)) + e.recordErrors.asScala.foreach(assertNotNull(_)) } @Test @@ -1399,7 +1394,7 @@ class LogValidatorTest { // if there is a mix of both regular InvalidRecordException and InvalidTimestampException, // InvalidTimestampException takes precedence assertTrue(e.invalidException.isInstanceOf[InvalidTimestampException]) - assertTrue(e.recordErrors.nonEmpty) + assertFalse(e.recordErrors.isEmpty) assertEquals(6, e.recordErrors.size) } @@ -1413,22 +1408,21 @@ class LogValidatorTest { isTransactional, false) buffer.flip() val records = MemoryRecords.readableRecords(buffer) - assertThrows(classOf[InvalidRecordException], () => LogValidator.validateMessagesAndAssignOffsets(records, + assertThrows(classOf[InvalidRecordException], () => new LogValidator(records, topicPartition, - offsetCounter = new LongRef(offset), - time = time, - now = System.currentTimeMillis(), - sourceCompression = sourceCompression, - targetCompression = targetCompression, - compactedTopic = false, - magic = RecordBatch.CURRENT_MAGIC_VALUE, - timestampType = TimestampType.CREATE_TIME, - timestampDiffMaxMs = 5000L, - partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH, - origin = AppendOrigin.Client, - interBrokerProtocolVersion = MetadataVersion.latest, - brokerTopicStats = brokerTopicStats, - requestLocal = RequestLocal.withThreadConfinedCaching)) + time, + sourceCompression, + targetCompression, + false, + RecordBatch.CURRENT_MAGIC_VALUE, + TimestampType.CREATE_TIME, + 5000L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + AppendOrigin.CLIENT, + MetadataVersion.latest + ).validateMessagesAndAssignOffsets( + PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching.bufferSupplier + )) } private def createRecords(magicValue: Byte, diff --git a/core/src/test/scala/unit/kafka/log/OffsetMapTest.scala b/core/src/test/scala/unit/kafka/log/OffsetMapTest.scala index dd5294bf92b3b..8f3885920dadd 100644 --- a/core/src/test/scala/unit/kafka/log/OffsetMapTest.scala +++ b/core/src/test/scala/unit/kafka/log/OffsetMapTest.scala @@ -18,8 +18,8 @@ package kafka.log import java.nio._ - import kafka.utils.Exit +import org.apache.kafka.server.log.internals.SkimpyOffsetMap import org.junit.jupiter.api._ import org.junit.jupiter.api.Assertions._ diff --git a/core/src/test/scala/unit/kafka/log/ProducerStateManagerTest.scala b/core/src/test/scala/unit/kafka/log/ProducerStateManagerTest.scala index c808d03a72e2b..359c46de0b733 100644 --- a/core/src/test/scala/unit/kafka/log/ProducerStateManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/ProducerStateManagerTest.scala @@ -30,7 +30,7 @@ import org.apache.kafka.common.errors._ import org.apache.kafka.common.internals.Topic import org.apache.kafka.common.record._ import org.apache.kafka.common.utils.{MockTime, Utils} -import org.apache.kafka.server.log.internals.CompletedTxn +import org.apache.kafka.server.log.internals.{AppendOrigin, CompletedTxn} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} import org.mockito.Mockito.{mock, when} @@ -110,7 +110,7 @@ class ProducerStateManagerTest { val epoch = 15.toShort val sequence = Int.MaxValue val offset = 735L - append(stateManager, producerId, epoch, sequence, offset, origin = AppendOrigin.Replication) + append(stateManager, producerId, epoch, sequence, offset, origin = AppendOrigin.REPLICATION) append(stateManager, producerId, epoch, 0, offset + 500) @@ -128,7 +128,7 @@ class ProducerStateManagerTest { def testProducerSequenceWithWrapAroundBatchRecord(): Unit = { val epoch = 15.toShort - val appendInfo = stateManager.prepareUpdate(producerId, origin = AppendOrigin.Replication) + val appendInfo = stateManager.prepareUpdate(producerId, origin = AppendOrigin.REPLICATION) // Sequence number wrap around appendInfo.appendDataBatch(epoch, Int.MaxValue - 10, 9, time.milliseconds(), LogOffsetMetadata(2000L), 2020L, isTransactional = false) @@ -148,7 +148,7 @@ class ProducerStateManagerTest { val epoch = 15.toShort val sequence = Int.MaxValue val offset = 735L - append(stateManager, producerId, epoch, sequence, offset, origin = AppendOrigin.Replication) + append(stateManager, producerId, epoch, sequence, offset, origin = AppendOrigin.REPLICATION) assertThrows(classOf[OutOfOrderSequenceException], () => append(stateManager, producerId, epoch, 1, offset + 500)) } @@ -157,7 +157,7 @@ class ProducerStateManagerTest { val epoch = 5.toShort val sequence = 16 val offset = 735L - append(stateManager, producerId, epoch, sequence, offset, origin = AppendOrigin.Replication) + append(stateManager, producerId, epoch, sequence, offset, origin = AppendOrigin.REPLICATION) val maybeLastEntry = stateManager.lastEntry(producerId) assertTrue(maybeLastEntry.isDefined) @@ -197,7 +197,7 @@ class ProducerStateManagerTest { val producerEpoch = 0.toShort val offset = 992342L val seq = 0 - val producerAppendInfo = new ProducerAppendInfo(partition, producerId, ProducerStateEntry.empty(producerId), AppendOrigin.Client) + val producerAppendInfo = new ProducerAppendInfo(partition, producerId, ProducerStateEntry.empty(producerId), AppendOrigin.CLIENT) val firstOffsetMetadata = LogOffsetMetadata(messageOffset = offset, segmentBaseOffset = 990000L, relativePositionInSegment = 234224) @@ -235,7 +235,7 @@ class ProducerStateManagerTest { } // Start one transaction in a separate append - val firstAppend = stateManager.prepareUpdate(producerId, origin = AppendOrigin.Client) + val firstAppend = stateManager.prepareUpdate(producerId, origin = AppendOrigin.CLIENT) appendData(16L, 20L, firstAppend) assertEquals(new TxnMetadata(producerId, 16L), firstAppend.startedTransactions.head) stateManager.update(firstAppend) @@ -245,7 +245,7 @@ class ProducerStateManagerTest { // Now do a single append which completes the old transaction, mixes in // some empty transactions, one non-empty complete transaction, and one // incomplete transaction - val secondAppend = stateManager.prepareUpdate(producerId, origin = AppendOrigin.Client) + val secondAppend = stateManager.prepareUpdate(producerId, origin = AppendOrigin.CLIENT) val firstCompletedTxn = appendEndTxn(ControlRecordType.COMMIT, 21, secondAppend) assertEquals(Some(new CompletedTxn(producerId, 16L, 21, false)), firstCompletedTxn) assertEquals(None, appendEndTxn(ControlRecordType.COMMIT, 22, secondAppend)) @@ -371,7 +371,7 @@ class ProducerStateManagerTest { partition, producerId, ProducerStateEntry.empty(producerId), - AppendOrigin.Client + AppendOrigin.CLIENT ) val firstOffsetMetadata = LogOffsetMetadata(messageOffset = startOffset, segmentBaseOffset = segmentBaseOffset, relativePositionInSegment = 50 * relativeOffset) @@ -418,14 +418,14 @@ class ProducerStateManagerTest { def testPrepareUpdateDoesNotMutate(): Unit = { val producerEpoch = 0.toShort - val appendInfo = stateManager.prepareUpdate(producerId, origin = AppendOrigin.Client) + val appendInfo = stateManager.prepareUpdate(producerId, origin = AppendOrigin.CLIENT) appendInfo.appendDataBatch(producerEpoch, 0, 5, time.milliseconds(), LogOffsetMetadata(15L), 20L, isTransactional = false) assertEquals(None, stateManager.lastEntry(producerId)) stateManager.update(appendInfo) assertTrue(stateManager.lastEntry(producerId).isDefined) - val nextAppendInfo = stateManager.prepareUpdate(producerId, origin = AppendOrigin.Client) + val nextAppendInfo = stateManager.prepareUpdate(producerId, origin = AppendOrigin.CLIENT) nextAppendInfo.appendDataBatch(producerEpoch, 6, 10, time.milliseconds(), LogOffsetMetadata(26L), 30L, isTransactional = false) assertTrue(stateManager.lastEntry(producerId).isDefined) @@ -449,7 +449,7 @@ class ProducerStateManagerTest { val offset = 9L append(stateManager, producerId, producerEpoch, 0, offset) - val appendInfo = stateManager.prepareUpdate(producerId, origin = AppendOrigin.Client) + val appendInfo = stateManager.prepareUpdate(producerId, origin = AppendOrigin.CLIENT) appendInfo.appendDataBatch(producerEpoch, 1, 5, time.milliseconds(), LogOffsetMetadata(16L), 20L, isTransactional = true) var lastEntry = appendInfo.toEntry @@ -670,7 +670,7 @@ class ProducerStateManagerTest { // entry added after recovery. The pid should be expired now, and would not exist in the pid mapping. Nonetheless // the append on a replica should be accepted with the local producer state updated to the appended value. assertFalse(recoveredMapping.activeProducers.contains(producerId)) - append(recoveredMapping, producerId, epoch, sequence, 2L, 70001, origin = AppendOrigin.Replication) + append(recoveredMapping, producerId, epoch, sequence, 2L, 70001, origin = AppendOrigin.REPLICATION) assertTrue(recoveredMapping.activeProducers.contains(producerId)) val producerStateEntry = recoveredMapping.activeProducers.get(producerId).head assertEquals(epoch, producerStateEntry.producerEpoch) @@ -685,10 +685,10 @@ class ProducerStateManagerTest { val outOfOrderSequence = 3 // First we ensure that we raise an OutOfOrderSequenceException is raised when the append comes from a client. - assertThrows(classOf[OutOfOrderSequenceException], () => append(stateManager, producerId, epoch, outOfOrderSequence, 1L, 1, origin = AppendOrigin.Client)) + assertThrows(classOf[OutOfOrderSequenceException], () => append(stateManager, producerId, epoch, outOfOrderSequence, 1L, 1, origin = AppendOrigin.CLIENT)) assertEquals(0L, stateManager.activeProducers(producerId).lastSeq) - append(stateManager, producerId, epoch, outOfOrderSequence, 1L, 1, origin = AppendOrigin.Replication) + append(stateManager, producerId, epoch, outOfOrderSequence, 1L, 1, origin = AppendOrigin.REPLICATION) assertEquals(outOfOrderSequence, stateManager.activeProducers(producerId).lastSeq) } @@ -905,9 +905,9 @@ class ProducerStateManagerTest { val epoch = 0.toShort append(stateManager, producerId, epoch, RecordBatch.NO_SEQUENCE, offset = 99, - isTransactional = true, origin = AppendOrigin.Coordinator) + isTransactional = true, origin = AppendOrigin.COORDINATOR) append(stateManager, producerId, epoch, RecordBatch.NO_SEQUENCE, offset = 100, - isTransactional = true, origin = AppendOrigin.Coordinator) + isTransactional = true, origin = AppendOrigin.COORDINATOR) } @Test @@ -994,7 +994,7 @@ class ProducerStateManagerTest { when(batch.iterator).thenReturn(Collections.emptyIterator[Record]) // Appending the empty control batch should not throw and a new transaction shouldn't be started - append(stateManager, producerId, baseOffset, batch, origin = AppendOrigin.Client) + append(stateManager, producerId, baseOffset, batch, origin = AppendOrigin.CLIENT) assertEquals(None, stateManager.lastEntry(producerId).get.currentTxnFirstOffset) } @@ -1102,7 +1102,7 @@ class ProducerStateManagerTest { offset: Long, coordinatorEpoch: Int = 0, timestamp: Long = time.milliseconds()): Option[CompletedTxn] = { - val producerAppendInfo = stateManager.prepareUpdate(producerId, origin = AppendOrigin.Coordinator) + val producerAppendInfo = stateManager.prepareUpdate(producerId, origin = AppendOrigin.COORDINATOR) val endTxnMarker = new EndTransactionMarker(controlType, coordinatorEpoch) val completedTxnOpt = producerAppendInfo.appendEndTxnMarker(endTxnMarker, producerEpoch, offset, timestamp) mapping.update(producerAppendInfo) @@ -1118,7 +1118,7 @@ class ProducerStateManagerTest { offset: Long, timestamp: Long = time.milliseconds(), isTransactional: Boolean = false, - origin : AppendOrigin = AppendOrigin.Client): Unit = { + origin : AppendOrigin = AppendOrigin.CLIENT): Unit = { val producerAppendInfo = stateManager.prepareUpdate(producerId, origin) producerAppendInfo.appendDataBatch(producerEpoch, seq, seq, timestamp, LogOffsetMetadata(offset), offset, isTransactional) diff --git a/core/src/test/scala/unit/kafka/log/TimeIndexTest.scala b/core/src/test/scala/unit/kafka/log/TimeIndexTest.scala index 49ce3f64ba68c..8502eb06535e2 100644 --- a/core/src/test/scala/unit/kafka/log/TimeIndexTest.scala +++ b/core/src/test/scala/unit/kafka/log/TimeIndexTest.scala @@ -20,7 +20,7 @@ package kafka.log import java.io.File import kafka.utils.TestUtils import org.apache.kafka.common.errors.InvalidOffsetException -import org.apache.kafka.server.log.internals.{CorruptIndexException, TimestampOffset} +import org.apache.kafka.server.log.internals.{CorruptIndexException, TimeIndex, TimestampOffset} import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} import org.junit.jupiter.api.Assertions.{assertEquals, assertThrows} @@ -34,7 +34,7 @@ class TimeIndexTest { @BeforeEach def setup(): Unit = { - this.idx = new TimeIndex(nonExistantTempFile(), baseOffset = baseOffset, maxIndexSize = maxEntries * 12) + this.idx = new TimeIndex(nonExistantTempFile(), baseOffset, maxEntries * 12) } @AfterEach @@ -114,7 +114,7 @@ class TimeIndexTest { var shouldCorruptOffset = false var shouldCorruptTimestamp = false var shouldCorruptLength = false - idx = new TimeIndex(idx.file, baseOffset = baseOffset, maxIndexSize = maxEntries * 12) { + idx = new TimeIndex(idx.file, baseOffset, maxEntries * 12) { override def lastEntry = { val superLastEntry = super.lastEntry val offset = if (shouldCorruptOffset) baseOffset - 1 else superLastEntry.offset diff --git a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala index 45335eec8eca5..4a28079501708 100755 --- a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala +++ b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala @@ -22,7 +22,7 @@ import java.nio.ByteBuffer import java.nio.file.Files import java.util.concurrent.{Callable, ConcurrentHashMap, Executors} import java.util.{Optional, Properties} -import kafka.common.{OffsetsOutOfOrderException, RecordValidationException, UnexpectedAppendOffsetException} +import kafka.common.{OffsetsOutOfOrderException, UnexpectedAppendOffsetException} import kafka.log.remote.RemoteLogManager import kafka.server.checkpoints.LeaderEpochCheckpointFile import kafka.server.epoch.{EpochEntry, LeaderEpochFileCache} @@ -37,7 +37,7 @@ import org.apache.kafka.common.record.MemoryRecords.RecordFilter import org.apache.kafka.common.record._ import org.apache.kafka.common.requests.{ListOffsetsRequest, ListOffsetsResponse} import org.apache.kafka.common.utils.{BufferSupplier, Time, Utils} -import org.apache.kafka.server.log.internals.AbortedTxn +import org.apache.kafka.server.log.internals.{AbortedTxn, AppendOrigin, RecordValidationException} import org.apache.kafka.server.log.remote.metadata.storage.TopicBasedRemoteLogMetadataManagerConfig import org.apache.kafka.server.metrics.KafkaYammerMetrics import org.junit.jupiter.api.Assertions._ @@ -127,16 +127,16 @@ class UnifiedLogTest { new SimpleRecord(mockTime.milliseconds, "c".getBytes, "value".getBytes) ), baseOffset = offset, partitionLeaderEpoch = leaderEpoch) - log.appendAsLeader(records(0), leaderEpoch, AppendOrigin.RaftLeader) + log.appendAsLeader(records(0), leaderEpoch, AppendOrigin.RAFT_LEADER) assertEquals(0, log.logStartOffset) assertEquals(3L, log.logEndOffset) // Since raft leader is responsible for assigning offsets, and the LogValidator is bypassed from the performance perspective, // so the first offset of the MemoryRecords to be append should equal to the next offset in the log - assertThrows(classOf[UnexpectedAppendOffsetException], () => (log.appendAsLeader(records(1), leaderEpoch, AppendOrigin.RaftLeader))) + assertThrows(classOf[UnexpectedAppendOffsetException], () => log.appendAsLeader(records(1), leaderEpoch, AppendOrigin.RAFT_LEADER)) // When the first offset of the MemoryRecords to be append equals to the next offset in the log, append will succeed - log.appendAsLeader(records(3), leaderEpoch, AppendOrigin.RaftLeader) + log.appendAsLeader(records(3), leaderEpoch, AppendOrigin.RAFT_LEADER) assertEquals(6, log.logEndOffset) } @@ -1788,22 +1788,22 @@ class UnifiedLogTest { () => log.appendAsLeader(messageSetWithUnkeyedMessage, leaderEpoch = 0)) assertTrue(e.invalidException.isInstanceOf[InvalidRecordException]) assertEquals(1, e.recordErrors.size) - assertEquals(0, e.recordErrors.head.batchIndex) - assertTrue(e.recordErrors.head.message.startsWith(errorMsgPrefix)) + assertEquals(0, e.recordErrors.get(0).batchIndex) + assertTrue(e.recordErrors.get(0).message.startsWith(errorMsgPrefix)) e = assertThrows(classOf[RecordValidationException], () => log.appendAsLeader(messageSetWithOneUnkeyedMessage, leaderEpoch = 0)) assertTrue(e.invalidException.isInstanceOf[InvalidRecordException]) assertEquals(1, e.recordErrors.size) - assertEquals(0, e.recordErrors.head.batchIndex) - assertTrue(e.recordErrors.head.message.startsWith(errorMsgPrefix)) + assertEquals(0, e.recordErrors.get(0).batchIndex) + assertTrue(e.recordErrors.get(0).message.startsWith(errorMsgPrefix)) e = assertThrows(classOf[RecordValidationException], () => log.appendAsLeader(messageSetWithCompressedUnkeyedMessage, leaderEpoch = 0)) assertTrue(e.invalidException.isInstanceOf[InvalidRecordException]) assertEquals(1, e.recordErrors.size) - assertEquals(1, e.recordErrors.head.batchIndex) // batch index is 1 - assertTrue(e.recordErrors.head.message.startsWith(errorMsgPrefix)) + assertEquals(1, e.recordErrors.get(0).batchIndex) // batch index is 1 + assertTrue(e.recordErrors.get(0).message.startsWith(errorMsgPrefix)) // check if metric for NoKeyCompactedTopicRecordsPerSec is logged assertEquals(metricsKeySet.count(_.getMBeanName.endsWith(s"${BrokerTopicStats.NoKeyCompactedTopicRecordsPerSec}")), 1) diff --git a/core/src/test/scala/unit/kafka/log/remote/RemoteIndexCacheTest.scala b/core/src/test/scala/unit/kafka/log/remote/RemoteIndexCacheTest.scala index 102b137dd6510..3c45f0916c463 100644 --- a/core/src/test/scala/unit/kafka/log/remote/RemoteIndexCacheTest.scala +++ b/core/src/test/scala/unit/kafka/log/remote/RemoteIndexCacheTest.scala @@ -16,10 +16,10 @@ */ package kafka.log.remote -import kafka.log.{TimeIndex, UnifiedLog} +import kafka.log.UnifiedLog import kafka.utils.MockTime import org.apache.kafka.common.{TopicIdPartition, TopicPartition, Uuid} -import org.apache.kafka.server.log.internals.{OffsetIndex, OffsetPosition} +import org.apache.kafka.server.log.internals.{OffsetIndex, OffsetPosition, TimeIndex} import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType import org.apache.kafka.server.log.remote.storage.{RemoteLogSegmentId, RemoteLogSegmentMetadata, RemoteStorageManager} import org.apache.kafka.test.TestUtils @@ -65,7 +65,7 @@ class RemoteIndexCacheTest { val offsetIdx = new OffsetIndex(new File(tpDir, String.valueOf(metadata.startOffset()) + UnifiedLog.IndexFileSuffix), metadata.startOffset(), maxEntries * 8) val timeIdx = new TimeIndex(new File(tpDir, String.valueOf(metadata.startOffset()) + UnifiedLog.TimeIndexFileSuffix), - metadata.startOffset(), maxIndexSize = maxEntries * 12) + metadata.startOffset(), maxEntries * 12) maybeAppendIndexEntries(offsetIdx, timeIdx) indexType match { case IndexType.OFFSET => new FileInputStream(offsetIdx.file) @@ -229,7 +229,7 @@ class RemoteIndexCacheTest { for (i <- 0 until offsetIndex.maxEntries) { val offset = offsetIndex.baseOffset + i offsetIndex.append(offset, i) - timeIndex.maybeAppend(curTime + i, offset, skipFullCheck = true) + timeIndex.maybeAppend(curTime + i, offset, true) } offsetIndex.flush() timeIndex.flush() diff --git a/core/src/test/scala/unit/kafka/log/remote/RemoteLogManagerTest.scala b/core/src/test/scala/unit/kafka/log/remote/RemoteLogManagerTest.scala index 142921d6bdd5e..8876dbd27f703 100644 --- a/core/src/test/scala/unit/kafka/log/remote/RemoteLogManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/remote/RemoteLogManagerTest.scala @@ -17,7 +17,7 @@ package kafka.log.remote import kafka.cluster.Partition -import kafka.log.{TimeIndex, UnifiedLog} +import kafka.log.UnifiedLog import kafka.server.KafkaConfig import kafka.server.checkpoints.LeaderEpochCheckpoint import kafka.server.epoch.{EpochEntry, LeaderEpochFileCache} @@ -26,7 +26,7 @@ import org.apache.kafka.common.config.AbstractConfig import org.apache.kafka.common.record.FileRecords.TimestampAndOffset import org.apache.kafka.common.record.{CompressionType, MemoryRecords, SimpleRecord} import org.apache.kafka.common.{KafkaException, TopicIdPartition, TopicPartition, Uuid} -import org.apache.kafka.server.log.internals.OffsetIndex +import org.apache.kafka.server.log.internals.{OffsetIndex, TimeIndex} import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType import org.apache.kafka.server.log.remote.storage._ import org.apache.kafka.test.TestUtils @@ -190,7 +190,7 @@ class RemoteLogManagerTest { val offsetIdx = new OffsetIndex(new File(tpDir, String.valueOf(metadata.startOffset()) + UnifiedLog.IndexFileSuffix), metadata.startOffset(), maxEntries * 8) val timeIdx = new TimeIndex(new File(tpDir, String.valueOf(metadata.startOffset()) + UnifiedLog.TimeIndexFileSuffix), - metadata.startOffset(), maxIndexSize = maxEntries * 12) + metadata.startOffset(), maxEntries * 12) indexType match { case IndexType.OFFSET => new FileInputStream(offsetIdx.file) case IndexType.TIMESTAMP => new FileInputStream(timeIdx.file) diff --git a/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala b/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala index b03427b1aa78b..283a8daec4f64 100755 --- a/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala +++ b/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala @@ -30,6 +30,7 @@ import kafka.cluster.Partition import kafka.server.metadata.MockConfigRepository import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.record.SimpleRecord +import org.apache.kafka.server.log.internals.LogDirFailureChannel class HighwatermarkPersistenceTest { diff --git a/core/src/test/scala/unit/kafka/server/IsrExpirationTest.scala b/core/src/test/scala/unit/kafka/server/IsrExpirationTest.scala index baed478a0bfeb..e4d929553e46c 100644 --- a/core/src/test/scala/unit/kafka/server/IsrExpirationTest.scala +++ b/core/src/test/scala/unit/kafka/server/IsrExpirationTest.scala @@ -28,6 +28,7 @@ import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.utils.Time import org.apache.kafka.metadata.LeaderRecoveryState +import org.apache.kafka.server.log.internals.LogDirFailureChannel import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} import org.mockito.Mockito.{atLeastOnce, mock, verify, when} diff --git a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala index 848f35425495d..4b039b5543f7e 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala @@ -28,7 +28,6 @@ import kafka.cluster.Broker import kafka.controller.{ControllerContext, KafkaController} import kafka.coordinator.group._ import kafka.coordinator.transaction.{InitProducerIdResult, TransactionCoordinator} -import kafka.log.AppendOrigin import kafka.network.RequestChannel import kafka.server.QuotaFactory.QuotaManagers import kafka.server.metadata.{ConfigRepository, KRaftMetadataCache, MockConfigRepository, ZkMetadataCache} @@ -93,6 +92,7 @@ import org.apache.kafka.common.message.CreatePartitionsRequestData.CreatePartiti import org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.server.common.MetadataVersion.{IBP_0_10_2_IV0, IBP_2_2_IV1} +import org.apache.kafka.server.log.internals.AppendOrigin class KafkaApisTest { private val requestChannel: RequestChannel = mock(classOf[RequestChannel]) @@ -1645,7 +1645,7 @@ class KafkaApisTest { when(replicaManager.appendRecords(anyLong, anyShort, ArgumentMatchers.eq(false), - ArgumentMatchers.eq(AppendOrigin.Client), + ArgumentMatchers.eq(AppendOrigin.CLIENT), any(), responseCallback.capture(), any(), @@ -1786,7 +1786,7 @@ class KafkaApisTest { when(replicaManager.appendRecords(anyLong, anyShort, ArgumentMatchers.eq(true), - ArgumentMatchers.eq(AppendOrigin.Coordinator), + ArgumentMatchers.eq(AppendOrigin.COORDINATOR), any(), responseCallback.capture(), any(), @@ -1916,7 +1916,7 @@ class KafkaApisTest { when(replicaManager.appendRecords(anyLong, anyShort, ArgumentMatchers.eq(true), - ArgumentMatchers.eq(AppendOrigin.Coordinator), + ArgumentMatchers.eq(AppendOrigin.COORDINATOR), any(), responseCallback.capture(), any(), @@ -1948,7 +1948,7 @@ class KafkaApisTest { verify(replicaManager).appendRecords(anyLong, anyShort, ArgumentMatchers.eq(true), - ArgumentMatchers.eq(AppendOrigin.Coordinator), + ArgumentMatchers.eq(AppendOrigin.COORDINATOR), any(), any(), any(), diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerConcurrencyTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerConcurrencyTest.scala index 559485ff4ddd6..1b91f61724bcc 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerConcurrencyTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerConcurrencyTest.scala @@ -21,7 +21,7 @@ import java.util import java.util.concurrent.{CompletableFuture, Executors, LinkedBlockingQueue, TimeUnit} import java.util.{Optional, Properties} import kafka.api.LeaderAndIsr -import kafka.log.{AppendOrigin, LogConfig} +import kafka.log.LogConfig import kafka.server.metadata.KRaftMetadataCache import kafka.server.metadata.MockConfigRepository import kafka.utils.TestUtils.waitUntilTrue @@ -39,6 +39,7 @@ import org.apache.kafka.common.{IsolationLevel, TopicIdPartition, TopicPartition import org.apache.kafka.image.{MetadataDelta, MetadataImage} import org.apache.kafka.metadata.LeaderRecoveryState import org.apache.kafka.metadata.PartitionRegistration +import org.apache.kafka.server.log.internals.{AppendOrigin, LogDirFailureChannel} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, Test} import org.mockito.Mockito @@ -283,7 +284,7 @@ class ReplicaManagerConcurrencyTest { timeout = 30000, requiredAcks = (-1).toShort, internalTopicsAllowed = false, - origin = AppendOrigin.Client, + origin = AppendOrigin.CLIENT, entriesPerPartition = collection.Map(topicPartition -> TestUtils.records(records)), responseCallback = produceCallback ) diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala index d0826f54c5d3a..a4d907c4acc8f 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala @@ -29,6 +29,7 @@ import org.apache.kafka.common.requests.FetchRequest import org.apache.kafka.common.requests.FetchRequest.PartitionData import org.apache.kafka.common.{TopicIdPartition, TopicPartition, Uuid} import org.apache.kafka.metadata.LeaderRecoveryState +import org.apache.kafka.server.log.internals.LogDirFailureChannel import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, Test} import org.mockito.ArgumentMatchers.{any, anyBoolean, anyInt, anyLong} diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index 3273824ffa450..8c63350e396e7 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -58,6 +58,7 @@ import org.apache.kafka.image.{AclsImage, ClientQuotasImage, ClusterImageTest, C import org.apache.kafka.metadata.LeaderConstants.NO_LEADER import org.apache.kafka.metadata.LeaderRecoveryState import org.apache.kafka.server.common.MetadataVersion.IBP_2_6_IV0 +import org.apache.kafka.server.log.internals.{AppendOrigin, LogDirFailureChannel} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} import org.junit.jupiter.params.ParameterizedTest @@ -178,7 +179,7 @@ class ReplicaManagerTest { timeout = 0, requiredAcks = 3, internalTopicsAllowed = false, - origin = AppendOrigin.Client, + origin = AppendOrigin.CLIENT, entriesPerPartition = Map(new TopicPartition("test1", 0) -> MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("first message".getBytes))), responseCallback = callback) @@ -456,7 +457,7 @@ class ReplicaManagerTest { val abortTxnMarker = new EndTransactionMarker(ControlRecordType.ABORT, 0) val abortRecordBatch = MemoryRecords.withEndTransactionMarker(producerId, epoch, abortTxnMarker) appendRecords(replicaManager, new TopicPartition(topic, 0), - abortRecordBatch, origin = AppendOrigin.Coordinator).onFire { response => + abortRecordBatch, origin = AppendOrigin.COORDINATOR).onFire { response => assertEquals(Errors.NONE, response.error) } assertLateTransactionCount(Some(0)) @@ -550,7 +551,7 @@ class ReplicaManagerTest { val endTxnMarker = new EndTransactionMarker(ControlRecordType.COMMIT, 0) val commitRecordBatch = MemoryRecords.withEndTransactionMarker(producerId, epoch, endTxnMarker) appendRecords(replicaManager, new TopicPartition(topic, 0), commitRecordBatch, - origin = AppendOrigin.Coordinator) + origin = AppendOrigin.COORDINATOR) .onFire { response => assertEquals(Errors.NONE, response.error) } // the LSO has advanced, but the appended commit marker has not been replicated, so @@ -635,7 +636,7 @@ class ReplicaManagerTest { val endTxnMarker = new EndTransactionMarker(ControlRecordType.ABORT, 0) val abortRecordBatch = MemoryRecords.withEndTransactionMarker(producerId, epoch, endTxnMarker) appendRecords(replicaManager, new TopicPartition(topic, 0), abortRecordBatch, - origin = AppendOrigin.Coordinator) + origin = AppendOrigin.COORDINATOR) .onFire { response => assertEquals(Errors.NONE, response.error) } // fetch as follower to advance the high watermark @@ -1323,6 +1324,17 @@ class ReplicaManagerTest { replicaManager.becomeLeaderOrFollower(2, leaderAndIsrRequest, (_, _) => ()) + appendRecords(replicaManager, tp0, TestUtils.singletonRecords(s"message".getBytes)).onFire { response => + assertEquals(Errors.NONE, response.error) + } + // Fetch as follower to initialise the log end offset of the replica + fetchPartitionAsFollower( + replicaManager, + new TopicIdPartition(topicId, new TopicPartition(topic, 0)), + new PartitionData(Uuid.ZERO_UUID, 0, 0, 100000, Optional.empty()), + replicaId = 1 + ) + val metadata = new DefaultClientMetadata("rack-b", "client-id", InetAddress.getByName("localhost"), KafkaPrincipal.ANONYMOUS, "default") @@ -1337,7 +1349,7 @@ class ReplicaManagerTest { assertTrue(consumerResult.hasFired) // PartitionView passed to ReplicaSelector should not contain the follower as it's not in the ISR - val expectedReplicaViews = Set(new DefaultReplicaView(leaderNode, 0, 0)) + val expectedReplicaViews = Set(new DefaultReplicaView(leaderNode, 1, 0)) val partitionView = replicaManager.replicaSelectorOpt.get .asInstanceOf[MockReplicaSelector].getPartitionViewArgument @@ -1499,7 +1511,7 @@ class ReplicaManagerTest { val simpleRecords = Seq(new SimpleRecord("a".getBytes), new SimpleRecord("b".getBytes)) val appendResult = appendRecords(replicaManager, tp0, - MemoryRecords.withRecords(CompressionType.NONE, simpleRecords.toSeq: _*), AppendOrigin.Client) + MemoryRecords.withRecords(CompressionType.NONE, simpleRecords.toSeq: _*), AppendOrigin.CLIENT) // Increment the hw in the leader by fetching from the last offset val fetchOffset = simpleRecords.size @@ -1918,7 +1930,7 @@ class ReplicaManagerTest { timeout = 10, requiredAcks = -1, internalTopicsAllowed = false, - origin = AppendOrigin.Client, + origin = AppendOrigin.CLIENT, entriesPerPartition = Map(topicPartition -> records), responseCallback = callback ) @@ -2138,7 +2150,7 @@ class ReplicaManagerTest { private def appendRecords(replicaManager: ReplicaManager, partition: TopicPartition, records: MemoryRecords, - origin: AppendOrigin = AppendOrigin.Client, + origin: AppendOrigin = AppendOrigin.CLIENT, requiredAcks: Short = -1): CallbackResult[PartitionResponse] = { val result = new CallbackResult[PartitionResponse]() def appendCallback(responses: Map[TopicPartition, PartitionResponse]): Unit = { @@ -2775,7 +2787,7 @@ class ReplicaManagerTest { val batch = TestUtils.records(records = List( new SimpleRecord(10, "k1".getBytes, "v1".getBytes), new SimpleRecord(11, "k2".getBytes, "v2".getBytes))) - partition.appendRecordsToLeader(batch, AppendOrigin.Client, requiredAcks = 0, RequestLocal.withThreadConfinedCaching) + partition.appendRecordsToLeader(batch, AppendOrigin.CLIENT, requiredAcks = 0, RequestLocal.withThreadConfinedCaching) partition.log.get.updateHighWatermark(2L) partition.log.get.maybeIncrementLogStartOffset(1L, LeaderOffsetIncremented) replicaManager.logManager.checkpointLogRecoveryOffsets() diff --git a/core/src/test/scala/unit/kafka/server/checkpoints/OffsetCheckpointFileWithFailureHandlerTest.scala b/core/src/test/scala/unit/kafka/server/checkpoints/OffsetCheckpointFileWithFailureHandlerTest.scala index 4889c541125e7..9439f388d4325 100644 --- a/core/src/test/scala/unit/kafka/server/checkpoints/OffsetCheckpointFileWithFailureHandlerTest.scala +++ b/core/src/test/scala/unit/kafka/server/checkpoints/OffsetCheckpointFileWithFailureHandlerTest.scala @@ -16,10 +16,10 @@ */ package kafka.server.checkpoints -import kafka.server.LogDirFailureChannel import kafka.utils.{Logging, TestUtils} import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.errors.KafkaStorageException +import org.apache.kafka.server.log.internals.LogDirFailureChannel import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Test import org.mockito.Mockito diff --git a/core/src/test/scala/unit/kafka/server/epoch/OffsetsForLeaderEpochTest.scala b/core/src/test/scala/unit/kafka/server/epoch/OffsetsForLeaderEpochTest.scala index 4c6d74652cf83..8983ad4497c8f 100644 --- a/core/src/test/scala/unit/kafka/server/epoch/OffsetsForLeaderEpochTest.scala +++ b/core/src/test/scala/unit/kafka/server/epoch/OffsetsForLeaderEpochTest.scala @@ -29,6 +29,7 @@ import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.record.RecordBatch import org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.{UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET} +import org.apache.kafka.server.log.internals.LogDirFailureChannel import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} import org.mockito.Mockito.{mock, when} diff --git a/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala b/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala index 08acbf3827abf..628d1b66169da 100644 --- a/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala +++ b/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala @@ -21,10 +21,9 @@ import java.io.{ByteArrayOutputStream, File, PrintWriter} import java.nio.ByteBuffer import java.util import java.util.Properties - -import kafka.log.{AppendOrigin, Defaults, LogConfig, LogTestUtils, ProducerStateManagerConfig, UnifiedLog} +import kafka.log.{Defaults, LogConfig, LogTestUtils, ProducerStateManagerConfig, UnifiedLog} import kafka.raft.{KafkaMetadataLog, MetadataLogConfig} -import kafka.server.{BrokerTopicStats, FetchLogEnd, KafkaRaftServer, LogDirFailureChannel} +import kafka.server.{BrokerTopicStats, FetchLogEnd, KafkaRaftServer} import kafka.tools.DumpLogSegments.TimeIndexDumpErrors import kafka.utils.{MockTime, TestUtils} import org.apache.kafka.common.Uuid @@ -36,6 +35,7 @@ import org.apache.kafka.common.utils.Utils import org.apache.kafka.metadata.MetadataRecordSerde import org.apache.kafka.raft.{KafkaRaftClient, OffsetAndEpoch} import org.apache.kafka.server.common.ApiMessageAndVersion +import org.apache.kafka.server.log.internals.{AppendOrigin, LogDirFailureChannel} import org.apache.kafka.snapshot.RecordsSnapshotWriter import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} @@ -136,7 +136,7 @@ class DumpLogSegmentsTest { log.appendAsLeader(MemoryRecords.withEndTransactionMarker(98323L, 99.toShort, new EndTransactionMarker(ControlRecordType.COMMIT, 100) - ), origin = AppendOrigin.Coordinator, leaderEpoch = 7) + ), origin = AppendOrigin.COORDINATOR, leaderEpoch = 7) assertDumpLogRecordMetadata() } diff --git a/core/src/test/scala/unit/kafka/utils/CoreUtilsTest.scala b/core/src/test/scala/unit/kafka/utils/CoreUtilsTest.scala index 7a86df81fb066..c43f99c39cabe 100755 --- a/core/src/test/scala/unit/kafka/utils/CoreUtilsTest.scala +++ b/core/src/test/scala/unit/kafka/utils/CoreUtilsTest.scala @@ -139,16 +139,6 @@ class CoreUtilsTest extends Logging { assertEquals("foo.txt", CoreUtils.replaceSuffix("foo", "", ".txt")) } - @Test - def testReadInt(): Unit = { - val values = Array(0, 1, -1, Byte.MaxValue, Short.MaxValue, 2 * Short.MaxValue, Int.MaxValue/2, Int.MinValue/2, Int.MaxValue, Int.MinValue, Int.MaxValue) - val buffer = ByteBuffer.allocate(4 * values.size) - for(i <- 0 until values.length) { - buffer.putInt(i*4, values(i)) - assertEquals(values(i), CoreUtils.readInt(buffer.array, i*4), "Written value should match read value.") - } - } - @Test def testCsvList(): Unit = { val emptyString:String = "" diff --git a/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala b/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala index b1a1a66fa302d..1f104a39ff10a 100644 --- a/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala +++ b/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala @@ -20,8 +20,9 @@ import java.util.Properties import java.util.concurrent.atomic._ import java.util.concurrent.{CountDownLatch, Executors, TimeUnit} import kafka.log.{LocalLog, LogConfig, LogLoader, LogSegments, ProducerStateManager, ProducerStateManagerConfig, UnifiedLog} -import kafka.server.{BrokerTopicStats, LogDirFailureChannel} +import kafka.server.BrokerTopicStats import kafka.utils.TestUtils.retry +import org.apache.kafka.server.log.internals.LogDirFailureChannel import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, Timeout} diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index 7110237ce4cdb..5d1abea0e66ef 100755 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -70,6 +70,7 @@ import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.controller.QuorumController import org.apache.kafka.server.authorizer.{AuthorizableRequestContext, Authorizer => JAuthorizer} import org.apache.kafka.server.common.MetadataVersion +import org.apache.kafka.server.log.internals.LogDirFailureChannel import org.apache.kafka.server.metrics.KafkaYammerMetrics import org.apache.kafka.test.{TestSslUtils, TestUtils => JTestUtils} import org.apache.zookeeper.KeeperException.SessionExpiredException diff --git a/gradle/spotbugs-exclude.xml b/gradle/spotbugs-exclude.xml index 93702b7b9dbdd..4c3a5fad94c73 100644 --- a/gradle/spotbugs-exclude.xml +++ b/gradle/spotbugs-exclude.xml @@ -495,6 +495,13 @@ For a detailed description of spotbugs bug categories, see https://spotbugs.read + + + + + + + diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/fetcher/ReplicaFetcherThreadBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/fetcher/ReplicaFetcherThreadBenchmark.java index 30b93b4b1c01f..107b0e16d7f60 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/fetcher/ReplicaFetcherThreadBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/fetcher/ReplicaFetcherThreadBenchmark.java @@ -32,7 +32,6 @@ import kafka.server.FailedPartitions; import kafka.server.InitialFetchState; import kafka.server.KafkaConfig; -import kafka.server.LogDirFailureChannel; import kafka.server.MetadataCache; import kafka.server.OffsetAndEpoch; import kafka.server.OffsetTruncationState; @@ -73,6 +72,7 @@ import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.server.common.MetadataVersion; +import org.apache.kafka.server.log.internals.LogDirFailureChannel; import org.mockito.Mockito; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/partition/PartitionMakeFollowerBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/partition/PartitionMakeFollowerBenchmark.java index 634927c9a21d5..7c5b342d9cb80 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/partition/PartitionMakeFollowerBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/partition/PartitionMakeFollowerBenchmark.java @@ -26,7 +26,6 @@ import kafka.log.LogManager; import kafka.server.AlterPartitionManager; import kafka.server.BrokerTopicStats; -import kafka.server.LogDirFailureChannel; import kafka.server.MetadataCache; import kafka.server.builders.LogManagerBuilder; import kafka.server.checkpoints.OffsetCheckpoints; @@ -41,6 +40,7 @@ import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.server.common.MetadataVersion; +import org.apache.kafka.server.log.internals.LogDirFailureChannel; import org.mockito.Mockito; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/partition/UpdateFollowerFetchStateBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/partition/UpdateFollowerFetchStateBenchmark.java index 36f596a498075..75da931a65876 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/partition/UpdateFollowerFetchStateBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/partition/UpdateFollowerFetchStateBenchmark.java @@ -27,7 +27,6 @@ import kafka.log.LogManager; import kafka.server.AlterPartitionManager; import kafka.server.BrokerTopicStats; -import kafka.server.LogDirFailureChannel; import kafka.server.LogOffsetMetadata; import kafka.server.MetadataCache; import kafka.server.builders.LogManagerBuilder; @@ -39,6 +38,7 @@ import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState; import org.apache.kafka.common.utils.Time; import org.apache.kafka.server.common.MetadataVersion; +import org.apache.kafka.server.log.internals.LogDirFailureChannel; import org.mockito.Mockito; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/BaseRecordBatchBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/BaseRecordBatchBenchmark.java index e9910da57c00a..0b241f61e32b0 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/BaseRecordBatchBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/BaseRecordBatchBenchmark.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.jmh.record; +import kafka.log.UnifiedLog; import kafka.server.BrokerTopicStats; import kafka.server.RequestLocal; import org.apache.kafka.common.header.Header; @@ -26,6 +27,7 @@ import org.apache.kafka.common.record.Record; import org.apache.kafka.common.record.RecordBatch; import org.apache.kafka.common.record.TimestampType; +import org.apache.kafka.server.log.internals.LogValidator; import org.openjdk.jmh.annotations.Param; import org.openjdk.jmh.annotations.Scope; import org.openjdk.jmh.annotations.Setup; @@ -75,7 +77,8 @@ public enum Bytes { // Used by measureVariableBatchSize ByteBuffer[] batchBuffers; RequestLocal requestLocal; - final BrokerTopicStats brokerTopicStats = new BrokerTopicStats(); + LogValidator.MetricsRecorder validatorMetricsRecorder = UnifiedLog.newValidatorMetricsRecorder( + new BrokerTopicStats().allTopicsStats()); @Setup public void init() { diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/CompressedRecordBatchValidationBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/CompressedRecordBatchValidationBenchmark.java index dd21c19b7a521..7abbfd886f087 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/CompressedRecordBatchValidationBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/CompressedRecordBatchValidationBenchmark.java @@ -16,15 +16,15 @@ */ package org.apache.kafka.jmh.record; -import kafka.common.LongRef; -import kafka.log.AppendOrigin; -import kafka.log.LogValidator; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.record.CompressionType; import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.TimestampType; +import org.apache.kafka.common.utils.PrimitiveRef; import org.apache.kafka.common.utils.Time; import org.apache.kafka.server.common.MetadataVersion; +import org.apache.kafka.server.log.internals.AppendOrigin; +import org.apache.kafka.server.log.internals.LogValidator; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.Fork; import org.openjdk.jmh.annotations.Measurement; @@ -51,14 +51,11 @@ CompressionType compressionType() { @Benchmark public void measureValidateMessagesAndAssignOffsetsCompressed(Blackhole bh) { MemoryRecords records = MemoryRecords.readableRecords(singleBatchBuffer.duplicate()); - LogValidator.validateMessagesAndAssignOffsetsCompressed(records, new TopicPartition("a", 0), - new LongRef(startingOffset), Time.SYSTEM, System.currentTimeMillis(), - compressionType, - compressionType, - false, messageVersion, TimestampType.CREATE_TIME, Long.MAX_VALUE, 0, - new AppendOrigin.Client$(), - MetadataVersion.latest(), - brokerTopicStats, - requestLocal); + new LogValidator(records, new TopicPartition("a", 0), + Time.SYSTEM, compressionType, compressionType, false, messageVersion, + TimestampType.CREATE_TIME, Long.MAX_VALUE, 0, AppendOrigin.CLIENT, + MetadataVersion.latest() + ).validateMessagesAndAssignOffsetsCompressed(PrimitiveRef.ofLong(startingOffset), + validatorMetricsRecorder, requestLocal.bufferSupplier()); } } diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/UncompressedRecordBatchValidationBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/UncompressedRecordBatchValidationBenchmark.java index 001837ef6eea5..f100a63ec4d99 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/UncompressedRecordBatchValidationBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/UncompressedRecordBatchValidationBenchmark.java @@ -16,13 +16,15 @@ */ package org.apache.kafka.jmh.record; -import kafka.common.LongRef; -import kafka.log.AppendOrigin; -import kafka.log.LogValidator; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.record.CompressionType; import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.TimestampType; +import org.apache.kafka.common.utils.PrimitiveRef; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.server.common.MetadataVersion; +import org.apache.kafka.server.log.internals.AppendOrigin; +import org.apache.kafka.server.log.internals.LogValidator; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.Fork; import org.openjdk.jmh.annotations.Measurement; @@ -45,9 +47,10 @@ CompressionType compressionType() { @Benchmark public void measureAssignOffsetsNonCompressed(Blackhole bh) { MemoryRecords records = MemoryRecords.readableRecords(singleBatchBuffer.duplicate()); - LogValidator.assignOffsetsNonCompressed(records, new TopicPartition("a", 0), - new LongRef(startingOffset), System.currentTimeMillis(), false, - TimestampType.CREATE_TIME, Long.MAX_VALUE, 0, - new AppendOrigin.Client$(), messageVersion, brokerTopicStats); + new LogValidator(records, new TopicPartition("a", 0), + Time.SYSTEM, CompressionType.NONE, CompressionType.NONE, false, + messageVersion, TimestampType.CREATE_TIME, Long.MAX_VALUE, 0, AppendOrigin.CLIENT, + MetadataVersion.latest() + ).assignOffsetsNonCompressed(PrimitiveRef.ofLong(startingOffset), validatorMetricsRecorder); } } diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/server/CheckpointBench.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/server/CheckpointBench.java index d83f748bcad91..bb07e63868fed 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/server/CheckpointBench.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/server/CheckpointBench.java @@ -24,7 +24,7 @@ import kafka.server.BrokerFeatures; import kafka.server.BrokerTopicStats; import kafka.server.KafkaConfig; -import kafka.server.LogDirFailureChannel; +import org.apache.kafka.server.log.internals.LogDirFailureChannel; import kafka.server.MetadataCache; import kafka.server.QuotaFactory; import kafka.server.ReplicaManager; diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/server/PartitionCreationBench.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/server/PartitionCreationBench.java index 3a8343772cbbe..68880830cae7f 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/server/PartitionCreationBench.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/server/PartitionCreationBench.java @@ -25,7 +25,6 @@ import kafka.server.BrokerFeatures; import kafka.server.BrokerTopicStats; import kafka.server.KafkaConfig; -import kafka.server.LogDirFailureChannel; import kafka.server.MetadataCache; import kafka.server.QuotaFactory; import kafka.server.ReplicaManager; @@ -45,6 +44,7 @@ import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.server.common.MetadataVersion; +import org.apache.kafka.server.log.internals.LogDirFailureChannel; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; import org.openjdk.jmh.annotations.Fork; diff --git a/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java index d636b4b8fc6ca..7a58433789b61 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java @@ -420,6 +420,7 @@ public void replay(RegisterBrokerRecord record, long offset) { feature.minSupportedVersion(), feature.maxSupportedVersion())); } + // Update broker registrations. BrokerRegistration prevRegistration = brokerRegistrations.put(brokerId, new BrokerRegistration(brokerId, record.brokerEpoch(), diff --git a/metadata/src/main/java/org/apache/kafka/metadata/BrokerRegistration.java b/metadata/src/main/java/org/apache/kafka/metadata/BrokerRegistration.java index b365ae81266d7..3f87d2830aba4 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/BrokerRegistration.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/BrokerRegistration.java @@ -268,7 +268,7 @@ public boolean equals(Object o) { other.rack.equals(rack) && other.fenced == fenced && other.inControlledShutdown == inControlledShutdown && - other.migratingZkBrokerEpoch == migratingZkBrokerEpoch; + other.migratingZkBrokerEpoch.equals(migratingZkBrokerEpoch); } @Override diff --git a/metadata/src/main/resources/common/metadata/RegisterBrokerRecord.json b/metadata/src/main/resources/common/metadata/RegisterBrokerRecord.json index 18dd6e06dd879..f8de544127c05 100644 --- a/metadata/src/main/resources/common/metadata/RegisterBrokerRecord.json +++ b/metadata/src/main/resources/common/metadata/RegisterBrokerRecord.json @@ -23,7 +23,7 @@ { "name": "BrokerId", "type": "int32", "versions": "0+", "entityType": "brokerId", "about": "The broker id." }, { "name": "MigratingZkBrokerEpoch", "type": "int64", "versions": "2+", "default": "-1", - "about": "True if the registering broker is a ZK broker." }, + "about": "The ZK broker epoch if this record is for a ZK broker. Otherwise, -1" }, { "name": "IncarnationId", "type": "uuid", "versions": "0+", "about": "The incarnation ID of the broker process" }, { "name": "BrokerEpoch", "type": "int64", "versions": "0+", diff --git a/metadata/src/test/java/org/apache/kafka/metadata/BrokerRegistrationTest.java b/metadata/src/test/java/org/apache/kafka/metadata/BrokerRegistrationTest.java index afdb15e7272d3..50516fbfccdc3 100644 --- a/metadata/src/test/java/org/apache/kafka/metadata/BrokerRegistrationTest.java +++ b/metadata/src/test/java/org/apache/kafka/metadata/BrokerRegistrationTest.java @@ -54,13 +54,19 @@ public class BrokerRegistrationTest { Stream.of(new SimpleEntry<>("foo", VersionRange.of((short) 2, (short) 3)), new SimpleEntry<>("bar", VersionRange.of((short) 1, (short) 4))).collect( Collectors.toMap(SimpleEntry::getKey, SimpleEntry::getValue)), - Optional.of("myrack"), false, true)); + Optional.of("myrack"), false, true), + new BrokerRegistration(3, 0, Uuid.fromString("1t8VyWx2TCSTpUWuqj-FOw"), + Arrays.asList(new Endpoint("INTERNAL", SecurityProtocol.PLAINTEXT, "localhost", 9093)), + Stream.of(new SimpleEntry<>("metadata.version", VersionRange.of((short) 7, (short) 7))) + .collect(Collectors.toMap(SimpleEntry::getKey, SimpleEntry::getValue)), + Optional.empty(), false, true, Optional.of(10L))); @Test public void testValues() { assertEquals(0, REGISTRATIONS.get(0).id()); assertEquals(1, REGISTRATIONS.get(1).id()); assertEquals(2, REGISTRATIONS.get(2).id()); + assertEquals(3, REGISTRATIONS.get(3).id()); } @Test @@ -69,9 +75,13 @@ public void testEquals() { assertNotEquals(REGISTRATIONS.get(1), REGISTRATIONS.get(0)); assertNotEquals(REGISTRATIONS.get(0), REGISTRATIONS.get(2)); assertNotEquals(REGISTRATIONS.get(2), REGISTRATIONS.get(0)); + assertNotEquals(REGISTRATIONS.get(3), REGISTRATIONS.get(0)); + assertNotEquals(REGISTRATIONS.get(3), REGISTRATIONS.get(1)); + assertNotEquals(REGISTRATIONS.get(3), REGISTRATIONS.get(2)); assertEquals(REGISTRATIONS.get(0), REGISTRATIONS.get(0)); assertEquals(REGISTRATIONS.get(1), REGISTRATIONS.get(1)); assertEquals(REGISTRATIONS.get(2), REGISTRATIONS.get(2)); + assertEquals(REGISTRATIONS.get(3), REGISTRATIONS.get(3)); } @Test @@ -80,14 +90,20 @@ public void testToString() { "incarnationId=3MfdxWlNSn2UDYsmDP1pYg, listeners=[Endpoint(" + "listenerName='INTERNAL', securityProtocol=PLAINTEXT, " + "host='localhost', port=9091)], supportedFeatures={foo: 1-2}, " + - "rack=Optional.empty, fenced=true, inControlledShutdown=false, isMigratingZkBroker=false)", + "rack=Optional.empty, fenced=true, inControlledShutdown=false, migratingZkBrokerEpoch=-1)", REGISTRATIONS.get(1).toString()); assertEquals("BrokerRegistration(id=2, epoch=0, " + "incarnationId=eY7oaG1RREie5Kk9uy1l6g, listeners=[Endpoint(" + "listenerName='INTERNAL', securityProtocol=PLAINTEXT, " + "host='localhost', port=9092)], supportedFeatures={bar: 1-4, foo: 2-3}, " + - "rack=Optional[myrack], fenced=false, inControlledShutdown=true, isMigratingZkBroker=false)", + "rack=Optional[myrack], fenced=false, inControlledShutdown=true, migratingZkBrokerEpoch=-1)", REGISTRATIONS.get(2).toString()); + assertEquals("BrokerRegistration(id=3, epoch=0, " + + "incarnationId=1t8VyWx2TCSTpUWuqj-FOw, listeners=[Endpoint(" + + "listenerName='INTERNAL', securityProtocol=PLAINTEXT, " + + "host='localhost', port=9093)], supportedFeatures={metadata.version: 7}, " + + "rack=Optional.empty, fenced=false, inControlledShutdown=true, migratingZkBrokerEpoch=10)", + REGISTRATIONS.get(3).toString()); } @Test @@ -95,6 +111,7 @@ public void testFromRecordAndToRecord() { testRoundTrip(REGISTRATIONS.get(0)); testRoundTrip(REGISTRATIONS.get(1)); testRoundTrip(REGISTRATIONS.get(2)); + testRoundTrip(REGISTRATIONS.get(3)); } private void testRoundTrip(BrokerRegistration registration) { @@ -117,5 +134,7 @@ public void testToNode() { REGISTRATIONS.get(1).node("INTERNAL")); assertEquals(Optional.of(new Node(2, "localhost", 9092, "myrack")), REGISTRATIONS.get(2).node("INTERNAL")); + assertEquals(Optional.of(new Node(3, "localhost", 9093, null)), + REGISTRATIONS.get(3).node("INTERNAL")); } } diff --git a/storage/src/main/java/org/apache/kafka/server/log/internals/AbstractIndex.java b/storage/src/main/java/org/apache/kafka/server/log/internals/AbstractIndex.java index 36e7e50be079a..d5a510c94ca68 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/internals/AbstractIndex.java +++ b/storage/src/main/java/org/apache/kafka/server/log/internals/AbstractIndex.java @@ -40,14 +40,8 @@ */ public abstract class AbstractIndex implements Closeable { - private static class BinarySearchResult { - public final int largestLowerBound; - public final int smallestUpperBound; - - private BinarySearchResult(int largestLowerBound, int smallestUpperBound) { - this.largestLowerBound = largestLowerBound; - this.smallestUpperBound = smallestUpperBound; - } + private enum SearchResultType { + LARGEST_LOWER_BOUND, SMALLEST_UPPER_BOUND } private static final Logger log = LoggerFactory.getLogger(AbstractIndex.class); @@ -447,14 +441,14 @@ protected final T maybeLock(Lock lock, StorageAction 0) - return new BinarySearchResult(-1, 0); + if (compareIndexEntry(parseEntry(idx, 0), target, searchEntity) > 0) { + switch (searchResultType) { + case LARGEST_LOWER_BOUND: + return -1; + case SMALLEST_UPPER_BOUND: + return 0; + } + } - return binarySearch(idx, target, searchEntity, 0, firstHotEntry); + return binarySearch(idx, target, searchEntity, searchResultType, 0, firstHotEntry); } - private BinarySearchResult binarySearch(ByteBuffer idx, long target, IndexSearchType searchEntity, int begin, int end) { + private int binarySearch(ByteBuffer idx, long target, IndexSearchType searchEntity, + SearchResultType searchResultType, int begin, int end) { // binary search for the entry int lo = begin; int hi = end; @@ -517,13 +520,19 @@ private BinarySearchResult binarySearch(ByteBuffer idx, long target, IndexSearch else if (compareResult < 0) lo = mid; else - return new BinarySearchResult(mid, mid); + return mid; + } + switch (searchResultType) { + case LARGEST_LOWER_BOUND: + return lo; + case SMALLEST_UPPER_BOUND: + if (lo == entries - 1) + return -1; + else + return lo + 1; + default: + throw new IllegalStateException("Unexpected searchResultType " + searchResultType); } - if (lo == entries - 1) - hi = -1; - else - hi = lo + 1; - return new BinarySearchResult(lo, hi); } private int compareIndexEntry(IndexEntry indexEntry, long target, IndexSearchType searchEntity) { diff --git a/storage/src/main/java/org/apache/kafka/server/log/internals/AppendOrigin.java b/storage/src/main/java/org/apache/kafka/server/log/internals/AppendOrigin.java new file mode 100644 index 0000000000000..aba8faa89fd2d --- /dev/null +++ b/storage/src/main/java/org/apache/kafka/server/log/internals/AppendOrigin.java @@ -0,0 +1,45 @@ +/* + * 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.server.log.internals; + +/** + * The source of an append to the log. This is used when determining required validations. + */ +public enum AppendOrigin { + /** + * The log append came through replication from the leader. This typically implies minimal validation. + * Particularly, we do not decompress record batches in order to validate records individually. + */ + REPLICATION, + + /** + * The log append came from either the group coordinator or the transaction coordinator. We validate + * producer epochs for normal log entries (specifically offset commits from the group coordinator) and + * we validate coordinate end transaction markers from the transaction coordinator. + */ + COORDINATOR, + + /** + * The log append came from the client, which implies full validation. + */ + CLIENT, + + /** + * The log append come from the raft leader, which implies the offsets has been assigned + */ + RAFT_LEADER; +} diff --git a/storage/src/main/java/org/apache/kafka/server/log/internals/LazyIndex.java b/storage/src/main/java/org/apache/kafka/server/log/internals/LazyIndex.java new file mode 100644 index 0000000000000..220ab6464dcc4 --- /dev/null +++ b/storage/src/main/java/org/apache/kafka/server/log/internals/LazyIndex.java @@ -0,0 +1,245 @@ +/* + * 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.server.log.internals; + +import java.io.Closeable; +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.NoSuchFileException; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; +import org.apache.kafka.common.utils.Utils; + +/** + * A wrapper over an `AbstractIndex` instance that provides a mechanism to defer loading + * (i.e. memory mapping) the underlying index until it is accessed for the first time via the + * `get` method. + * + * In addition, this class exposes a number of methods (e.g. updateParentDir, renameTo, close, + * etc.) that provide the desired behavior without causing the index to be loaded. If the index + * had previously been loaded, the methods in this class simply delegate to the relevant method in + * the index. + * + * This is an important optimization with regards to broker start-up and shutdown time if it has a + * large number of segments. + * + * Methods of this class are thread safe. Make sure to check `AbstractIndex` subclasses + * documentation to establish their thread safety. + */ +public class LazyIndex { + + private enum IndexType { + OFFSET, TIME; + } + + private interface IndexWrapper extends Closeable { + File file(); + void updateParentDir(File file); + void renameTo(File file) throws IOException; + boolean deleteIfExists() throws IOException; + void close() throws IOException; + void closeHandler(); + } + + private static class IndexFile implements IndexWrapper { + + private volatile File file; + + IndexFile(File file) { + this.file = file; + } + + @Override + public File file() { + return file; + } + + @Override + public void updateParentDir(File parentDir) { + file = new File(parentDir, file.getName()); + } + + @Override + public void renameTo(File f) throws IOException { + try { + Utils.atomicMoveWithFallback(file.toPath(), f.toPath(), false); + } catch (NoSuchFileException e) { + if (file.exists()) + throw e; + } finally { + file = f; + } + } + + @Override + public boolean deleteIfExists() throws IOException { + return Files.deleteIfExists(file.toPath()); + } + + @Override + public void close() { } + + @Override + public void closeHandler() { } + + } + + private static class IndexValue implements IndexWrapper { + + private final T index; + + IndexValue(T index) { + this.index = index; + } + + @Override + public File file() { + return index.file(); + } + + @Override + public void updateParentDir(File parentDir) { + index.updateParentDir(parentDir); + } + + @Override + public void renameTo(File f) throws IOException { + index.renameTo(f); + } + + @Override + public boolean deleteIfExists() throws IOException { + return index.deleteIfExists(); + } + + @Override + public void close() throws IOException { + index.close(); + } + + @Override + public void closeHandler() { + index.closeHandler(); + } + } + + private final Lock lock = new ReentrantLock(); + private final long baseOffset; + private final int maxIndexSize; + private final boolean writable; + private final IndexType indexType; + + private volatile IndexWrapper indexWrapper; + + private LazyIndex(IndexWrapper indexWrapper, long baseOffset, int maxIndexSize, boolean writable, + IndexType indexType) { + this.indexWrapper = indexWrapper; + this.baseOffset = baseOffset; + this.maxIndexSize = maxIndexSize; + this.writable = writable; + this.indexType = indexType; + } + + public static LazyIndex forOffset(File file, long baseOffset, int maxIndexSize, boolean writable) { + return new LazyIndex<>(new IndexFile(file), baseOffset, maxIndexSize, writable, IndexType.OFFSET); + } + + public static LazyIndex forTime(File file, long baseOffset, int maxIndexSize, boolean writable) { + return new LazyIndex<>(new IndexFile(file), baseOffset, maxIndexSize, writable, IndexType.TIME); + } + + public File file() { + return indexWrapper.file(); + } + + @SuppressWarnings("unchecked") + public T get() throws IOException { + if (indexWrapper instanceof IndexValue) + return ((IndexValue) indexWrapper).index; + else if (indexWrapper instanceof IndexFile) { + lock.lock(); + try { + IndexFile indexFile = (IndexFile) indexWrapper; + IndexValue indexValue = new IndexValue<>(loadIndex(indexFile.file)); + indexWrapper = indexValue; + return indexValue.index; + } finally { + lock.unlock(); + } + } else + throw new IllegalStateException("Unexpected type for indexWrapper " + indexWrapper.getClass()); + } + + public void updateParentDir(File parentDir) { + lock.lock(); + try { + indexWrapper.updateParentDir(parentDir); + } finally { + lock.unlock(); + } + } + + public void renameTo(File f) throws IOException { + lock.lock(); + try { + indexWrapper.renameTo(f); + } finally { + lock.unlock(); + } + } + + public boolean deleteIfExists() throws IOException { + lock.lock(); + try { + return indexWrapper.deleteIfExists(); + } finally { + lock.unlock(); + } + } + + public void close() throws IOException { + lock.lock(); + try { + indexWrapper.close(); + } finally { + lock.unlock(); + } + } + + public void closeHandler() { + lock.lock(); + try { + indexWrapper.closeHandler(); + } finally { + lock.unlock(); + } + } + + @SuppressWarnings("unchecked") + private T loadIndex(File file) throws IOException { + switch (indexType) { + case OFFSET: + return (T) new OffsetIndex(file, baseOffset, maxIndexSize, writable); + case TIME: + return (T) new TimeIndex(file, baseOffset, maxIndexSize, writable); + default: + throw new IllegalStateException("Unexpected indexType " + indexType); + } + } + +} diff --git a/storage/src/main/java/org/apache/kafka/server/log/internals/LogDirFailureChannel.java b/storage/src/main/java/org/apache/kafka/server/log/internals/LogDirFailureChannel.java new file mode 100644 index 0000000000000..2d0e8d1e6eb39 --- /dev/null +++ b/storage/src/main/java/org/apache/kafka/server/log/internals/LogDirFailureChannel.java @@ -0,0 +1,77 @@ +/* + * 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.server.log.internals; + +import java.io.IOException; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/* + * LogDirFailureChannel allows an external thread to block waiting for new offline log dirs. + * + * There should be a single instance of LogDirFailureChannel accessible by any class that does disk-IO operation. + * If IOException is encountered while accessing a log directory, the corresponding class can add the log directory name + * to the LogDirFailureChannel using maybeAddOfflineLogDir(). Each log directory will be added only once. After a log + * directory is added for the first time, a thread which is blocked waiting for new offline log directories + * can take the name of the new offline log directory out of the LogDirFailureChannel and handle the log failure properly. + * An offline log directory will stay offline until the broker is restarted. + */ +public class LogDirFailureChannel { + private static final Logger log = LoggerFactory.getLogger(LogDirFailureChannel.class); + private final ConcurrentMap offlineLogDirs; + private final BlockingQueue offlineLogDirQueue; + + public LogDirFailureChannel(int logDirNum) { + this.offlineLogDirs = new ConcurrentHashMap<>(); + this.offlineLogDirQueue = new ArrayBlockingQueue<>(logDirNum); + } + + public boolean hasOfflineLogDir(String logDir) { + return offlineLogDirs.containsKey(logDir); + } + + /** + * If the given logDir is not already offline, add it to the + * set of offline log dirs and enqueue it to the logDirFailureEvent queue. + * + * @param logDir The offline logDir. + * @param msg Error message. + * @param e Exception instance. + */ + public void maybeAddOfflineLogDir(String logDir, String msg, IOException e) { + log.error(msg, e); + if (offlineLogDirs.putIfAbsent(logDir, logDir) == null) { + offlineLogDirQueue.add(logDir); + } + } + + /** + * Get the next offline log dir from logDirFailureEvent queue. + * The method will wait if necessary until a new offline log directory becomes available + * + * @return The next offline log dir. + * @throws InterruptedException + */ + public String takeNextOfflineLogDir() throws InterruptedException { + return offlineLogDirQueue.take(); + } +} diff --git a/storage/src/main/java/org/apache/kafka/server/log/internals/LogValidator.java b/storage/src/main/java/org/apache/kafka/server/log/internals/LogValidator.java new file mode 100644 index 0000000000000..ef693b59a2630 --- /dev/null +++ b/storage/src/main/java/org/apache/kafka/server/log/internals/LogValidator.java @@ -0,0 +1,631 @@ +/* + * 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.server.log.internals; + +import static org.apache.kafka.server.common.MetadataVersion.IBP_2_1_IV0; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; +import java.util.stream.Collectors; +import org.apache.kafka.common.InvalidRecordException; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.CorruptRecordException; +import org.apache.kafka.common.errors.InvalidTimestampException; +import org.apache.kafka.common.errors.UnsupportedCompressionTypeException; +import org.apache.kafka.common.errors.UnsupportedForMessageFormatException; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.record.AbstractRecords; +import org.apache.kafka.common.record.CompressionType; +import org.apache.kafka.common.record.MemoryRecords; +import org.apache.kafka.common.record.MemoryRecordsBuilder; +import org.apache.kafka.common.record.MemoryRecordsBuilder.RecordsInfo; +import org.apache.kafka.common.record.MutableRecordBatch; +import org.apache.kafka.common.record.Record; +import org.apache.kafka.common.record.RecordBatch; +import org.apache.kafka.common.record.RecordConversionStats; +import org.apache.kafka.common.record.TimestampType; +import org.apache.kafka.common.requests.ProduceResponse.RecordError; +import org.apache.kafka.common.utils.BufferSupplier; +import org.apache.kafka.common.utils.CloseableIterator; +import org.apache.kafka.common.utils.PrimitiveRef; +import org.apache.kafka.common.utils.PrimitiveRef.LongRef; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.server.common.MetadataVersion; + +import java.util.Iterator; + +public class LogValidator { + + public interface MetricsRecorder { + void recordInvalidMagic(); + + void recordInvalidOffset(); + + void recordInvalidSequence(); + + void recordInvalidChecksums(); + + void recordNoKeyCompactedTopic(); + } + + public static class ValidationResult { + public final long logAppendTimeMs; + public final MemoryRecords validatedRecords; + public final long maxTimestampMs; + public final long shallowOffsetOfMaxTimestampMs; + public final boolean messageSizeMaybeChanged; + public final RecordConversionStats recordConversionStats; + + public ValidationResult(long logAppendTimeMs, MemoryRecords validatedRecords, long maxTimestampMs, + long shallowOffsetOfMaxTimestampMs, boolean messageSizeMaybeChanged, + RecordConversionStats recordConversionStats) { + this.logAppendTimeMs = logAppendTimeMs; + this.validatedRecords = validatedRecords; + this.maxTimestampMs = maxTimestampMs; + this.shallowOffsetOfMaxTimestampMs = shallowOffsetOfMaxTimestampMs; + this.messageSizeMaybeChanged = messageSizeMaybeChanged; + this.recordConversionStats = recordConversionStats; + } + } + + private static class ApiRecordError { + final Errors apiError; + final RecordError recordError; + + private ApiRecordError(Errors apiError, RecordError recordError) { + this.apiError = apiError; + this.recordError = recordError; + } + } + + private final MemoryRecords records; + private final TopicPartition topicPartition; + private final Time time; + private final CompressionType sourceCompression; + private final CompressionType targetCompression; + private final boolean compactedTopic; + private final byte toMagic; + private final TimestampType timestampType; + private final long timestampDiffMaxMs; + private final int partitionLeaderEpoch; + private final AppendOrigin origin; + private final MetadataVersion interBrokerProtocolVersion; + + public LogValidator(MemoryRecords records, + TopicPartition topicPartition, + Time time, + CompressionType sourceCompression, + CompressionType targetCompression, + boolean compactedTopic, + byte toMagic, + TimestampType timestampType, + long timestampDiffMaxMs, + int partitionLeaderEpoch, + AppendOrigin origin, + MetadataVersion interBrokerProtocolVersion) { + this.records = records; + this.topicPartition = topicPartition; + this.time = time; + this.sourceCompression = sourceCompression; + this.targetCompression = targetCompression; + this.compactedTopic = compactedTopic; + this.toMagic = toMagic; + this.timestampType = timestampType; + this.timestampDiffMaxMs = timestampDiffMaxMs; + this.partitionLeaderEpoch = partitionLeaderEpoch; + this.origin = origin; + this.interBrokerProtocolVersion = interBrokerProtocolVersion; + } + + /** + * Update the offsets for this message set and do further validation on messages including: + * 1. Messages for compacted topics must have keys + * 2. When magic value >= 1, inner messages of a compressed message set must have monotonically increasing offsets + * starting from 0. + * 3. When magic value >= 1, validate and maybe overwrite timestamps of messages. + * 4. Declared count of records in DefaultRecordBatch must match number of valid records contained therein. + * + * This method will convert messages as necessary to the topic's configured message format version. If no format + * conversion or value overwriting is required for messages, this method will perform in-place operations to + * avoid expensive re-compression. + * + * Returns a ValidationAndOffsetAssignResult containing the validated message set, maximum timestamp, the offset + * of the shallow message with the max timestamp and a boolean indicating whether the message sizes may have changed. + */ + public ValidationResult validateMessagesAndAssignOffsets(PrimitiveRef.LongRef offsetCounter, + MetricsRecorder metricsRecorder, + BufferSupplier bufferSupplier) { + if (sourceCompression == CompressionType.NONE && targetCompression == CompressionType.NONE) { + // check the magic value + if (!records.hasMatchingMagic(toMagic)) + return convertAndAssignOffsetsNonCompressed(offsetCounter, metricsRecorder); + else + // Do in-place validation, offset assignment and maybe set timestamp + return assignOffsetsNonCompressed(offsetCounter, metricsRecorder); + } else + return validateMessagesAndAssignOffsetsCompressed(offsetCounter, metricsRecorder, bufferSupplier); + + } + + private static MutableRecordBatch getFirstBatchAndMaybeValidateNoMoreBatches(MemoryRecords records, + CompressionType sourceCompression) { + Iterator batchIterator = records.batches().iterator(); + + if (!batchIterator.hasNext()) + throw new InvalidRecordException("Record batch has no batches at all"); + + MutableRecordBatch batch = batchIterator.next(); + + // if the format is v2 and beyond, or if the messages are compressed, we should check there's only one batch. + if (batch.magic() >= RecordBatch.MAGIC_VALUE_V2 || sourceCompression != CompressionType.NONE) { + if (batchIterator.hasNext()) + throw new InvalidRecordException("Compressed outer record has more than one batch"); + } + + return batch; + } + + private ValidationResult convertAndAssignOffsetsNonCompressed(LongRef offsetCounter, + MetricsRecorder metricsRecorder) { + long now = time.milliseconds(); + long startNanos = time.nanoseconds(); + int sizeInBytesAfterConversion = AbstractRecords.estimateSizeInBytes(toMagic, offsetCounter.value, + CompressionType.NONE, records.records()); + + RecordBatch firstBatch = getFirstBatchAndMaybeValidateNoMoreBatches(records, CompressionType.NONE); + long producerId = firstBatch.producerId(); + short producerEpoch = firstBatch.producerEpoch(); + int sequence = firstBatch.baseSequence(); + boolean isTransactional = firstBatch.isTransactional(); + + // The current implementation of BufferSupplier is naive and works best when the buffer size + // cardinality is low, so don't use it here + ByteBuffer newBuffer = ByteBuffer.allocate(sizeInBytesAfterConversion); + MemoryRecordsBuilder builder = MemoryRecords.builder(newBuffer, toMagic, CompressionType.NONE, + timestampType, offsetCounter.value, now, producerId, producerEpoch, sequence, isTransactional, + partitionLeaderEpoch); + + for (RecordBatch batch : records.batches()) { + validateBatch(topicPartition, firstBatch, batch, origin, toMagic, metricsRecorder); + + List recordErrors = new ArrayList<>(0); + int batchIndex = 0; + for (Record record : batch) { + Optional recordError = validateRecord(batch, topicPartition, + record, batchIndex, now, timestampType, timestampDiffMaxMs, compactedTopic, + metricsRecorder); + recordError.ifPresent(e -> recordErrors.add(e)); + // we fail the batch if any record fails, so we stop appending if any record fails + if (recordErrors.isEmpty()) + builder.appendWithOffset(offsetCounter.value++, record); + ++batchIndex; + } + + processRecordErrors(recordErrors); + } + + MemoryRecords convertedRecords = builder.build(); + + RecordsInfo info = builder.info(); + RecordConversionStats recordConversionStats = new RecordConversionStats( + builder.uncompressedBytesWritten(), builder.numRecords(), time.nanoseconds() - startNanos); + return new ValidationResult( + now, + convertedRecords, + info.maxTimestamp, + info.shallowOffsetOfMaxTimestamp, + true, + recordConversionStats); + } + + // Visible for benchmarking + public ValidationResult assignOffsetsNonCompressed(LongRef offsetCounter, + MetricsRecorder metricsRecorder) { + long now = time.milliseconds(); + long maxTimestamp = RecordBatch.NO_TIMESTAMP; + long offsetOfMaxTimestamp = -1L; + long initialOffset = offsetCounter.value; + + RecordBatch firstBatch = getFirstBatchAndMaybeValidateNoMoreBatches(records, CompressionType.NONE); + + for (MutableRecordBatch batch : records.batches()) { + validateBatch(topicPartition, firstBatch, batch, origin, toMagic, metricsRecorder); + + long maxBatchTimestamp = RecordBatch.NO_TIMESTAMP; + long offsetOfMaxBatchTimestamp = -1L; + + List recordErrors = new ArrayList<>(0); + // This is a hot path and we want to avoid any unnecessary allocations. + // That said, there is no benefit in using `skipKeyValueIterator` for the uncompressed + // case since we don't do key/value copies in this path (we just slice the ByteBuffer) + int batchIndex = 0; + for (Record record : batch) { + Optional recordError = validateRecord(batch, topicPartition, record, + batchIndex, now, timestampType, timestampDiffMaxMs, compactedTopic, metricsRecorder); + recordError.ifPresent(e -> recordErrors.add(e)); + + long offset = offsetCounter.value++; + if (batch.magic() > RecordBatch.MAGIC_VALUE_V0 && record.timestamp() > maxBatchTimestamp) { + maxBatchTimestamp = record.timestamp(); + offsetOfMaxBatchTimestamp = offset; + } + ++batchIndex; + } + + processRecordErrors(recordErrors); + + if (batch.magic() > RecordBatch.MAGIC_VALUE_V0 && maxBatchTimestamp > maxTimestamp) { + maxTimestamp = maxBatchTimestamp; + offsetOfMaxTimestamp = offsetOfMaxBatchTimestamp; + } + + batch.setLastOffset(offsetCounter.value - 1); + + if (batch.magic() >= RecordBatch.MAGIC_VALUE_V2) + batch.setPartitionLeaderEpoch(partitionLeaderEpoch); + + if (batch.magic() > RecordBatch.MAGIC_VALUE_V0) { + if (timestampType == TimestampType.LOG_APPEND_TIME) + batch.setMaxTimestamp(TimestampType.LOG_APPEND_TIME, now); + else + batch.setMaxTimestamp(timestampType, maxBatchTimestamp); + } + } + + if (timestampType == TimestampType.LOG_APPEND_TIME) { + maxTimestamp = now; + if (toMagic >= RecordBatch.MAGIC_VALUE_V2) + offsetOfMaxTimestamp = offsetCounter.value - 1; + else + offsetOfMaxTimestamp = initialOffset; + } + + return new ValidationResult( + now, + records, + maxTimestamp, + offsetOfMaxTimestamp, + false, + RecordConversionStats.EMPTY); + } + + /** + * We cannot do in place assignment in one of the following situations: + * 1. Source and target compression codec are different + * 2. When the target magic is not equal to batches' magic, meaning format conversion is needed. + * 3. When the target magic is equal to V0, meaning absolute offsets need to be re-assigned. + */ + // Visible for benchmarking + public ValidationResult validateMessagesAndAssignOffsetsCompressed(LongRef offsetCounter, + MetricsRecorder metricsRecorder, + BufferSupplier bufferSupplier) { + if (targetCompression == CompressionType.ZSTD && interBrokerProtocolVersion.isLessThan(IBP_2_1_IV0)) + throw new UnsupportedCompressionTypeException("Produce requests to inter.broker.protocol.version < 2.1 broker " + + "are not allowed to use ZStandard compression"); + + // No in place assignment situation 1 + boolean inPlaceAssignment = sourceCompression == targetCompression; + long now = time.milliseconds(); + + long maxTimestamp = RecordBatch.NO_TIMESTAMP; + LongRef expectedInnerOffset = PrimitiveRef.ofLong(0); + List validatedRecords = new ArrayList<>(); + + int uncompressedSizeInBytes = 0; + + // Assume there's only one batch with compressed memory records; otherwise, return InvalidRecordException + // One exception though is that with format smaller than v2, if sourceCompression is noCompression, then each batch is actually + // a single record so we'd need to special handle it by creating a single wrapper batch that includes all the records + MutableRecordBatch firstBatch = getFirstBatchAndMaybeValidateNoMoreBatches(records, sourceCompression); + + // No in place assignment situation 2 and 3: we only need to check for the first batch because: + // 1. For most cases (compressed records, v2, for example), there's only one batch anyways. + // 2. For cases that there may be multiple batches, all batches' magic should be the same. + if (firstBatch.magic() != toMagic || toMagic == RecordBatch.MAGIC_VALUE_V0) + inPlaceAssignment = false; + + // Do not compress control records unless they are written compressed + if (sourceCompression == CompressionType.NONE && firstBatch.isControlBatch()) + inPlaceAssignment = true; + + for (MutableRecordBatch batch : records.batches()) { + validateBatch(topicPartition, firstBatch, batch, origin, toMagic, metricsRecorder); + uncompressedSizeInBytes += AbstractRecords.recordBatchHeaderSizeInBytes(toMagic, batch.compressionType()); + + // if we are on version 2 and beyond, and we know we are going for in place assignment, + // then we can optimize the iterator to skip key / value / headers since they would not be used at all + CloseableIterator recordsIterator; + if (inPlaceAssignment && firstBatch.magic() >= RecordBatch.MAGIC_VALUE_V2) + recordsIterator = batch.skipKeyValueIterator(bufferSupplier); + else + recordsIterator = batch.streamingIterator(bufferSupplier); + + try { + List recordErrors = new ArrayList<>(0); + // this is a hot path and we want to avoid any unnecessary allocations. + int batchIndex = 0; + while (recordsIterator.hasNext()) { + Record record = recordsIterator.next(); + long expectedOffset = expectedInnerOffset.value++; + + Optional recordError = validateRecordCompression(sourceCompression, + batchIndex, record); + if (!recordError.isPresent()) { + recordError = validateRecord(batch, topicPartition, record, batchIndex, now, + timestampType, timestampDiffMaxMs, compactedTopic, metricsRecorder); + } + + if (!recordError.isPresent() + && batch.magic() > RecordBatch.MAGIC_VALUE_V0 + && toMagic > RecordBatch.MAGIC_VALUE_V0) { + + if (record.timestamp() > maxTimestamp) + maxTimestamp = record.timestamp(); + + // Some older clients do not implement the V1 internal offsets correctly. + // Historically the broker handled this by rewriting the batches rather + // than rejecting the request. We must continue this handling here to avoid + // breaking these clients. + if (record.offset() != expectedOffset) + inPlaceAssignment = false; + } + + if (recordError.isPresent()) + recordErrors.add(recordError.get()); + else { + uncompressedSizeInBytes += record.sizeInBytes(); + validatedRecords.add(record); + } + + ++batchIndex; + } + + processRecordErrors(recordErrors); + + } finally { + recordsIterator.close(); + } + } + + if (!inPlaceAssignment) { + return buildRecordsAndAssignOffsets(offsetCounter, now, firstBatch, validatedRecords, + uncompressedSizeInBytes); + } else { + // we can update the batch only and write the compressed payload as is; + // again we assume only one record batch within the compressed set + offsetCounter.value += validatedRecords.size(); + long lastOffset = offsetCounter.value - 1; + firstBatch.setLastOffset(lastOffset); + + if (timestampType == TimestampType.LOG_APPEND_TIME) + maxTimestamp = now; + + if (toMagic >= RecordBatch.MAGIC_VALUE_V1) + firstBatch.setMaxTimestamp(timestampType, maxTimestamp); + + if (toMagic >= RecordBatch.MAGIC_VALUE_V2) + firstBatch.setPartitionLeaderEpoch(partitionLeaderEpoch); + + RecordConversionStats recordConversionStats = new RecordConversionStats(uncompressedSizeInBytes, 0, 0); + return new ValidationResult( + now, + records, + maxTimestamp, + lastOffset, + false, + recordConversionStats); + } + } + + private ValidationResult buildRecordsAndAssignOffsets(LongRef offsetCounter, + long logAppendTime, + RecordBatch firstBatch, + List validatedRecords, + int uncompressedSizeInBytes) { + long startNanos = time.nanoseconds(); + int estimatedSize = AbstractRecords.estimateSizeInBytes(toMagic, offsetCounter.value, targetCompression, + validatedRecords); + // The current implementation of BufferSupplier is naive and works best when the buffer size + // cardinality is low, so don't use it here + ByteBuffer buffer = ByteBuffer.allocate(estimatedSize); + MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, toMagic, targetCompression, + timestampType, offsetCounter.value, logAppendTime, firstBatch.producerId(), + firstBatch.producerEpoch(), firstBatch.baseSequence(), firstBatch.isTransactional(), + partitionLeaderEpoch); + + for (Record record : validatedRecords) + builder.appendWithOffset(offsetCounter.value++, record); + + MemoryRecords records = builder.build(); + + RecordsInfo info = builder.info(); + + // This is not strictly correct, it represents the number of records where in-place assignment is not possible + // instead of the number of records that were converted. It will over-count cases where the source and target are + // message format V0 or if the inner offsets are not consecutive. This is OK since the impact is the same: we have + // to rebuild the records (including recompression if enabled). + int conversionCount = builder.numRecords(); + RecordConversionStats recordConversionStats = new RecordConversionStats( + uncompressedSizeInBytes + builder.uncompressedBytesWritten(), conversionCount, + time.nanoseconds() - startNanos); + + return new ValidationResult( + logAppendTime, + records, + info.maxTimestamp, + info.shallowOffsetOfMaxTimestamp, + true, + recordConversionStats); + } + + + private static void validateBatch(TopicPartition topicPartition, + RecordBatch firstBatch, + RecordBatch batch, + AppendOrigin origin, + byte toMagic, + MetricsRecorder metricsRecorder) { + // batch magic byte should have the same magic as the first batch + if (firstBatch.magic() != batch.magic()) { + metricsRecorder.recordInvalidMagic(); + throw new InvalidRecordException("Batch magic " + batch.magic() + " is not the same as the first batch's magic byte " + + firstBatch.magic() + " in topic partition " + topicPartition); + } + + if (origin == AppendOrigin.CLIENT) { + if (batch.magic() >= RecordBatch.MAGIC_VALUE_V2) { + long countFromOffsets = batch.lastOffset() - batch.baseOffset() + 1; + if (countFromOffsets <= 0) { + metricsRecorder.recordInvalidOffset(); + throw new InvalidRecordException("Batch has an invalid offset range: [" + batch.baseOffset() + ", " + + batch.lastOffset() + "] in topic partition " + topicPartition); + } + + // v2 and above messages always have a non-null count + long count = batch.countOrNull(); + if (count <= 0) { + metricsRecorder.recordInvalidOffset(); + throw new InvalidRecordException("Invalid reported count for record batch: " + count + + " in topic partition " + topicPartition); + } + + if (countFromOffsets != count) { + metricsRecorder.recordInvalidOffset(); + throw new InvalidRecordException("Inconsistent batch offset range [" + batch.baseOffset() + ", " + + batch.lastOffset() + "] and count of records " + count + " in topic partition " + topicPartition); + } + } + + if (batch.isControlBatch()) { + metricsRecorder.recordInvalidOffset(); + throw new InvalidRecordException("Clients are not allowed to write control records in topic partition " + topicPartition); + } + + if (batch.hasProducerId() && batch.baseSequence() < 0) { + metricsRecorder.recordInvalidSequence(); + throw new InvalidRecordException("Invalid sequence number " + batch.baseSequence() + " in record batch with producerId " + + batch.producerId() + " in topic partition " + topicPartition); + } + } + + if (batch.isTransactional() && toMagic < RecordBatch.MAGIC_VALUE_V2) + throw new UnsupportedForMessageFormatException("Transactional records cannot be used with magic version " + toMagic); + + if (batch.hasProducerId() && toMagic < RecordBatch.MAGIC_VALUE_V2) + throw new UnsupportedForMessageFormatException("Idempotent records cannot be used with magic version " + toMagic); + } + + private static Optional validateRecord(RecordBatch batch, + TopicPartition topicPartition, + Record record, + int batchIndex, + long now, + TimestampType timestampType, + long timestampDiffMaxMs, + boolean compactedTopic, + MetricsRecorder metricsRecorder) { + if (!record.hasMagic(batch.magic())) { + metricsRecorder.recordInvalidMagic(); + return Optional.of(new ApiRecordError(Errors.INVALID_RECORD, + new RecordError(batchIndex, "Record " + record + + "'s magic does not match outer magic " + batch.magic() + " in topic partition " + + topicPartition))); + } + + // verify the record-level CRC only if this is one of the deep entries of a compressed message + // set for magic v0 and v1. For non-compressed messages, there is no inner record for magic v0 and v1, + // so we depend on the batch-level CRC check in Log.analyzeAndValidateRecords(). For magic v2 and above, + // there is no record-level CRC to check. + if (batch.magic() <= RecordBatch.MAGIC_VALUE_V1 && batch.isCompressed()) { + try { + record.ensureValid(); + } catch (InvalidRecordException e) { + metricsRecorder.recordInvalidChecksums(); + throw new CorruptRecordException(e.getMessage() + " in topic partition " + topicPartition); + } + } + + Optional keyError = validateKey(record, batchIndex, topicPartition, + compactedTopic, metricsRecorder); + if (keyError.isPresent()) + return keyError; + else + return validateTimestamp(batch, record, batchIndex, now, timestampType, timestampDiffMaxMs); + } + + private static Optional validateKey(Record record, + int batchIndex, + TopicPartition topicPartition, + boolean compactedTopic, + MetricsRecorder metricsRecorder) { + if (compactedTopic && !record.hasKey()) { + metricsRecorder.recordNoKeyCompactedTopic(); + return Optional.of(new ApiRecordError(Errors.INVALID_RECORD, new RecordError(batchIndex, + "Compacted topic cannot accept message without key in topic partition " + + topicPartition))); + } else + return Optional.empty(); + } + + private static Optional validateTimestamp(RecordBatch batch, + Record record, + int batchIndex, + long now, + TimestampType timestampType, + long timestampDiffMaxMs) { + if (timestampType == TimestampType.CREATE_TIME + && record.timestamp() != RecordBatch.NO_TIMESTAMP + && Math.abs(record.timestamp() - now) > timestampDiffMaxMs) + return Optional.of(new ApiRecordError(Errors.INVALID_TIMESTAMP, new RecordError(batchIndex, + "Timestamp " + record.timestamp() + " of message with offset " + record.offset() + + " is out of range. The timestamp should be within [" + (now - timestampDiffMaxMs) + + ", " + (now + timestampDiffMaxMs) + "]"))); + else if (batch.timestampType() == TimestampType.LOG_APPEND_TIME) + return Optional.of(new ApiRecordError(Errors.INVALID_TIMESTAMP, new RecordError(batchIndex, + "Invalid timestamp type in message " + record + ". Producer should not set timestamp " + + "type to LogAppendTime."))); + else + return Optional.empty(); + } + + private static Optional validateRecordCompression(CompressionType sourceCompression, + int batchIndex, + Record record) { + if (sourceCompression != CompressionType.NONE && record.isCompressed()) + return Optional.of(new ApiRecordError(Errors.INVALID_RECORD, new RecordError(batchIndex, + "Compressed outer record should not have an inner record with a compression attribute set: " + + record))); + else + return Optional.empty(); + } + + private static void processRecordErrors(List recordErrors) { + if (!recordErrors.isEmpty()) { + List errors = recordErrors.stream().map(e -> e.recordError).collect(Collectors.toList()); + if (recordErrors.stream().anyMatch(e -> e.apiError == Errors.INVALID_TIMESTAMP)) { + throw new RecordValidationException(new InvalidTimestampException( + "One or more records have been rejected due to invalid timestamp"), errors); + } else { + throw new RecordValidationException(new InvalidRecordException( + "One or more records have been rejected due to " + errors.size() + " record errors " + + "in total, and only showing the first three errors at most: " + errors.subList(0, Math.min(errors.size(), 3))), errors); + } + } + } +} diff --git a/core/src/main/scala/kafka/common/RecordValidationException.scala b/storage/src/main/java/org/apache/kafka/server/log/internals/OffsetMap.java similarity index 53% rename from core/src/main/scala/kafka/common/RecordValidationException.scala rename to storage/src/main/java/org/apache/kafka/server/log/internals/OffsetMap.java index baa7d72557604..d8df1dcffb34c 100644 --- a/core/src/main/scala/kafka/common/RecordValidationException.scala +++ b/storage/src/main/java/org/apache/kafka/server/log/internals/OffsetMap.java @@ -1,10 +1,10 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with + * 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 + * the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -14,15 +14,20 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +package org.apache.kafka.server.log.internals; -package kafka.common +import java.nio.ByteBuffer; +import java.security.DigestException; -import org.apache.kafka.common.errors.ApiException -import org.apache.kafka.common.requests.ProduceResponse.RecordError - -import scala.collection.Seq - -class RecordValidationException(val invalidException: ApiException, - val recordErrors: Seq[RecordError]) - extends RuntimeException(invalidException) { +public interface OffsetMap { + int slots(); + void put(ByteBuffer key, long offset) throws DigestException; + long get(ByteBuffer key) throws DigestException; + void updateLatestOffset(long offset); + void clear(); + int size(); + long latestOffset(); + default double utilization() { + return size() / (double) slots(); + } } diff --git a/storage/src/main/java/org/apache/kafka/server/log/internals/RecordValidationException.java b/storage/src/main/java/org/apache/kafka/server/log/internals/RecordValidationException.java new file mode 100644 index 0000000000000..164e1e209c644 --- /dev/null +++ b/storage/src/main/java/org/apache/kafka/server/log/internals/RecordValidationException.java @@ -0,0 +1,41 @@ +/* + * 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.server.log.internals; + +import java.util.Collections; +import java.util.List; +import org.apache.kafka.common.errors.ApiException; +import org.apache.kafka.common.requests.ProduceResponse.RecordError; + +public class RecordValidationException extends RuntimeException { + private final ApiException invalidException; + private final List recordErrors; + + public RecordValidationException(ApiException invalidException, List recordErrors) { + super(invalidException); + this.invalidException = invalidException; + this.recordErrors = Collections.unmodifiableList(recordErrors); + } + + public ApiException invalidException() { + return invalidException; + } + + public List recordErrors() { + return recordErrors; + } +} diff --git a/storage/src/main/java/org/apache/kafka/server/log/internals/SkimpyOffsetMap.java b/storage/src/main/java/org/apache/kafka/server/log/internals/SkimpyOffsetMap.java new file mode 100644 index 0000000000000..b5b47d94512a3 --- /dev/null +++ b/storage/src/main/java/org/apache/kafka/server/log/internals/SkimpyOffsetMap.java @@ -0,0 +1,242 @@ +/* + * 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.server.log.internals; + +import org.apache.kafka.common.utils.ByteUtils; +import org.apache.kafka.common.utils.Utils; + +import java.nio.ByteBuffer; +import java.security.DigestException; +import java.security.MessageDigest; +import java.security.NoSuchAlgorithmException; +import java.util.Arrays; + +/** + * A hash table used for de-duplicating the log. This hash table uses a cryptographically secure hash of the key as a proxy for the key + * for comparisons and to save space on object overhead. Collisions are resolved by probing. This hash table does not support deletes. + */ +public class SkimpyOffsetMap implements OffsetMap { + + /** + * The number of bytes of space each entry uses (the number of bytes in the hash plus an 8 byte offset) + */ + public final int bytesPerEntry; + + private final ByteBuffer bytes; + + /* the hash algorithm instance to use */ + private final MessageDigest digest; + + /* the number of bytes for this hash algorithm */ + private final int hashSize; + + /** + * The maximum number of entries this map can contain + */ + private final int slots; + + /* cache some hash buffers to avoid reallocating each time */ + private final byte[] hash1; + private final byte[] hash2; + + /* number of entries put into the map */ + private int entries = 0; + + /* number of lookups on the map */ + private long lookups = 0L; + + /* the number of probes for all lookups */ + private long probes = 0L; + + /* the latest offset written into the map */ + private long lastOffset = -1L; + + /** + * Create an instance of SkimplyOffsetMap with the default hash algorithm (MD5). + * + * @param memory The amount of memory this map can use + */ + public SkimpyOffsetMap(int memory) throws NoSuchAlgorithmException { + this(memory, "MD5"); + } + + /** + * Create an instance of SkimpyOffsetMap. + * + * @param memory The amount of memory this map can use + * @param hashAlgorithm The hash algorithm instance to use: MD2, MD5, SHA-1, SHA-256, SHA-384, SHA-512 + */ + public SkimpyOffsetMap(int memory, String hashAlgorithm) throws NoSuchAlgorithmException { + this.bytes = ByteBuffer.allocate(memory); + + this.digest = MessageDigest.getInstance(hashAlgorithm); + + this.hashSize = digest.getDigestLength(); + this.bytesPerEntry = hashSize + 8; + this.slots = memory / bytesPerEntry; + + this.hash1 = new byte[hashSize]; + this.hash2 = new byte[hashSize]; + } + + @Override + public int slots() { + return slots; + } + + /** + * Get the offset associated with this key. + * @param key The key + * @return The offset associated with this key or -1 if the key is not found + */ + @Override + public long get(ByteBuffer key) throws DigestException { + ++lookups; + hashInto(key, hash1); + // search for the hash of this key by repeated probing until we find the hash we are looking for or we find an empty slot + int attempt = 0; + int pos = 0; + //we need to guard against attempt integer overflow if the map is full + //limit attempt to number of slots once positionOf(..) enters linear search mode + int maxAttempts = slots + hashSize - 4; + do { + if (attempt >= maxAttempts) + return -1L; + pos = positionOf(hash1, attempt); + bytes.position(pos); + if (isEmpty(pos)) + return -1L; + bytes.get(hash2); + ++attempt; + } while (!Arrays.equals(hash1, hash2)); + return bytes.getLong(); + } + + /** + * Associate this offset to the given key. + * @param key The key + * @param offset The offset + */ + @Override + public void put(ByteBuffer key, long offset) throws DigestException { + if (entries >= slots) + throw new IllegalArgumentException("Attempted to add a new entry to a full offset map, " + + "entries: " + entries + ", slots: " + slots); + + ++lookups; + hashInto(key, hash1); + + // probe until we find the first empty slot + int attempt = 0; + int pos = positionOf(hash1, attempt); + while (!isEmpty(pos)) { + bytes.position(pos); + bytes.get(hash2); + if (Arrays.equals(hash1, hash2)) { + // we found an existing entry, overwrite it and return (size does not change) + bytes.putLong(offset); + lastOffset = offset; + return; + } + ++attempt; + pos = positionOf(hash1, attempt); + } + + // found an empty slot, update it - size grows by 1 + bytes.position(pos); + bytes.put(hash1); + bytes.putLong(offset); + lastOffset = offset; + ++entries; + } + + @Override + public void updateLatestOffset(long offset) { + this.lastOffset = offset; + } + + /** + * Change the salt used for key hashing making all existing keys unfindable. + */ + @Override + public void clear() { + this.entries = 0; + this.lookups = 0L; + this.probes = 0L; + this.lastOffset = -1L; + Arrays.fill(bytes.array(), bytes.arrayOffset(), bytes.arrayOffset() + bytes.limit(), (byte) 0); + } + + /** + * The number of entries put into the map (note that not all may remain) + */ + @Override + public int size() { + return entries; + } + + /** + * The latest offset put into the map + */ + @Override + public long latestOffset() { + return lastOffset; + } + + /** + * The rate of collisions in the lookups + */ + // Visible for testing + public double collisionRate() { + return (this.probes - this.lookups) / (double) this.lookups; + } + + /** + * Check that there is no entry at the given position + */ + private boolean isEmpty(int position) { + return bytes.getLong(position) == 0 + && bytes.getLong(position + 8) == 0 + && bytes.getLong(position + 16) == 0; + } + + /** + * Calculate the ith probe position. We first try reading successive integers from the hash itself + * then if all of those fail we degrade to linear probing. + * @param hash The hash of the key to find the position for + * @param attempt The ith probe + * @return The byte offset in the buffer at which the ith probing for the given hash would reside + */ + private int positionOf(byte[] hash, int attempt) { + int probe = ByteUtils.readIntBE(hash, Math.min(attempt, hashSize - 4)) + Math.max(0, attempt - hashSize + 4); + int slot = Utils.abs(probe) % slots; + ++this.probes; + return slot * bytesPerEntry; + } + + /** + * The offset at which we have stored the given key + * @param key The key to hash + * @param buffer The buffer to store the hash into + */ + private void hashInto(ByteBuffer key, byte[] buffer) throws DigestException { + key.mark(); + digest.update(key); + key.reset(); + digest.digest(buffer, 0, hashSize); + } +} diff --git a/storage/src/main/java/org/apache/kafka/server/log/internals/TimeIndex.java b/storage/src/main/java/org/apache/kafka/server/log/internals/TimeIndex.java new file mode 100644 index 0000000000000..8c7c6d8d58c2a --- /dev/null +++ b/storage/src/main/java/org/apache/kafka/server/log/internals/TimeIndex.java @@ -0,0 +1,286 @@ +/* + * 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.server.log.internals; + +import org.apache.kafka.common.errors.InvalidOffsetException; +import org.apache.kafka.common.record.RecordBatch; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.MappedByteBuffer; + +/** + * An index that maps from the timestamp to the logical offsets of the messages in a segment. This index might be + * sparse, i.e. it may not hold an entry for all the messages in the segment. + * + * The index is stored in a file that is preallocated to hold a fixed maximum amount of 12-byte time index entries. + * The file format is a series of time index entries. The physical format is a 8 bytes timestamp and a 4 bytes "relative" + * offset used in the [[OffsetIndex]]. A time index entry (TIMESTAMP, OFFSET) means that the biggest timestamp seen + * before OFFSET is TIMESTAMP. i.e. Any message whose timestamp is greater than TIMESTAMP must come after OFFSET. + * + * All external APIs translate from relative offsets to full offsets, so users of this class do not interact with the internal + * storage format. + * + * The timestamps in the same time index file are guaranteed to be monotonically increasing. + * + * The index supports timestamp lookup for a memory map of this file. The lookup is done using a binary search to find + * the offset of the message whose indexed timestamp is closest but smaller or equals to the target timestamp. + * + * Time index files can be opened in two ways: either as an empty, mutable index that allows appending or + * an immutable read-only index file that has previously been populated. The makeReadOnly method will turn a mutable file into an + * immutable one and truncate off any extra bytes. This is done when the index file is rolled over. + * + * No attempt is made to checksum the contents of this file, in the event of a crash it is rebuilt. + * + */ +public class TimeIndex extends AbstractIndex { + private static final Logger log = LoggerFactory.getLogger(TimeIndex.class); + private static final int ENTRY_SIZE = 12; + + private volatile TimestampOffset lastEntry; + + public TimeIndex(File file, long baseOffset, int maxIndexSize) throws IOException { + this(file, baseOffset, maxIndexSize, true); + } + + public TimeIndex(File file, long baseOffset, int maxIndexSize, boolean writable) throws IOException { + super(file, baseOffset, maxIndexSize, writable); + + this.lastEntry = lastEntryFromIndexFile(); + + log.debug("Loaded index file {} with maxEntries = {}, maxIndexSize = {}, entries = {}, lastOffset = {}, file position = {}", + file.getAbsolutePath(), maxEntries(), maxIndexSize, entries(), lastEntry, mmap().position()); + } + + @Override + public void sanityCheck() { + TimestampOffset entry = lastEntry(); + long lastTimestamp = entry.timestamp; + long lastOffset = entry.offset; + if (entries() != 0 && lastTimestamp < timestamp(mmap(), 0)) + throw new CorruptIndexException("Corrupt time index found, time index file (" + file().getAbsolutePath() + ") has " + + "non-zero size but the last timestamp is " + lastTimestamp + " which is less than the first timestamp " + + timestamp(mmap(), 0)); + if (entries() != 0 && lastOffset < baseOffset()) + throw new CorruptIndexException("Corrupt time index found, time index file (" + file().getAbsolutePath() + ") has " + + "non-zero size but the last offset is " + lastOffset + " which is less than the first offset " + baseOffset()); + if (length() % ENTRY_SIZE != 0) + throw new CorruptIndexException("Time index file " + file().getAbsolutePath() + " is corrupt, found " + length() + + " bytes which is neither positive nor a multiple of " + ENTRY_SIZE); + } + + /** + * Remove all entries from the index which have an offset greater than or equal to the given offset. + * Truncating to an offset larger than the largest in the index has no effect. + */ + @Override + public void truncateTo(long offset) { + lock.lock(); + try { + ByteBuffer idx = mmap().duplicate(); + int slot = largestLowerBoundSlotFor(idx, offset, IndexSearchType.VALUE); + + /* There are 3 cases for choosing the new size + * 1) if there is no entry in the index <= the offset, delete everything + * 2) if there is an entry for this exact offset, delete it and everything larger than it + * 3) if there is no entry for this offset, delete everything larger than the next smallest + */ + int newEntries; + if (slot < 0) + newEntries = 0; + else if (relativeOffset(idx, slot) == offset - baseOffset()) + newEntries = slot; + else + newEntries = slot + 1; + + truncateToEntries(newEntries); + } finally { + lock.unlock(); + } + } + + // We override the full check to reserve the last time index entry slot for the on roll call. + @Override + public boolean isFull() { + return entries() >= maxEntries() - 1; + } + + public TimestampOffset lastEntry() { + return lastEntry; + } + + /** + * Get the nth timestamp mapping from the time index + * @param n The entry number in the time index + * @return The timestamp/offset pair at that entry + */ + public TimestampOffset entry(int n) { + return maybeLock(lock, () -> { + if (n >= entries()) + throw new IllegalArgumentException("Attempt to fetch the " + n + "th entry from time index " + + file().getAbsolutePath() + " which has size " + entries()); + return parseEntry(mmap(), n); + }); + } + + /** + * Find the time index entry whose timestamp is less than or equal to the given timestamp. + * If the target timestamp is smaller than the least timestamp in the time index, (NoTimestamp, baseOffset) is + * returned. + * + * @param targetTimestamp The timestamp to look up. + * @return The time index entry found. + */ + public TimestampOffset lookup(long targetTimestamp) { + return maybeLock(lock, () -> { + ByteBuffer idx = mmap().duplicate(); + int slot = largestLowerBoundSlotFor(idx, targetTimestamp, IndexSearchType.KEY); + if (slot == -1) + return new TimestampOffset(RecordBatch.NO_TIMESTAMP, baseOffset()); + else + return parseEntry(idx, slot); + }); + } + + /** + * Equivalent to invoking `maybeAppend(timestamp, offset, false)`. + * + * @see #maybeAppend(long, long, boolean) + */ + public void maybeAppend(long timestamp, long offset) { + maybeAppend(timestamp, offset, false); + } + + /** + * Attempt to append a time index entry to the time index. + * The new entry is appended only if both the timestamp and offset are greater than the last appended timestamp and + * the last appended offset. + * + * @param timestamp The timestamp of the new time index entry + * @param offset The offset of the new time index entry + * @param skipFullCheck To skip checking whether the segment is full or not. We only skip the check when the segment + * gets rolled or the segment is closed. + */ + public void maybeAppend(long timestamp, long offset, boolean skipFullCheck) { + lock.lock(); + try { + if (!skipFullCheck && isFull()) + throw new IllegalArgumentException("Attempt to append to a full time index (size = " + entries() + ")."); + + // We do not throw exception when the offset equals to the offset of last entry. That means we are trying + // to insert the same time index entry as the last entry. + // If the timestamp index entry to be inserted is the same as the last entry, we simply ignore the insertion + // because that could happen in the following two scenarios: + // 1. A log segment is closed. + // 2. LogSegment.onBecomeInactiveSegment() is called when an active log segment is rolled. + if (entries() != 0 && offset < lastEntry.offset) + throw new InvalidOffsetException("Attempt to append an offset (" + offset + ") to slot " + entries() + + " no larger than the last offset appended (" + lastEntry.offset + ") to " + file().getAbsolutePath()); + if (entries() != 0 && timestamp < lastEntry.timestamp) + throw new IllegalStateException("Attempt to append a timestamp (" + timestamp + ") to slot " + entries() + + " no larger than the last timestamp appended (" + lastEntry.timestamp + ") to " + file().getAbsolutePath()); + + // We only append to the time index when the timestamp is greater than the last inserted timestamp. + // If all the messages are in message format v0, the timestamp will always be NoTimestamp. In that case, the time + // index will be empty. + if (timestamp > lastEntry.timestamp) { + log.trace("Adding index entry {} => {} to {}.", timestamp, offset, file().getAbsolutePath()); + MappedByteBuffer mmap = mmap(); + mmap.putLong(timestamp); + mmap.putInt(relativeOffset(offset)); + incrementEntries(); + this.lastEntry = new TimestampOffset(timestamp, offset); + if (entries() * ENTRY_SIZE != mmap.position()) + throw new IllegalStateException(entries() + " entries but file position in index is " + mmap.position()); + } + } finally { + lock.unlock(); + } + } + + @Override + public boolean resize(int newSize) throws IOException { + lock.lock(); + try { + if (super.resize(newSize)) { + this.lastEntry = lastEntryFromIndexFile(); + return true; + } else + return false; + } finally { + lock.unlock(); + } + } + + // Visible for testing, we can make this protected once TimeIndexTest is in the same package as this class + @Override + public void truncate() { + truncateToEntries(0); + } + + @Override + protected int entrySize() { + return ENTRY_SIZE; + } + + @Override + protected TimestampOffset parseEntry(ByteBuffer buffer, int n) { + return new TimestampOffset(timestamp(buffer, n), baseOffset() + relativeOffset(buffer, n)); + } + + private long timestamp(ByteBuffer buffer, int n) { + return buffer.getLong(n * ENTRY_SIZE); + } + + private int relativeOffset(ByteBuffer buffer, int n) { + return buffer.getInt(n * ENTRY_SIZE + 8); + } + + /** + * Read the last entry from the index file. This operation involves disk access. + */ + private TimestampOffset lastEntryFromIndexFile() { + lock.lock(); + try { + int entries = entries(); + if (entries == 0) + return new TimestampOffset(RecordBatch.NO_TIMESTAMP, baseOffset()); + else + return parseEntry(mmap(), entries - 1); + } finally { + lock.unlock(); + } + } + + /** + * Truncates index to a known number of entries. + */ + private void truncateToEntries(int entries) { + lock.lock(); + try { + super.truncateToEntries0(entries); + this.lastEntry = lastEntryFromIndexFile(); + log.debug("Truncated index {} to {} entries; position is now {} and last entry is now {}", + file().getAbsolutePath(), entries, mmap().position(), lastEntry); + } finally { + lock.unlock(); + } + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/DefaultStateUpdater.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/DefaultStateUpdater.java index de193eff2e73d..d96593c5011d7 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/DefaultStateUpdater.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/DefaultStateUpdater.java @@ -106,6 +106,7 @@ public void run() { } catch (final RuntimeException anyOtherException) { handleRuntimeException(anyOtherException); } finally { + Thread.interrupted(); // Clear the interrupted flag. removeAddedTasksFromInputQueue(); removeUpdatingAndPausedTasks(); shutdownGate.countDown(); @@ -443,8 +444,8 @@ public void start() { @Override public void shutdown(final Duration timeout) { if (stateUpdaterThread != null) { - stateUpdaterThread.isRunning.set(false); stateUpdaterThread.interrupt(); + stateUpdaterThread.isRunning.set(false); try { if (!shutdownGate.await(timeout.toMillis(), TimeUnit.MILLISECONDS)) { throw new StreamsException("State updater thread did not shutdown within the timeout"); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java index 4b7d85a5d7831..1875f57b649f8 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java @@ -422,7 +422,6 @@ public GroupAssignment assign(final Cluster metadata, final GroupSubscription gr if (minReceivedMetadataVersion >= 2) { populatePartitionsByHostMaps(partitionsByHost, standbyPartitionsByHost, partitionsForTask, clientMetadataMap); } - streamsMetadataState.onChange(partitionsByHost, standbyPartitionsByHost, fullMetadata); // ---------------- Step Four ---------------- // diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/StoreQueryIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/StoreQueryIntegrationTest.java index 260d6992f29b2..5799383bcde9a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/StoreQueryIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/StoreQueryIntegrationTest.java @@ -41,17 +41,17 @@ import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.streams.state.QueryableStoreType; import org.apache.kafka.streams.state.ReadOnlyKeyValueStore; -import org.apache.kafka.test.IntegrationTest; import org.apache.kafka.test.TestCondition; import org.apache.kafka.test.TestUtils; import org.hamcrest.Matcher; -import org.junit.After; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.junit.rules.TestName; -import org.junit.rules.Timeout; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.api.TestInfo; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -86,10 +86,9 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -@Category({IntegrationTest.class}) +@Timeout(600) +@Tag("integration") public class StoreQueryIntegrationTest { - @Rule - public Timeout globalTimeout = Timeout.seconds(600); private static final Logger LOG = LoggerFactory.getLogger(StoreQueryIntegrationTest.class); @@ -98,27 +97,35 @@ public class StoreQueryIntegrationTest { private static final String INPUT_TOPIC_NAME = "input-topic"; private static final String TABLE_NAME = "source-table"; - public final EmbeddedKafkaCluster cluster = new EmbeddedKafkaCluster(NUM_BROKERS); + public final static EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster(NUM_BROKERS); - @Rule - public TestName testName = new TestName(); + private String appId; private final List streamsToCleanup = new ArrayList<>(); - private final MockTime mockTime = cluster.time; + private final MockTime mockTime = CLUSTER.time; - @Before - public void before() throws InterruptedException, IOException { - cluster.start(); - cluster.createTopic(INPUT_TOPIC_NAME, 2, 1); + @BeforeAll + public static void setupCluster() throws InterruptedException, IOException { + CLUSTER.start(); + CLUSTER.createTopic(INPUT_TOPIC_NAME, 2, 1); } - @After + @BeforeEach + public void before(final TestInfo testInfo) throws InterruptedException, IOException { + this.appId = safeUniqueTestName(getClass(), testInfo); + } + + @AfterEach public void after() { for (final KafkaStreams kafkaStreams : streamsToCleanup) { kafkaStreams.close(); } streamsToCleanup.clear(); - cluster.stop(); + } + + @AfterAll + public static void stopCluster() { + CLUSTER.stop(); } @Test @@ -131,6 +138,7 @@ public void shouldQueryOnlyActivePartitionStoresByDefault() throws Exception { getStreamsBuilderWithTopology(builder, semaphore); final KafkaStreams kafkaStreams1 = createKafkaStreams(builder, streamsConfiguration()); + final int kafkaStreams1Port = port; final KafkaStreams kafkaStreams2 = createKafkaStreams(builder, streamsConfiguration()); final List kafkaStreamsList = Arrays.asList(kafkaStreams1, kafkaStreams2); @@ -148,7 +156,7 @@ public void shouldQueryOnlyActivePartitionStoresByDefault() throws Exception { final ReadOnlyKeyValueStore store1 = getStore(TABLE_NAME, kafkaStreams1, queryableStoreType); final ReadOnlyKeyValueStore store2 = getStore(TABLE_NAME, kafkaStreams2, queryableStoreType); - final boolean kafkaStreams1IsActive = (keyQueryMetadata.activeHost().port() % 2) == 0; + final boolean kafkaStreams1IsActive = keyQueryMetadata.activeHost().port() == kafkaStreams1Port; try { if (kafkaStreams1IsActive) { @@ -177,6 +185,7 @@ public void shouldQuerySpecificActivePartitionStores() throws Exception { getStreamsBuilderWithTopology(builder, semaphore); final KafkaStreams kafkaStreams1 = createKafkaStreams(builder, streamsConfiguration()); + final int kafkaStreams1Port = port; final KafkaStreams kafkaStreams2 = createKafkaStreams(builder, streamsConfiguration()); final List kafkaStreamsList = Arrays.asList(kafkaStreams1, kafkaStreams2); @@ -194,7 +203,7 @@ public void shouldQuerySpecificActivePartitionStores() throws Exception { //key doesn't belongs to this partition final int keyDontBelongPartition = (keyPartition == 0) ? 1 : 0; - final boolean kafkaStreams1IsActive = (keyQueryMetadata.activeHost().port() % 2) == 1; + final boolean kafkaStreams1IsActive = keyQueryMetadata.activeHost().port() == kafkaStreams1Port; final StoreQueryParameters> storeQueryParam = StoreQueryParameters.>fromNameAndType(TABLE_NAME, keyValueStore()) @@ -541,7 +550,7 @@ public void shouldQueryStoresAfterAddingAndRemovingStreamThread() throws Excepti }); } - @Test(expected = IllegalArgumentException.class) + @Test public void shouldFailWithIllegalArgumentExceptionWhenIQPartitionerReturnsMultiplePartitions() throws Exception { class BroadcastingPartitioner implements StreamPartitioner { @@ -571,7 +580,7 @@ public Optional> partitions(final String topic, final Integer key, assertThat(semaphore.tryAcquire(batch1NumMessages, 60, TimeUnit.SECONDS), is(equalTo(true))); - kafkaStreams1.queryMetadataForKey(TABLE_NAME, key, new BroadcastingPartitioner()); + assertThrows(IllegalArgumentException.class, () -> kafkaStreams1.queryMetadataForKey(TABLE_NAME, key, new BroadcastingPartitioner())); } @@ -636,7 +645,7 @@ private KafkaStreamsNamedTopologyWrapper createNamedTopologyKafkaStreams(final P private void produceValueRange(final int key, final int start, final int endExclusive) { final Properties producerProps = new Properties(); - producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.bootstrapServers()); + producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, IntegerSerializer.class); producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, IntegerSerializer.class); @@ -650,12 +659,11 @@ private void produceValueRange(final int key, final int start, final int endExcl } private Properties streamsConfiguration() { - final String safeTestName = safeUniqueTestName(getClass(), testName); final Properties config = new Properties(); config.put(StreamsConfig.TOPOLOGY_OPTIMIZATION_CONFIG, StreamsConfig.OPTIMIZE); - config.put(StreamsConfig.APPLICATION_ID_CONFIG, "app-" + safeTestName); + config.put(StreamsConfig.APPLICATION_ID_CONFIG, "app-" + appId); config.put(StreamsConfig.APPLICATION_SERVER_CONFIG, "localhost:" + (++port)); - config.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.bootstrapServers()); + config.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); config.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath()); config.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.Integer().getClass()); config.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.Integer().getClass()); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionStoreReceiveProcessorSupplierTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionStoreReceiveProcessorSupplierTest.java index 17f34d56d3ffc..59636556c402b 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionStoreReceiveProcessorSupplierTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionStoreReceiveProcessorSupplierTest.java @@ -50,6 +50,7 @@ public class SubscriptionStoreReceiveProcessorSupplierTest { private final Properties props = StreamsTestUtils.getStreamsConfig(Serdes.String(), Serdes.String()); private File stateDir; private MockInternalNewProcessorContext, Change>>> context; + private TimestampedKeyValueStore> stateStore = null; private static final String FK = "fk1"; private static final String PK1 = "pk1"; @@ -71,6 +72,9 @@ public void before() { @After public void after() throws IOException { + if (stateStore != null) { + stateStore.close(); + } Utils.delete(stateDir); } @@ -88,7 +92,7 @@ public void shouldDeleteKeyAndPropagateV0() { SubscriptionWrapper, CombinedKey, Change>>> processor = supplier.get(); - final TimestampedKeyValueStore> stateStore = storeBuilder.build(); + stateStore = storeBuilder.build(); context.addStateStore(stateStore); stateStore.init((StateStoreContext) context, stateStore); @@ -140,10 +144,9 @@ public void shouldDeleteKeyAndPropagateV1() { SubscriptionWrapper, CombinedKey, Change>>> processor = supplier.get(); - final TimestampedKeyValueStore> stateStore = storeBuilder.build(); + stateStore = storeBuilder.build(); context.addStateStore(stateStore); stateStore.init((StateStoreContext) context, stateStore); - final SubscriptionWrapper oldWrapper = new SubscriptionWrapper<>( new long[]{1L, 2L}, Instruction.DELETE_KEY_AND_PROPAGATE, @@ -192,7 +195,7 @@ public void shouldDeleteKeyNoPropagateV0() { SubscriptionWrapper, CombinedKey, Change>>> processor = supplier.get(); - final TimestampedKeyValueStore> stateStore = storeBuilder.build(); + stateStore = storeBuilder.build(); context.addStateStore(stateStore); stateStore.init((StateStoreContext) context, stateStore); @@ -244,7 +247,7 @@ public void shouldDeleteKeyNoPropagateV1() { SubscriptionWrapper, CombinedKey, Change>>> processor = supplier.get(); - final TimestampedKeyValueStore> stateStore = storeBuilder.build(); + stateStore = storeBuilder.build(); context.addStateStore(stateStore); stateStore.init((StateStoreContext) context, stateStore); @@ -296,7 +299,7 @@ public void shouldPropagateOnlyIfFKValAvailableV0() { SubscriptionWrapper, CombinedKey, Change>>> processor = supplier.get(); - final TimestampedKeyValueStore> stateStore = storeBuilder.build(); + stateStore = storeBuilder.build(); context.addStateStore(stateStore); stateStore.init((StateStoreContext) context, stateStore); @@ -348,7 +351,7 @@ public void shouldPropagateOnlyIfFKValAvailableV1() { SubscriptionWrapper, CombinedKey, Change>>> processor = supplier.get(); - final TimestampedKeyValueStore> stateStore = storeBuilder.build(); + stateStore = storeBuilder.build(); context.addStateStore(stateStore); stateStore.init((StateStoreContext) context, stateStore); @@ -400,7 +403,7 @@ public void shouldPropagateNullIfNoFKValAvailableV0() { SubscriptionWrapper, CombinedKey, Change>>> processor = supplier.get(); - final TimestampedKeyValueStore> stateStore = storeBuilder.build(); + stateStore = storeBuilder.build(); context.addStateStore(stateStore); stateStore.init((StateStoreContext) context, stateStore); @@ -452,7 +455,7 @@ public void shouldPropagateNullIfNoFKValAvailableV1() { SubscriptionWrapper, CombinedKey, Change>>> processor = supplier.get(); - final TimestampedKeyValueStore> stateStore = storeBuilder.build(); + stateStore = storeBuilder.build(); context.addStateStore(stateStore); stateStore.init((StateStoreContext) context, stateStore); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStoreTest.java index 6e1b3bfcf8cbf..852f6fd3cebff 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStoreTest.java @@ -480,6 +480,7 @@ public void shouldCreateWriteBatches() { assertEquals(2, writeBatchMap.size()); for (final WriteBatch batch : writeBatchMap.values()) { assertEquals(1, batch.count()); + batch.close(); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/BlockBasedTableConfigWithAccessibleCacheTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/BlockBasedTableConfigWithAccessibleCacheTest.java index 1904789ec72a6..ffcbef23ce850 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/BlockBasedTableConfigWithAccessibleCacheTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/BlockBasedTableConfigWithAccessibleCacheTest.java @@ -44,11 +44,12 @@ public void shouldReturnNoBlockCacheIfNoneIsSet() { public void shouldSetBlockCacheAndMakeItAccessible() { final BlockBasedTableConfigWithAccessibleCache configWithAccessibleCache = new BlockBasedTableConfigWithAccessibleCache(); - final Cache blockCache = new LRUCache(1024); + try (final Cache blockCache = new LRUCache(1024)) { - final BlockBasedTableConfig updatedConfig = configWithAccessibleCache.setBlockCache(blockCache); + final BlockBasedTableConfig updatedConfig = configWithAccessibleCache.setBlockCache(blockCache); - assertThat(updatedConfig, sameInstance(configWithAccessibleCache)); - assertThat(configWithAccessibleCache.blockCache(), sameInstance(blockCache)); + assertThat(updatedConfig, sameInstance(configWithAccessibleCache)); + assertThat(configWithAccessibleCache.blockCache(), sameInstance(blockCache)); + } } } \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsAdapterTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsAdapterTest.java index 8eec0a40056fe..727f46f81a1e9 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsAdapterTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsAdapterTest.java @@ -64,6 +64,7 @@ import static org.easymock.EasyMock.mock; import static org.easymock.EasyMock.replay; import static org.easymock.EasyMock.reset; +import static org.easymock.EasyMock.resetToNice; import static org.easymock.EasyMock.verify; import static org.hamcrest.CoreMatchers.hasItem; import static org.hamcrest.MatcherAssert.assertThat; @@ -148,6 +149,9 @@ private void verifyDBOptionsMethodCall(final Method method) throws Exception { assertThat(undeclaredMockMethodCall.getCause(), instanceOf(AssertionError.class)); assertThat(undeclaredMockMethodCall.getCause().getMessage().trim(), matchesPattern("Unexpected method call DBOptions\\." + method.getName() + "((.*\n*)*):")); + } finally { + resetToNice(mockedDbOptions); + optionsFacadeDbOptions.close(); } } @@ -253,6 +257,9 @@ private void verifyColumnFamilyOptionsMethodCall(final Method method) throws Exc assertThat(undeclaredMockMethodCall.getCause(), instanceOf(AssertionError.class)); assertThat(undeclaredMockMethodCall.getCause().getMessage().trim(), matchesPattern("Unexpected method call ColumnFamilyOptions\\." + method.getName() + "(.*)")); + } finally { + resetToNice(mockedColumnFamilyOptions); + optionsFacadeColumnFamilyOptions.close(); } } @@ -333,24 +340,23 @@ public void shouldLogWarningWhenSettingWalOptions() throws Exception { try (final LogCaptureAppender appender = LogCaptureAppender.createAndRegister(RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsAdapter.class)) { - final RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsAdapter adapter - = new RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsAdapter(new DBOptions(), new ColumnFamilyOptions()); - - for (final Method method : RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsAdapter.class.getDeclaredMethods()) { - if (walRelatedMethods.contains(method.getName())) { - method.invoke(adapter, getDBOptionsParameters(method.getParameterTypes())); + try (RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsAdapter adapter = + new RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsAdapter(new DBOptions(), new ColumnFamilyOptions())) { + for (final Method method : RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsAdapter.class.getDeclaredMethods()) { + if (walRelatedMethods.contains(method.getName())) { + method.invoke(adapter, getDBOptionsParameters(method.getParameterTypes())); + } } - } - final List walOptions = Arrays.asList("walDir", "walFilter", "walRecoveryMode", "walBytesPerSync", "walSizeLimitMB", "manualWalFlush", "maxTotalWalSize", "walTtlSeconds"); + final List walOptions = Arrays.asList("walDir", "walFilter", "walRecoveryMode", "walBytesPerSync", "walSizeLimitMB", "manualWalFlush", "maxTotalWalSize", "walTtlSeconds"); - final Set logMessages = appender.getEvents().stream() - .filter(e -> e.getLevel().equals("WARN")) - .map(LogCaptureAppender.Event::getMessage) - .collect(Collectors.toSet()); - - walOptions.forEach(option -> assertThat(logMessages, hasItem(String.format("WAL is explicitly disabled by Streams in RocksDB. Setting option '%s' will be ignored", option)))); + final Set logMessages = appender.getEvents().stream() + .filter(e -> e.getLevel().equals("WARN")) + .map(LogCaptureAppender.Event::getMessage) + .collect(Collectors.toSet()); + walOptions.forEach(option -> assertThat(logMessages, hasItem(String.format("WAL is explicitly disabled by Streams in RocksDB. Setting option '%s' will be ignored", option)))); + } } } } diff --git a/tests/kafkatest/services/kafka/kafka.py b/tests/kafkatest/services/kafka/kafka.py index 53fcb6f528e9c..e354e9db66abc 100644 --- a/tests/kafkatest/services/kafka/kafka.py +++ b/tests/kafkatest/services/kafka/kafka.py @@ -1445,7 +1445,7 @@ def execute_reassign_partitions(self, reassignment, node=None, self.logger.debug(output) def search_data_files(self, topic, messages): - """Check if a set of messages made it into the Kakfa data files. Note that + """Check if a set of messages made it into the Kafka data files. Note that this method takes no account of replication. It simply looks for the payload in all the partition files of the specified topic. 'messages' should be an array of numbers. The list of missing messages is returned. diff --git a/tools/src/test/java/org/apache/kafka/tools/MetadataQuorumCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/MetadataQuorumCommandTest.java index 7b52b351afd48..bd4a7a0c07c38 100644 --- a/tools/src/test/java/org/apache/kafka/tools/MetadataQuorumCommandTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/MetadataQuorumCommandTest.java @@ -53,12 +53,12 @@ public MetadataQuorumCommandTest(ClusterInstance cluster) { * 3. Fewer brokers than controllers */ @ClusterTests({ - @ClusterTest(clusterType = Type.CO_KRAFT, brokers = 3, controllers = 3), - @ClusterTest(clusterType = Type.KRAFT, brokers = 3, controllers = 3), - @ClusterTest(clusterType = Type.CO_KRAFT, brokers = 3, controllers = 2), - @ClusterTest(clusterType = Type.KRAFT, brokers = 3, controllers = 2), - @ClusterTest(clusterType = Type.CO_KRAFT, brokers = 2, controllers = 3), - @ClusterTest(clusterType = Type.KRAFT, brokers = 2, controllers = 3) + @ClusterTest(clusterType = Type.CO_KRAFT, brokers = 2, controllers = 2), + @ClusterTest(clusterType = Type.KRAFT, brokers = 2, controllers = 2), + @ClusterTest(clusterType = Type.CO_KRAFT, brokers = 2, controllers = 1), + @ClusterTest(clusterType = Type.KRAFT, brokers = 2, controllers = 1), + @ClusterTest(clusterType = Type.CO_KRAFT, brokers = 1, controllers = 2), + @ClusterTest(clusterType = Type.KRAFT, brokers = 1, controllers = 2) }) public void testDescribeQuorumReplicationSuccessful() throws InterruptedException { cluster.waitForReadyBrokers(); @@ -94,12 +94,12 @@ public void testDescribeQuorumReplicationSuccessful() throws InterruptedExceptio * 3. Fewer brokers than controllers */ @ClusterTests({ - @ClusterTest(clusterType = Type.CO_KRAFT, brokers = 3, controllers = 3), - @ClusterTest(clusterType = Type.KRAFT, brokers = 3, controllers = 3), - @ClusterTest(clusterType = Type.CO_KRAFT, brokers = 3, controllers = 2), - @ClusterTest(clusterType = Type.KRAFT, brokers = 3, controllers = 2), - @ClusterTest(clusterType = Type.CO_KRAFT, brokers = 2, controllers = 3), - @ClusterTest(clusterType = Type.KRAFT, brokers = 2, controllers = 3) + @ClusterTest(clusterType = Type.CO_KRAFT, brokers = 2, controllers = 2), + @ClusterTest(clusterType = Type.KRAFT, brokers = 2, controllers = 2), + @ClusterTest(clusterType = Type.CO_KRAFT, brokers = 2, controllers = 1), + @ClusterTest(clusterType = Type.KRAFT, brokers = 2, controllers = 1), + @ClusterTest(clusterType = Type.CO_KRAFT, brokers = 1, controllers = 2), + @ClusterTest(clusterType = Type.KRAFT, brokers = 1, controllers = 2) }) public void testDescribeQuorumStatusSuccessful() throws InterruptedException { cluster.waitForReadyBrokers(); @@ -141,7 +141,7 @@ public void testOnlyOneBrokerAndOneController() { assertEquals("0", replicationOutput.split("\n")[1].split("\\s+")[2]); } - @ClusterTest(clusterType = Type.ZK, brokers = 3) + @ClusterTest(clusterType = Type.ZK, brokers = 1) public void testDescribeQuorumInZkMode() { assertTrue( assertThrows( From c9fb0e63778ecd73946561945ff92f0842d846c4 Mon Sep 17 00:00:00 2001 From: David Arthur Date: Tue, 27 Dec 2022 13:25:29 -0500 Subject: [PATCH 16/30] Change the state machine a bit, get system test working --- checkstyle/import-control.xml | 1 + .../KRaftControllerToZkBrokersRpcClient.scala | 17 ++++ .../main/scala/kafka/server/KafkaServer.scala | 31 +++--- .../migration/KRaftMigrationDriver.java | 22 +++-- .../metadata/migration/MigrationState.java | 2 +- .../org/apache/kafka/queue/EventQueue.java | 4 +- .../tests/core/zookeeper_migration_test.py | 98 ++++++++++++++----- 7 files changed, 125 insertions(+), 50 deletions(-) diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index 32f13d0e34004..53b81c3022222 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -279,6 +279,7 @@ + diff --git a/core/src/main/scala/kafka/migration/KRaftControllerToZkBrokersRpcClient.scala b/core/src/main/scala/kafka/migration/KRaftControllerToZkBrokersRpcClient.scala index 9ec56cc57c4ac..9cb362bc855d6 100644 --- a/core/src/main/scala/kafka/migration/KRaftControllerToZkBrokersRpcClient.scala +++ b/core/src/main/scala/kafka/migration/KRaftControllerToZkBrokersRpcClient.scala @@ -1,3 +1,20 @@ +/* + * 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.migration import kafka.api.LeaderAndIsr diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 6423f6390ee6f..2645448e2f167 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -357,16 +357,25 @@ class KafkaServer( val brokerInfo = createBrokerInfo val brokerEpoch = zkClient.registerBroker(brokerInfo) - lifecycleManager = new BrokerLifecycleManager(config, - time, - threadNamePrefix, - zkBrokerEpochSupplier = Some(() => kafkaController.brokerEpoch)) - // Now that the broker is successfully registered, checkpoint its metadata val zkMetaProperties = ZkMetaProperties(clusterId, config.brokerId) checkpointBrokerMetadata(zkMetaProperties) + /* start token manager */ + tokenManager = new DelegationTokenManager(config, tokenCache, time , zkClient) + tokenManager.startup() + + /* start kafka controller */ + _kafkaController = new KafkaController(config, zkClient, time, metrics, brokerInfo, brokerEpoch, tokenManager, brokerFeatures, metadataCache, threadNamePrefix) + kafkaController.startup() + if (config.migrationEnabled) { + logger.info("Starting up additional components for ZooKeeper migration") + lifecycleManager = new BrokerLifecycleManager(config, + time, + threadNamePrefix, + zkBrokerEpochSupplier = Some(() => kafkaController.brokerEpoch)) + // If the ZK broker is in migration mode, start up a RaftManager to learn about the new KRaft controller val kraftMetaProps = MetaProperties(zkMetaProperties.clusterId, zkMetaProperties.brokerId) val controllerQuorumVotersFuture = CompletableFuture.completedFuture( @@ -396,6 +405,7 @@ class KafkaServer( val listener = new OffsetTrackingListener() raftManager.register(listener) + raftManager.startup() val networkListeners = new ListenerCollection() config.effectiveAdvertisedListeners.foreach { ep => @@ -419,18 +429,9 @@ class KafkaServer( networkListeners, ibpAsFeature ) - - raftManager.startup() + logger.debug("Start RaftManager") } - /* start token manager */ - tokenManager = new DelegationTokenManager(config, tokenCache, time , zkClient) - tokenManager.startup() - - /* start kafka controller */ - _kafkaController = new KafkaController(config, zkClient, time, metrics, brokerInfo, brokerEpoch, tokenManager, brokerFeatures, metadataCache, threadNamePrefix) - kafkaController.startup() - adminManager = new ZkAdminManager(config, metrics, metadataCache, zkClient) /* start group coordinator */ diff --git a/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java b/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java index d844dc7c641a0..adcc9a32ce75d 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java @@ -32,7 +32,9 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.*; +import java.util.Collections; +import java.util.HashSet; +import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.atomic.AtomicInteger; @@ -145,18 +147,18 @@ private boolean isValidStateChange(MigrationState newState) { case INACTIVE: return newState == MigrationState.WAIT_FOR_CONTROLLER_QUORUM; case WAIT_FOR_CONTROLLER_QUORUM: + return + newState == MigrationState.INACTIVE || + newState == MigrationState.WAIT_FOR_BROKERS; + case WAIT_FOR_BROKERS: return newState == MigrationState.INACTIVE || newState == MigrationState.BECOME_CONTROLLER; case BECOME_CONTROLLER: return newState == MigrationState.INACTIVE || - newState == MigrationState.WAIT_FOR_BROKERS || + newState == MigrationState.ZK_MIGRATION || newState == MigrationState.KRAFT_CONTROLLER_TO_BROKER_COMM; - case WAIT_FOR_BROKERS: - return - newState == MigrationState.INACTIVE || - newState == MigrationState.ZK_MIGRATION; case ZK_MIGRATION: return newState == MigrationState.INACTIVE || @@ -228,6 +230,8 @@ public void handleException(Throwable e) { class PollEvent extends MigrationEvent { @Override public void run() throws Exception { + log.info("Poll"); + switch (migrationState) { case UNINITIALIZED: initializeMigrationState(); @@ -323,7 +327,7 @@ public void run() throws Exception { log.debug("Controller Quorum is ready for Zk to KRaft migration"); // Note that leadership would not change here. Hence we do not need to // `apply` any leadership state change. - transitionTo(MigrationState.BECOME_CONTROLLER); + transitionTo(MigrationState.WAIT_FOR_BROKERS); } break; default: @@ -349,7 +353,7 @@ public void run() throws Exception { // We could not claim leadership, stay in BECOME_CONTROLLER to retry } else { if (!migrationLeadershipState.zkMigrationComplete()) { - transitionTo(MigrationState.WAIT_FOR_BROKERS); + transitionTo(MigrationState.ZK_MIGRATION); } else { transitionTo(MigrationState.KRAFT_CONTROLLER_TO_BROKER_COMM); } @@ -374,7 +378,7 @@ public void run() throws Exception { case WAIT_FOR_BROKERS: if (areZkBrokersReadyForMigration()) { log.debug("Zk brokers are registered and ready for migration"); - transitionTo(MigrationState.ZK_MIGRATION); + transitionTo(MigrationState.BECOME_CONTROLLER); } break; default: diff --git a/metadata/src/main/java/org/apache/kafka/metadata/migration/MigrationState.java b/metadata/src/main/java/org/apache/kafka/metadata/migration/MigrationState.java index 2935d678f2aa5..5a6d3df24a3f9 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/migration/MigrationState.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/migration/MigrationState.java @@ -38,8 +38,8 @@ public enum MigrationState { UNINITIALIZED(false), // Initial state. INACTIVE(false), // State when not the active controller. WAIT_FOR_CONTROLLER_QUORUM(false), // Ensure all the quorum nodes are ready for migration. + WAIT_FOR_BROKERS(false), // Wait for Zk brokers to be ready for migration. BECOME_CONTROLLER(false), // Become controller for the Zk Brokers. - WAIT_FOR_BROKERS(true), // Wait for Zk brokers to be ready for migration. ZK_MIGRATION(true), // The cluster has satisfied the migration criteria KRAFT_CONTROLLER_TO_BROKER_COMM(true), // First communication from Controller to send full RPCs to the Zk brokers. DUAL_WRITE(true); // The data has been migrated diff --git a/server-common/src/main/java/org/apache/kafka/queue/EventQueue.java b/server-common/src/main/java/org/apache/kafka/queue/EventQueue.java index d0c752e641da9..9fbf1e6dfa8f3 100644 --- a/server-common/src/main/java/org/apache/kafka/queue/EventQueue.java +++ b/server-common/src/main/java/org/apache/kafka/queue/EventQueue.java @@ -42,7 +42,9 @@ interface Event { * scheduled because the event queue has already been closed. * Otherwise, it will be whatever exception was thrown by run(). */ - default void handleException(Throwable e) {} + default void handleException(Throwable e) { + throw new RuntimeException("Unhandled exception in EventQueue when running " + this, e); + } } abstract class FailureLoggingEvent implements Event { diff --git a/tests/kafkatest/tests/core/zookeeper_migration_test.py b/tests/kafkatest/tests/core/zookeeper_migration_test.py index a0ac3b8b68d76..17a17638fa5d7 100644 --- a/tests/kafkatest/tests/core/zookeeper_migration_test.py +++ b/tests/kafkatest/tests/core/zookeeper_migration_test.py @@ -1,18 +1,72 @@ +# 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. + from functools import partial +from ducktape.utils.util import wait_until + +from kafkatest.services.console_consumer import ConsoleConsumer from kafkatest.services.kafka import KafkaService from kafkatest.services.kafka.config_property import CLUSTER_ID from kafkatest.services.kafka.quorum import remote_kraft, ServiceQuorumInfo, zk +from kafkatest.services.verifiable_producer import VerifiableProducer from kafkatest.services.zookeeper import ZookeeperService -from kafkatest.tests.end_to_end import EndToEndTest +from kafkatest.tests.produce_consume_validate import ProduceConsumeValidateTest +from kafkatest.utils import is_int from kafkatest.version import DEV_BRANCH -class TestMigration(EndToEndTest): +class TestMigration(ProduceConsumeValidateTest): def __init__(self, test_context): - super(TestMigration, self).__init__(topic="zk-topic", test_context=test_context) + super(TestMigration, self).__init__(test_context=test_context) + + def setUp(self): + self.topic = "test_topic" + self.partitions = 3 + self.replication_factor = 3 + + # Producer and consumer + self.producer_throughput = 1000 + self.num_producers = 1 + self.num_consumers = 1 + + def wait_until_rejoin(self): + for partition in range(0, self.partitions): + wait_until(lambda: len(self.kafka.isr_idx_list(self.topic, partition)) == self.replication_factor, timeout_sec=60, + backoff_sec=1, err_msg="Replicas did not rejoin the ISR in a reasonable amount of time") + + def do_migration(self): + # Start up KRaft controller in migration mode + remote_quorum = partial(ServiceQuorumInfo, remote_kraft) + controller = KafkaService(self.test_context, num_nodes=1, zk=self.zk, version=DEV_BRANCH, + allow_zk_with_kraft=True, + remote_kafka=self.kafka, + server_prop_overrides=[["zookeeper.connect", self.zk.connect_setting()], + ["zookeeper.metadata.migration.enable", "true"]], + quorum_info_provider=remote_quorum) + controller.start() + + self.logger.info("Restarting ZK brokers in migration mode") + self.kafka.reconfigure_zk_for_migration(controller) - def test_offline_migration(self): + for node in self.kafka.nodes: + self.kafka.stop_node(node) + self.kafka.start_node(node) + self.wait_until_rejoin() + + def test_online_migration(self): zk_quorum = partial(ServiceQuorumInfo, zk) self.zk = ZookeeperService(self.test_context, num_nodes=1, version=DEV_BRANCH) self.kafka = KafkaService(self.test_context, @@ -32,27 +86,23 @@ def test_offline_migration(self): self.zk.create(path="/cluster/id", value=cluster_id_json) self.kafka.start() - # Start up KRaft controller in migration mode - remote_quorum = partial(ServiceQuorumInfo, remote_kraft) - controller = KafkaService(self.test_context, num_nodes=1, zk=self.zk, version=DEV_BRANCH, - allow_zk_with_kraft=True, - remote_kafka=self.kafka, - server_prop_overrides=[["zookeeper.connect", self.zk.connect_setting()], - ["zookeeper.metadata.migration.enable", "true"]], - quorum_info_provider=remote_quorum) - controller.start() - - self.create_producer() - self.producer.start() + topic_cfg = { + "topic": self.topic, + "partitions": self.partitions, + "replication-factor": self.replication_factor, + "configs": {"min.insync.replicas": 2} + } + self.kafka.create_topic(topic_cfg) - self.create_consumer(log_level="DEBUG") - self.consumer.start() + self.producer = VerifiableProducer(self.test_context, self.num_producers, self.kafka, + self.topic, throughput=self.producer_throughput, + message_validator=is_int, + compression_types=["none"], + version=DEV_BRANCH) - self.logger.info("Restarting ZK brokers in migration mode") - self.kafka.stop() - self.kafka.reconfigure_zk_for_migration(controller) - self.kafka.start() + self.consumer = ConsoleConsumer(self.test_context, self.num_consumers, self.kafka, + self.topic, consumer_timeout_ms=30000, + message_validator=is_int, version=DEV_BRANCH) - self.await_startup() - self.run_validation() + self.run_produce_consume_validate(core_test_action=self.do_migration) # self.kafka.reconfigure_zk_as_kraft(controller) From 2f136557901424ff4218a767c744f6813938d433 Mon Sep 17 00:00:00 2001 From: David Arthur Date: Tue, 3 Jan 2023 13:51:32 -0500 Subject: [PATCH 17/30] Progress on end-to-end test --- .../kafka/server/BrokerMetadataCheckpoint.scala | 15 +++++++++++---- .../metadata/migration/KRaftMigrationDriver.java | 2 -- tests/kafkatest/services/kafka/kafka.py | 10 ++++++++++ .../tests/core/zookeeper_migration_test.py | 11 ++++++++++- 4 files changed, 31 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/kafka/server/BrokerMetadataCheckpoint.scala b/core/src/main/scala/kafka/server/BrokerMetadataCheckpoint.scala index 67d27cb052931..dfe29d1ca35ec 100755 --- a/core/src/main/scala/kafka/server/BrokerMetadataCheckpoint.scala +++ b/core/src/main/scala/kafka/server/BrokerMetadataCheckpoint.scala @@ -103,10 +103,16 @@ class RawMetaProperties(val props: Properties = new Properties()) { object MetaProperties { def parse(properties: RawMetaProperties): MetaProperties = { - properties.requireVersion(expectedVersion = 1) val clusterId = require(ClusterIdKey, properties.clusterId) - val nodeId = require(NodeIdKey, properties.nodeId) - new MetaProperties(clusterId, nodeId) + if (properties.version == 1) { + val nodeId = require(NodeIdKey, properties.nodeId) + new MetaProperties(clusterId, nodeId) + } else if (properties.version == 0) { + val brokerId = require(BrokerIdKey, properties.brokerId) + new MetaProperties(clusterId, brokerId) + } else { + throw new RuntimeException(s"Expected version 0 or 1, but got version ${properties.version}") + } } def require[T](key: String, value: Option[T]): T = { @@ -182,7 +188,8 @@ object BrokerMetadataCheckpoint extends Logging { if (brokerMetadataMap.isEmpty) { (new RawMetaProperties(), offlineDirs) } else { - val numDistinctMetaProperties = brokerMetadataMap.values.toSet.size + val parsedProperties = brokerMetadataMap.values.map(props => MetaProperties.parse(new RawMetaProperties(props))) + val numDistinctMetaProperties = parsedProperties.toSet.size if (numDistinctMetaProperties > 1) { val builder = new StringBuilder diff --git a/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java b/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java index adcc9a32ce75d..84da4793722eb 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java @@ -230,8 +230,6 @@ public void handleException(Throwable e) { class PollEvent extends MigrationEvent { @Override public void run() throws Exception { - log.info("Poll"); - switch (migrationState) { case UNINITIALIZED: initializeMigrationState(); diff --git a/tests/kafkatest/services/kafka/kafka.py b/tests/kafkatest/services/kafka/kafka.py index e354e9db66abc..6a497d85ba115 100644 --- a/tests/kafkatest/services/kafka/kafka.py +++ b/tests/kafkatest/services/kafka/kafka.py @@ -441,6 +441,16 @@ def reconfigure_zk_for_migration(self, kraft_quorum): self.port_mappings[kraft_quorum.controller_listener_names] = kraft_quorum.port_mappings.get(kraft_quorum.controller_listener_names) def reconfigure_zk_as_kraft(self, kraft_quorum): + # Remove the configs we set in reconfigure_zk_for_migration + props = [] + for prop in self.server_prop_overrides: + if not prop[0].startswith("controller"): + props.append(prop) + self.server_prop_overrides.clear() + self.server_prop_overrides.extend(props) + del self.port_mappings[kraft_quorum.controller_listener_names] + + # Set the quorum info to remote KRaft self.quorum_info = quorum.ServiceQuorumInfo(quorum.remote_kraft, self) self.remote_controller_quorum = kraft_quorum self.controller_quorum = kraft_quorum diff --git a/tests/kafkatest/tests/core/zookeeper_migration_test.py b/tests/kafkatest/tests/core/zookeeper_migration_test.py index 17a17638fa5d7..257b9f79158e8 100644 --- a/tests/kafkatest/tests/core/zookeeper_migration_test.py +++ b/tests/kafkatest/tests/core/zookeeper_migration_test.py @@ -14,6 +14,7 @@ # limitations under the License. from functools import partial +import time from ducktape.utils.util import wait_until @@ -66,6 +67,15 @@ def do_migration(self): self.kafka.start_node(node) self.wait_until_rejoin() + self.logger.info("Restarting ZK brokers as KRaft brokers") + time.sleep(10) + self.kafka.reconfigure_zk_as_kraft(controller) + + for node in self.kafka.nodes: + self.kafka.stop_node(node) + self.kafka.start_node(node) + self.wait_until_rejoin() + def test_online_migration(self): zk_quorum = partial(ServiceQuorumInfo, zk) self.zk = ZookeeperService(self.test_context, num_nodes=1, version=DEV_BRANCH) @@ -105,4 +115,3 @@ def test_online_migration(self): message_validator=is_int, version=DEV_BRANCH) self.run_produce_consume_validate(core_test_action=self.do_migration) - # self.kafka.reconfigure_zk_as_kraft(controller) From 8167ea5523aeb7f0661f44ca441d858b7208bfbe Mon Sep 17 00:00:00 2001 From: David Arthur Date: Tue, 3 Jan 2023 16:55:59 -0500 Subject: [PATCH 18/30] Use the correct epoch on ZK broker --- .../kafka/server/BrokerEpochManager.scala | 35 +++++++++++++++++++ .../main/scala/kafka/server/KafkaApis.scala | 15 ++++---- .../main/scala/kafka/server/KafkaServer.scala | 13 +++++-- .../scala/kafka/server/MetadataSupport.scala | 13 +++++-- 4 files changed, 63 insertions(+), 13 deletions(-) create mode 100644 core/src/main/scala/kafka/server/BrokerEpochManager.scala diff --git a/core/src/main/scala/kafka/server/BrokerEpochManager.scala b/core/src/main/scala/kafka/server/BrokerEpochManager.scala new file mode 100644 index 0000000000000..d88cad01b0519 --- /dev/null +++ b/core/src/main/scala/kafka/server/BrokerEpochManager.scala @@ -0,0 +1,35 @@ +package kafka.server + +import kafka.controller.KafkaController +import org.apache.kafka.common.requests.AbstractControlRequest + +class BrokerEpochManager(metadataCache: MetadataCache, + controller: KafkaController, + lifecycleManagerOpt: Option[BrokerLifecycleManager]) { + def get(): Long = { + lifecycleManagerOpt match { + case Some(lifecycleManager) => metadataCache.getControllerId match { + case Some(_: ZkCachedControllerId) => controller.brokerEpoch + case Some(_: KRaftCachedControllerId) => lifecycleManager.brokerEpoch + case None | _ => controller.brokerEpoch + } + case None => controller.brokerEpoch + } + } + + def isBrokerEpochStale(brokerEpochInRequest: Long, isKRaftControllerRequest: Boolean): Boolean = { + if (brokerEpochInRequest == AbstractControlRequest.UNKNOWN_BROKER_EPOCH) { + false + } else if (isKRaftControllerRequest) { + if (lifecycleManagerOpt.isDefined) { + brokerEpochInRequest < lifecycleManagerOpt.get.brokerEpoch + } else { + throw new IllegalStateException("Expected BrokerLifecycleManager to not be null.") + } + } else { + // brokerEpochInRequest > controller.brokerEpoch is possible in rare scenarios where the controller gets notified + // about the new broker epoch and sends a control request with this epoch before the broker learns about it + brokerEpochInRequest < controller.brokerEpoch + } + } +} diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 662b61a5fd39a..a20cc134290e1 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -109,7 +109,8 @@ class KafkaApis(val requestChannel: RequestChannel, val clusterId: String, time: Time, val tokenManager: DelegationTokenManager, - val apiVersionManager: ApiVersionManager + val apiVersionManager: ApiVersionManager, + val brokerEpochManager: BrokerEpochManager ) extends ApiRequestHandler with Logging { type FetchResponseStats = Map[TopicPartition, RecordConversionStats] @@ -268,7 +269,7 @@ class KafkaApis(val requestChannel: RequestChannel, val leaderAndIsrRequest = request.body[LeaderAndIsrRequest] authHelper.authorizeClusterOperation(request, CLUSTER_ACTION) - if (isBrokerEpochStale(zkSupport, leaderAndIsrRequest.brokerEpoch)) { + if (brokerEpochManager.isBrokerEpochStale(leaderAndIsrRequest.brokerEpoch, leaderAndIsrRequest.isKRaftController)) { // When the broker restarts very quickly, it is possible for this broker to receive request intended // for its previous generation so the broker should skip the stale request. info(s"Received LeaderAndIsr request with broker epoch ${leaderAndIsrRequest.brokerEpoch} " + @@ -289,7 +290,7 @@ class KafkaApis(val requestChannel: RequestChannel, // stop serving data to clients for the topic being deleted val stopReplicaRequest = request.body[StopReplicaRequest] authHelper.authorizeClusterOperation(request, CLUSTER_ACTION) - if (isBrokerEpochStale(zkSupport, stopReplicaRequest.brokerEpoch)) { + if (brokerEpochManager.isBrokerEpochStale(stopReplicaRequest.brokerEpoch, stopReplicaRequest.isKRaftController)) { // When the broker restarts very quickly, it is possible for this broker to receive request intended // for its previous generation so the broker should skip the stale request. info(s"Received StopReplica request with broker epoch ${stopReplicaRequest.brokerEpoch} " + @@ -349,7 +350,7 @@ class KafkaApis(val requestChannel: RequestChannel, val updateMetadataRequest = request.body[UpdateMetadataRequest] authHelper.authorizeClusterOperation(request, CLUSTER_ACTION) - if (isBrokerEpochStale(zkSupport, updateMetadataRequest.brokerEpoch)) { + if (brokerEpochManager.isBrokerEpochStale(updateMetadataRequest.brokerEpoch, updateMetadataRequest.isKRaftController)) { // When the broker restarts very quickly, it is possible for this broker to receive request intended // for its previous generation so the broker should skip the stale request. info(s"Received UpdateMetadata request with broker epoch ${updateMetadataRequest.brokerEpoch} " + @@ -3529,14 +3530,12 @@ class KafkaApis(val requestChannel: RequestChannel, request.temporaryMemoryBytes = conversionStats.temporaryMemoryBytes } - private def isBrokerEpochStale(zkSupport: ZkSupport, brokerEpochInRequest: Long): Boolean = { + private def isBrokerEpochStale(zkSupport: ZkSupport, brokerEpochInRequest: Long, isFromKRaftController: Boolean): Boolean = { // Broker epoch in LeaderAndIsr/UpdateMetadata/StopReplica request is unknown // if the controller hasn't been upgraded to use KIP-380 if (brokerEpochInRequest == AbstractControlRequest.UNKNOWN_BROKER_EPOCH) false else { - // brokerEpochInRequest > controller.brokerEpoch is possible in rare scenarios where the controller gets notified - // about the new broker epoch and sends a control request with this epoch before the broker learns about it - brokerEpochInRequest < zkSupport.controller.brokerEpoch + brokerEpochManager.isBrokerEpochStale(brokerEpochInRequest, isFromKRaftController) } } } diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 2645448e2f167..157e95a73b1d9 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -192,6 +192,8 @@ class KafkaServer( var lifecycleManager: BrokerLifecycleManager = _ + var brokerEpochManager: BrokerEpochManager = _ + /** * Start up API for bringing up a single instance of the Kafka server. * Instantiates the LogManager, the SocketServer and the request handlers - KafkaRequestHandlers @@ -343,7 +345,7 @@ class KafkaServer( time = time, metrics = metrics, threadNamePrefix = threadNamePrefix, - brokerEpochSupplier = () => kafkaController.brokerEpoch + brokerEpochSupplier = () => brokerEpochManager.get() ) } else { AlterPartitionManager(kafkaScheduler, time, zkClient) @@ -432,6 +434,10 @@ class KafkaServer( logger.debug("Start RaftManager") } + // Used by ZK brokers during a KRaft migration. When talking to a KRaft controller, we need to use the epoch + // from BrokerLifecycleManager rather than ZK (via KafkaController) + brokerEpochManager = new BrokerEpochManager(metadataCache, kafkaController, Option(lifecycleManager)) + adminManager = new ZkAdminManager(config, metrics, metadataCache, zkClient) /* start group coordinator */ @@ -443,7 +449,7 @@ class KafkaServer( val producerIdManager = if (config.interBrokerProtocolVersion.isAllocateProducerIdsSupported) { ProducerIdManager.rpc( config.brokerId, - brokerEpochSupplier = () => kafkaController.brokerEpoch, + brokerEpochSupplier = () => brokerEpochManager.get(), clientToControllerChannelManager, config.requestTimeoutMs ) @@ -513,7 +519,8 @@ class KafkaServer( clusterId = clusterId, time = time, tokenManager = tokenManager, - apiVersionManager = apiVersionManager) + apiVersionManager = apiVersionManager, + brokerEpochManager = brokerEpochManager) dataPlaneRequestProcessor = createKafkaApis(socketServer.dataPlaneRequestChannel) diff --git a/core/src/main/scala/kafka/server/MetadataSupport.scala b/core/src/main/scala/kafka/server/MetadataSupport.scala index 41ae2a620ffdc..f0ec3d74b858c 100644 --- a/core/src/main/scala/kafka/server/MetadataSupport.scala +++ b/core/src/main/scala/kafka/server/MetadataSupport.scala @@ -69,13 +69,16 @@ sealed trait MetadataSupport { handler(request) } } + + def brokerEpoch(): Long } case class ZkSupport(adminManager: ZkAdminManager, controller: KafkaController, zkClient: KafkaZkClient, forwardingManager: Option[ForwardingManager], - metadataCache: ZkMetadataCache) extends MetadataSupport { + metadataCache: ZkMetadataCache, + brokerEpochSupplier: () => Long) extends MetadataSupport { override def requireZkOrThrow(createException: => Exception): ZkSupport = this override def requireRaftOrThrow(createException: => Exception): RaftSupport = throw createException @@ -86,9 +89,13 @@ case class ZkSupport(adminManager: ZkAdminManager, } override def canForward(): Boolean = forwardingManager.isDefined && (!controller.isActive) + + override def brokerEpoch(): Long = brokerEpochSupplier.apply() } -case class RaftSupport(fwdMgr: ForwardingManager, metadataCache: KRaftMetadataCache) +case class RaftSupport(fwdMgr: ForwardingManager, + metadataCache: KRaftMetadataCache, + brokerEpochSupplier: () => Long) extends MetadataSupport { override val forwardingManager: Option[ForwardingManager] = Some(fwdMgr) override def requireZkOrThrow(createException: => Exception): ZkSupport = throw createException @@ -101,4 +108,6 @@ case class RaftSupport(fwdMgr: ForwardingManager, metadataCache: KRaftMetadataCa } override def canForward(): Boolean = true + + override def brokerEpoch(): Long = brokerEpochSupplier.apply() } From 4660527103d1f6922e2a7d509ef57cf3a078c88a Mon Sep 17 00:00:00 2001 From: David Arthur Date: Wed, 4 Jan 2023 12:19:36 -0500 Subject: [PATCH 19/30] wip on epoch --- .../main/scala/kafka/server/KafkaApis.scala | 20 +++++-------------- .../main/scala/kafka/server/KafkaServer.scala | 5 ++--- .../scala/kafka/server/MetadataSupport.scala | 14 ++++++------- .../unit/kafka/server/KafkaApisTest.scala | 5 +++-- 4 files changed, 16 insertions(+), 28 deletions(-) diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index a20cc134290e1..4ec7a7d9a6cf1 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -109,8 +109,7 @@ class KafkaApis(val requestChannel: RequestChannel, val clusterId: String, time: Time, val tokenManager: DelegationTokenManager, - val apiVersionManager: ApiVersionManager, - val brokerEpochManager: BrokerEpochManager + val apiVersionManager: ApiVersionManager ) extends ApiRequestHandler with Logging { type FetchResponseStats = Map[TopicPartition, RecordConversionStats] @@ -269,7 +268,7 @@ class KafkaApis(val requestChannel: RequestChannel, val leaderAndIsrRequest = request.body[LeaderAndIsrRequest] authHelper.authorizeClusterOperation(request, CLUSTER_ACTION) - if (brokerEpochManager.isBrokerEpochStale(leaderAndIsrRequest.brokerEpoch, leaderAndIsrRequest.isKRaftController)) { + if (zkSupport.isBrokerEpochStale(leaderAndIsrRequest.brokerEpoch, leaderAndIsrRequest.isKRaftController)) { // When the broker restarts very quickly, it is possible for this broker to receive request intended // for its previous generation so the broker should skip the stale request. info(s"Received LeaderAndIsr request with broker epoch ${leaderAndIsrRequest.brokerEpoch} " + @@ -290,7 +289,7 @@ class KafkaApis(val requestChannel: RequestChannel, // stop serving data to clients for the topic being deleted val stopReplicaRequest = request.body[StopReplicaRequest] authHelper.authorizeClusterOperation(request, CLUSTER_ACTION) - if (brokerEpochManager.isBrokerEpochStale(stopReplicaRequest.brokerEpoch, stopReplicaRequest.isKRaftController)) { + if (zkSupport.isBrokerEpochStale(stopReplicaRequest.brokerEpoch, stopReplicaRequest.isKRaftController)) { // When the broker restarts very quickly, it is possible for this broker to receive request intended // for its previous generation so the broker should skip the stale request. info(s"Received StopReplica request with broker epoch ${stopReplicaRequest.brokerEpoch} " + @@ -350,7 +349,7 @@ class KafkaApis(val requestChannel: RequestChannel, val updateMetadataRequest = request.body[UpdateMetadataRequest] authHelper.authorizeClusterOperation(request, CLUSTER_ACTION) - if (brokerEpochManager.isBrokerEpochStale(updateMetadataRequest.brokerEpoch, updateMetadataRequest.isKRaftController)) { + if (zkSupport.isBrokerEpochStale(updateMetadataRequest.brokerEpoch, updateMetadataRequest.isKRaftController)) { // When the broker restarts very quickly, it is possible for this broker to receive request intended // for its previous generation so the broker should skip the stale request. info(s"Received UpdateMetadata request with broker epoch ${updateMetadataRequest.brokerEpoch} " + @@ -3177,7 +3176,7 @@ class KafkaApis(val requestChannel: RequestChannel, describeClientQuotasRequest.getErrorResponse(requestThrottleMs, Errors.CLUSTER_AUTHORIZATION_FAILED.exception)) } else { metadataSupport match { - case ZkSupport(adminManager, controller, zkClient, forwardingManager, metadataCache) => + case ZkSupport(adminManager, controller, zkClient, forwardingManager, metadataCache, _) => val result = adminManager.describeClientQuotas(describeClientQuotasRequest.filter) val entriesData = result.iterator.map { case (quotaEntity, quotaValues) => @@ -3529,15 +3528,6 @@ class KafkaApis(val requestChannel: RequestChannel, } request.temporaryMemoryBytes = conversionStats.temporaryMemoryBytes } - - private def isBrokerEpochStale(zkSupport: ZkSupport, brokerEpochInRequest: Long, isFromKRaftController: Boolean): Boolean = { - // Broker epoch in LeaderAndIsr/UpdateMetadata/StopReplica request is unknown - // if the controller hasn't been upgraded to use KIP-380 - if (brokerEpochInRequest == AbstractControlRequest.UNKNOWN_BROKER_EPOCH) false - else { - brokerEpochManager.isBrokerEpochStale(brokerEpochInRequest, isFromKRaftController) - } - } } object KafkaApis { diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 157e95a73b1d9..a6b5ce3ea69ec 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -497,7 +497,7 @@ class KafkaServer( remoteLogManager.foreach(_.startup()) /* start processing requests */ - val zkSupport = ZkSupport(adminManager, kafkaController, zkClient, forwardingManager, metadataCache) + val zkSupport = ZkSupport(adminManager, kafkaController, zkClient, forwardingManager, metadataCache, brokerEpochManager) def createKafkaApis(requestChannel: RequestChannel): KafkaApis = new KafkaApis( requestChannel = requestChannel, @@ -519,8 +519,7 @@ class KafkaServer( clusterId = clusterId, time = time, tokenManager = tokenManager, - apiVersionManager = apiVersionManager, - brokerEpochManager = brokerEpochManager) + apiVersionManager = apiVersionManager) dataPlaneRequestProcessor = createKafkaApis(socketServer.dataPlaneRequestChannel) diff --git a/core/src/main/scala/kafka/server/MetadataSupport.scala b/core/src/main/scala/kafka/server/MetadataSupport.scala index f0ec3d74b858c..eb8cfae82e5a2 100644 --- a/core/src/main/scala/kafka/server/MetadataSupport.scala +++ b/core/src/main/scala/kafka/server/MetadataSupport.scala @@ -69,8 +69,6 @@ sealed trait MetadataSupport { handler(request) } } - - def brokerEpoch(): Long } case class ZkSupport(adminManager: ZkAdminManager, @@ -78,8 +76,9 @@ case class ZkSupport(adminManager: ZkAdminManager, zkClient: KafkaZkClient, forwardingManager: Option[ForwardingManager], metadataCache: ZkMetadataCache, - brokerEpochSupplier: () => Long) extends MetadataSupport { + brokerEpochManager: BrokerEpochManager) extends MetadataSupport { override def requireZkOrThrow(createException: => Exception): ZkSupport = this + override def requireRaftOrThrow(createException: => Exception): RaftSupport = throw createException override def ensureConsistentWith(config: KafkaConfig): Unit = { @@ -90,12 +89,13 @@ case class ZkSupport(adminManager: ZkAdminManager, override def canForward(): Boolean = forwardingManager.isDefined && (!controller.isActive) - override def brokerEpoch(): Long = brokerEpochSupplier.apply() + def isBrokerEpochStale(brokerEpochInRequest: Long, isKRaftControllerRequest: Boolean): Boolean = { + brokerEpochManager.isBrokerEpochStale(brokerEpochInRequest, isKRaftControllerRequest) + } } case class RaftSupport(fwdMgr: ForwardingManager, - metadataCache: KRaftMetadataCache, - brokerEpochSupplier: () => Long) + metadataCache: KRaftMetadataCache) extends MetadataSupport { override val forwardingManager: Option[ForwardingManager] = Some(fwdMgr) override def requireZkOrThrow(createException: => Exception): ZkSupport = throw createException @@ -108,6 +108,4 @@ case class RaftSupport(fwdMgr: ForwardingManager, } override def canForward(): Boolean = true - - override def brokerEpoch(): Long = brokerEpochSupplier.apply() } diff --git a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala index 4b039b5543f7e..0bc82f984c7ed 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala @@ -106,6 +106,7 @@ class KafkaApisTest { private val controller: KafkaController = mock(classOf[KafkaController]) private val forwardingManager: ForwardingManager = mock(classOf[ForwardingManager]) private val autoTopicCreationManager: AutoTopicCreationManager = mock(classOf[AutoTopicCreationManager]) + private val brokerEpochManager: BrokerEpochManager = mock(classOf[BrokerEpochManager]) private val kafkaPrincipalSerde = new KafkaPrincipalSerde { override def serialize(principal: KafkaPrincipal): Array[Byte] = Utils.utf8(principal.toString) @@ -171,7 +172,7 @@ class KafkaApisTest { } else { metadataCache match { case zkMetadataCache: ZkMetadataCache => - ZkSupport(adminManager, controller, zkClient, forwardingManagerOpt, zkMetadataCache) + ZkSupport(adminManager, controller, zkClient, forwardingManagerOpt, zkMetadataCache, brokerEpochManager) case _ => throw new IllegalStateException("Test must set an instance of ZkMetadataCache") } } @@ -185,8 +186,8 @@ class KafkaApisTest { val apiVersionManager = new SimpleApiVersionManager(listenerType, enabledApis, BrokerFeatures.defaultSupportedFeatures()) new KafkaApis( - metadataSupport = metadataSupport, requestChannel = requestChannel, + metadataSupport = metadataSupport, replicaManager = replicaManager, groupCoordinator = groupCoordinator, newGroupCoordinator = newGroupCoordinator, From 4dbb5a3f6d8f581251bdd5be69044f50735c8870 Mon Sep 17 00:00:00 2001 From: David Arthur Date: Wed, 4 Jan 2023 18:26:50 -0500 Subject: [PATCH 20/30] WIP on end-to-end test --- .../KRaftControllerToZkBrokersRpcClient.scala | 2 +- .../main/scala/kafka/server/BrokerEpochManager.scala | 2 +- core/src/main/scala/kafka/server/KafkaServer.scala | 11 ++++++++++- .../java/org/apache/kafka/image/ClusterImage.java | 2 +- 4 files changed, 13 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/kafka/migration/KRaftControllerToZkBrokersRpcClient.scala b/core/src/main/scala/kafka/migration/KRaftControllerToZkBrokersRpcClient.scala index 9cb362bc855d6..eb14c62dcc950 100644 --- a/core/src/main/scala/kafka/migration/KRaftControllerToZkBrokersRpcClient.scala +++ b/core/src/main/scala/kafka/migration/KRaftControllerToZkBrokersRpcClient.scala @@ -78,7 +78,7 @@ sealed class KRaftControllerBrokerRequestMetadata(val image: MetadataImage) exte override val liveBrokerIdAndEpochs: collection.Map[Int, Long] = { image.cluster().zkBrokers().asScala.map { - case (brokerId, broker) => brokerId.intValue() -> broker.migratingZkBrokerEpoch().orElse(-1L) + case (brokerId, broker) => brokerId.intValue() -> broker.epoch() } } diff --git a/core/src/main/scala/kafka/server/BrokerEpochManager.scala b/core/src/main/scala/kafka/server/BrokerEpochManager.scala index d88cad01b0519..09cefe036759e 100644 --- a/core/src/main/scala/kafka/server/BrokerEpochManager.scala +++ b/core/src/main/scala/kafka/server/BrokerEpochManager.scala @@ -24,7 +24,7 @@ class BrokerEpochManager(metadataCache: MetadataCache, if (lifecycleManagerOpt.isDefined) { brokerEpochInRequest < lifecycleManagerOpt.get.brokerEpoch } else { - throw new IllegalStateException("Expected BrokerLifecycleManager to not be null.") + throw new IllegalStateException("Expected BrokerLifecycleManager to be non-null.") } } else { // brokerEpochInRequest > controller.brokerEpoch is possible in rare scenarios where the controller gets notified diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index a6b5ce3ea69ec..b211019ed32a3 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -662,7 +662,7 @@ class KafkaServer( metadataCache.getControllerId.exists(_.isInstanceOf[KRaftCachedControllerId])) { info("ZkBroker currently has a KRaft controller. Controlled shutdown will be handled " + "through broker life cycle manager") - return false + return true } val metadataUpdater = new ManualMetadataUpdater() val networkClient = { @@ -815,6 +815,15 @@ class KafkaServer( // shutting down without waiting for the heartbeat to time out. info("Notifying KRaft of controlled shutdown") lifecycleManager.beginControlledShutdown() + try { + lifecycleManager.controlledShutdownFuture.get(5L, TimeUnit.MINUTES) + } catch { + case _: TimeoutException => + error("Timed out waiting for the controller to approve controlled shutdown") + case e: Throwable => + error("Got unexpected exception waiting for controlled shutdown future", e) + } + // TODO fix this ^ } val shutdownSucceeded = doControlledShutdown(config.controlledShutdownMaxRetries.intValue) diff --git a/metadata/src/main/java/org/apache/kafka/image/ClusterImage.java b/metadata/src/main/java/org/apache/kafka/image/ClusterImage.java index b59fa15cfffa1..b93eacd4625b9 100644 --- a/metadata/src/main/java/org/apache/kafka/image/ClusterImage.java +++ b/metadata/src/main/java/org/apache/kafka/image/ClusterImage.java @@ -41,7 +41,7 @@ public ClusterImage(Map brokers) { this.zkBrokers = Collections.unmodifiableMap(brokers .entrySet() .stream() - .filter(x -> x.getValue().isMigratingZkBroker()) + .filter(entry -> entry.getValue().isMigratingZkBroker()) .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue))); } From dacff92e86d4e87554d4ce2581b9e0e9a857e27e Mon Sep 17 00:00:00 2001 From: David Arthur Date: Thu, 5 Jan 2023 12:49:28 -0500 Subject: [PATCH 21/30] end to end test is passing --- .../KRaftControllerToZkBrokersRpcClient.scala | 4 ++-- tests/kafkatest/services/kafka/kafka.py | 15 +++++++++++++-- 2 files changed, 15 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/kafka/migration/KRaftControllerToZkBrokersRpcClient.scala b/core/src/main/scala/kafka/migration/KRaftControllerToZkBrokersRpcClient.scala index eb14c62dcc950..1409362270f91 100644 --- a/core/src/main/scala/kafka/migration/KRaftControllerToZkBrokersRpcClient.scala +++ b/core/src/main/scala/kafka/migration/KRaftControllerToZkBrokersRpcClient.scala @@ -77,13 +77,13 @@ sealed class KRaftControllerBrokerRequestMetadata(val image: MetadataImage) exte } override val liveBrokerIdAndEpochs: collection.Map[Int, Long] = { - image.cluster().zkBrokers().asScala.map { + image.cluster().brokers().asScala.map { case (brokerId, broker) => brokerId.intValue() -> broker.epoch() } } override val liveOrShuttingDownBrokers: collection.Set[Broker] = { - image.cluster().zkBrokers().asScala.values.map { registration => + image.cluster().brokers().asScala.values.map { registration => Broker.fromBrokerRegistration(registration) }.toSet } diff --git a/tests/kafkatest/services/kafka/kafka.py b/tests/kafkatest/services/kafka/kafka.py index 6a497d85ba115..064793a8c702b 100644 --- a/tests/kafkatest/services/kafka/kafka.py +++ b/tests/kafkatest/services/kafka/kafka.py @@ -275,6 +275,7 @@ def __init__(self, context, num_nodes, zk, security_protocol=SecurityConfig.PLAI self.quorum_info = quorum_info_provider(self) self.controller_quorum = None # will define below if necessary self.remote_controller_quorum = None # will define below if necessary + self.configured_for_zk_migration = False if num_nodes < 1: raise Exception("Must set a positive number of nodes: %i" % num_nodes) @@ -432,15 +433,23 @@ def __init__(self, context, num_nodes, zk, security_protocol=SecurityConfig.PLAI self.nodes_to_start = self.nodes def reconfigure_zk_for_migration(self, kraft_quorum): - #self.remote_controller_quorum = kraft_quorum + self.configured_for_zk_migration = True + self.controller_quorum = kraft_quorum + + # Set the migration properties self.server_prop_overrides.extend([ ["zookeeper.metadata.migration.enable", "true"], ["controller.quorum.voters", kraft_quorum.controller_quorum_voters], ["controller.listener.names", kraft_quorum.controller_listener_names] ]) + + # Add a port mapping for the controller listener. + # This is not added to "advertised.listeners" because of configured_for_zk_migration=True self.port_mappings[kraft_quorum.controller_listener_names] = kraft_quorum.port_mappings.get(kraft_quorum.controller_listener_names) def reconfigure_zk_as_kraft(self, kraft_quorum): + self.configured_for_zk_migration = True + # Remove the configs we set in reconfigure_zk_for_migration props = [] for prop in self.server_prop_overrides: @@ -794,7 +803,9 @@ def start_cmd(self, node): return cmd def controller_listener_name_list(self, node): - if self.quorum_info.using_zk: + if self.quorum_info.using_zk and self.configured_for_zk_migration: + return [self.controller_listener_name(self.controller_quorum.controller_security_protocol)] + elif self.quorum_info.using_zk: return [] broker_to_controller_listener_name = self.controller_listener_name(self.controller_quorum.controller_security_protocol) # Brokers always use the first controller listener, so include a second, inter-controller listener if and only if: From cd5141cb352370a19417d6f9538c53625d9766bb Mon Sep 17 00:00:00 2001 From: Akhilesh Chaganti Date: Thu, 5 Jan 2023 12:15:58 -0800 Subject: [PATCH 22/30] Address comments --- .../message/BrokerRegistrationRequest.json | 1 + .../message/BrokerRegistrationResponse.json | 1 + .../kafka/server/BrokerLifecycleManager.scala | 18 +++++++-- .../scala/kafka/server/BrokerServer.scala | 4 +- .../main/scala/kafka/server/KafkaServer.scala | 13 ++++--- .../scala/kafka/server/MetadataSupport.scala | 2 +- ...nager.scala => ZkBrokerEpochManager.scala} | 6 +-- .../server/BrokerLifecycleManagerTest.scala | 12 +++--- .../unit/kafka/server/KafkaApisTest.scala | 2 +- .../metadata/MetadataRequestBenchmark.java | 4 +- .../controller/MigrationControlManager.java | 2 +- .../metadata/migration/BrokersRpcClient.java | 37 ------------------- 12 files changed, 42 insertions(+), 60 deletions(-) rename core/src/main/scala/kafka/server/{BrokerEpochManager.scala => ZkBrokerEpochManager.scala} (91%) delete mode 100644 metadata/src/main/java/org/apache/kafka/metadata/migration/BrokersRpcClient.java diff --git a/clients/src/main/resources/common/message/BrokerRegistrationRequest.json b/clients/src/main/resources/common/message/BrokerRegistrationRequest.json index 43b126ec0278a..22b0edbc64c54 100644 --- a/clients/src/main/resources/common/message/BrokerRegistrationRequest.json +++ b/clients/src/main/resources/common/message/BrokerRegistrationRequest.json @@ -13,6 +13,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +// Version 1 adds Zk broker epoch to the request if the broker is migrating from Zk mode to KRaft mode. { "apiKey":62, "type": "request", diff --git a/clients/src/main/resources/common/message/BrokerRegistrationResponse.json b/clients/src/main/resources/common/message/BrokerRegistrationResponse.json index 7515d5ee4bc06..7d45b0fecabb8 100644 --- a/clients/src/main/resources/common/message/BrokerRegistrationResponse.json +++ b/clients/src/main/resources/common/message/BrokerRegistrationResponse.json @@ -13,6 +13,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +// Version 1 adds Zk broker epoch to the request if the broker is migrating from Zk mode to KRaft mode. { "apiKey": 62, "type": "response", diff --git a/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala b/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala index 090dd6c03e233..f560a8eafc3b9 100644 --- a/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala +++ b/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala @@ -55,7 +55,8 @@ class BrokerLifecycleManager( val config: KafkaConfig, val time: Time, val threadNamePrefix: Option[String], - val zkBrokerEpochSupplier: Option[() => Long] = None + val isZkBroker: Boolean, + val zkBrokerEpochSupplier: () => Long ) extends Logging { val logContext = new LogContext(s"[BrokerLifecycleManager id=${config.nodeId}] ") @@ -270,7 +271,7 @@ class BrokerLifecycleManager( _clusterId = clusterId _advertisedListeners = advertisedListeners.duplicate() _supportedFeatures = new util.HashMap[String, VersionRange](supportedFeatures) - if (zkBrokerEpochSupplier.isEmpty) { + if (!isZkBroker) { // Only KRaft brokers block on registration during startup eventQueue.scheduleDeferred("initialRegistrationTimeout", new DeadlineFunction(time.nanoseconds() + initialTimeoutNs), @@ -290,9 +291,20 @@ class BrokerLifecycleManager( setMinSupportedVersion(range.min()). setMaxSupportedVersion(range.max())) } + val migrationZkBrokerEpoch: Long = { + if (isZkBroker) { + val zkBrokerEpoch: Long = Option(zkBrokerEpochSupplier).map(_.apply()).getOrElse(-1) + if (zkBrokerEpoch < 0) { + throw new IllegalStateException("Trying to sending BrokerRegistration in migration Zk " + + "broker without valid zk broker epoch") + } + zkBrokerEpoch + } else + -1 + } val data = new BrokerRegistrationRequestData(). setBrokerId(nodeId). - setMigratingZkBrokerEpoch(zkBrokerEpochSupplier.map(_.apply()).getOrElse(-1)). + setMigratingZkBrokerEpoch(migrationZkBrokerEpoch). setClusterId(_clusterId). setFeatures(features). setIncarnationId(incarnationId). diff --git a/core/src/main/scala/kafka/server/BrokerServer.scala b/core/src/main/scala/kafka/server/BrokerServer.scala index cbe700358f6f6..eb8c58c50bf8b 100644 --- a/core/src/main/scala/kafka/server/BrokerServer.scala +++ b/core/src/main/scala/kafka/server/BrokerServer.scala @@ -186,7 +186,9 @@ class BrokerServer( lifecycleManager = new BrokerLifecycleManager(config, time, - threadNamePrefix) + threadNamePrefix, + isZkBroker = false, + () => -1) /* start scheduler */ kafkaScheduler = new KafkaScheduler(config.backgroundThreads) diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index b211019ed32a3..45a9fda1cec11 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -192,7 +192,9 @@ class KafkaServer( var lifecycleManager: BrokerLifecycleManager = _ - var brokerEpochManager: BrokerEpochManager = _ + @volatile var brokerEpochManager: ZkBrokerEpochManager = _ + + def brokerEpochSupplier(): Long = Option(brokerEpochManager).map(_.get()).getOrElse(-1) /** * Start up API for bringing up a single instance of the Kafka server. @@ -345,7 +347,7 @@ class KafkaServer( time = time, metrics = metrics, threadNamePrefix = threadNamePrefix, - brokerEpochSupplier = () => brokerEpochManager.get() + brokerEpochSupplier = brokerEpochSupplier ) } else { AlterPartitionManager(kafkaScheduler, time, zkClient) @@ -376,7 +378,8 @@ class KafkaServer( lifecycleManager = new BrokerLifecycleManager(config, time, threadNamePrefix, - zkBrokerEpochSupplier = Some(() => kafkaController.brokerEpoch)) + isZkBroker = true, + () => kafkaController.brokerEpoch) // If the ZK broker is in migration mode, start up a RaftManager to learn about the new KRaft controller val kraftMetaProps = MetaProperties(zkMetaProperties.clusterId, zkMetaProperties.brokerId) @@ -436,7 +439,7 @@ class KafkaServer( // Used by ZK brokers during a KRaft migration. When talking to a KRaft controller, we need to use the epoch // from BrokerLifecycleManager rather than ZK (via KafkaController) - brokerEpochManager = new BrokerEpochManager(metadataCache, kafkaController, Option(lifecycleManager)) + brokerEpochManager = new ZkBrokerEpochManager(metadataCache, kafkaController, Option(lifecycleManager)) adminManager = new ZkAdminManager(config, metrics, metadataCache, zkClient) @@ -449,7 +452,7 @@ class KafkaServer( val producerIdManager = if (config.interBrokerProtocolVersion.isAllocateProducerIdsSupported) { ProducerIdManager.rpc( config.brokerId, - brokerEpochSupplier = () => brokerEpochManager.get(), + brokerEpochSupplier = brokerEpochSupplier, clientToControllerChannelManager, config.requestTimeoutMs ) diff --git a/core/src/main/scala/kafka/server/MetadataSupport.scala b/core/src/main/scala/kafka/server/MetadataSupport.scala index eb8cfae82e5a2..335df7c42d74c 100644 --- a/core/src/main/scala/kafka/server/MetadataSupport.scala +++ b/core/src/main/scala/kafka/server/MetadataSupport.scala @@ -76,7 +76,7 @@ case class ZkSupport(adminManager: ZkAdminManager, zkClient: KafkaZkClient, forwardingManager: Option[ForwardingManager], metadataCache: ZkMetadataCache, - brokerEpochManager: BrokerEpochManager) extends MetadataSupport { + brokerEpochManager: ZkBrokerEpochManager) extends MetadataSupport { override def requireZkOrThrow(createException: => Exception): ZkSupport = this override def requireRaftOrThrow(createException: => Exception): RaftSupport = throw createException diff --git a/core/src/main/scala/kafka/server/BrokerEpochManager.scala b/core/src/main/scala/kafka/server/ZkBrokerEpochManager.scala similarity index 91% rename from core/src/main/scala/kafka/server/BrokerEpochManager.scala rename to core/src/main/scala/kafka/server/ZkBrokerEpochManager.scala index 97ebb48658fa8..36b2815444576 100644 --- a/core/src/main/scala/kafka/server/BrokerEpochManager.scala +++ b/core/src/main/scala/kafka/server/ZkBrokerEpochManager.scala @@ -20,9 +20,9 @@ package kafka.server import kafka.controller.KafkaController import org.apache.kafka.common.requests.AbstractControlRequest -class BrokerEpochManager(metadataCache: MetadataCache, - controller: KafkaController, - lifecycleManagerOpt: Option[BrokerLifecycleManager]) { +class ZkBrokerEpochManager(metadataCache: MetadataCache, + controller: KafkaController, + lifecycleManagerOpt: Option[BrokerLifecycleManager]) { def get(): Long = { lifecycleManagerOpt match { case Some(lifecycleManager) => metadataCache.getControllerId match { diff --git a/core/src/test/scala/unit/kafka/server/BrokerLifecycleManagerTest.scala b/core/src/test/scala/unit/kafka/server/BrokerLifecycleManagerTest.scala index da6d9a8aa80cf..1a7569a987112 100644 --- a/core/src/test/scala/unit/kafka/server/BrokerLifecycleManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/BrokerLifecycleManagerTest.scala @@ -59,7 +59,7 @@ class BrokerLifecycleManagerTest { def listenerName: ListenerName = new ListenerName("PLAINTEXT") - def securityProtocol: SecurityProtocol = SecurityProtocol.PLAINTEXT; + def securityProtocol: SecurityProtocol = SecurityProtocol.PLAINTEXT def saslMechanism: String = SaslConfigs.DEFAULT_SASL_MECHANISM @@ -98,14 +98,14 @@ class BrokerLifecycleManagerTest { @Test def testCreateAndClose(): Unit = { val context = new BrokerLifecycleManagerTestContext(configProperties) - val manager = new BrokerLifecycleManager(context.config, context.time, None) + val manager = new BrokerLifecycleManager(context.config, context.time, None, isZkBroker = false, () => -1) manager.close() } @Test def testCreateStartAndClose(): Unit = { val context = new BrokerLifecycleManagerTestContext(configProperties) - val manager = new BrokerLifecycleManager(context.config, context.time, None) + val manager = new BrokerLifecycleManager(context.config, context.time, None, isZkBroker = false, () => -1) assertEquals(BrokerState.NOT_RUNNING, manager.state) manager.start(() => context.highestMetadataOffset.get(), context.mockChannelManager, context.clusterId, context.advertisedListeners, @@ -120,7 +120,7 @@ class BrokerLifecycleManagerTest { @Test def testSuccessfulRegistration(): Unit = { val context = new BrokerLifecycleManagerTestContext(configProperties) - val manager = new BrokerLifecycleManager(context.config, context.time, None) + val manager = new BrokerLifecycleManager(context.config, context.time, None, isZkBroker = false, () => -1) val controllerNode = new Node(3000, "localhost", 8021) context.controllerNodeProvider.node.set(controllerNode) context.mockClient.prepareResponseFrom(new BrokerRegistrationResponse( @@ -140,7 +140,7 @@ class BrokerLifecycleManagerTest { def testRegistrationTimeout(): Unit = { val context = new BrokerLifecycleManagerTestContext(configProperties) val controllerNode = new Node(3000, "localhost", 8021) - val manager = new BrokerLifecycleManager(context.config, context.time, None) + val manager = new BrokerLifecycleManager(context.config, context.time, None, isZkBroker = false, () => -1) context.controllerNodeProvider.node.set(controllerNode) def newDuplicateRegistrationResponse(): Unit = { context.mockClient.prepareResponseFrom(new BrokerRegistrationResponse( @@ -181,7 +181,7 @@ class BrokerLifecycleManagerTest { @Test def testControlledShutdown(): Unit = { val context = new BrokerLifecycleManagerTestContext(configProperties) - val manager = new BrokerLifecycleManager(context.config, context.time, None) + val manager = new BrokerLifecycleManager(context.config, context.time, None, isZkBroker = false, () => -1) val controllerNode = new Node(3000, "localhost", 8021) context.controllerNodeProvider.node.set(controllerNode) context.mockClient.prepareResponseFrom(new BrokerRegistrationResponse( diff --git a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala index 0bc82f984c7ed..4244edda0f20c 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala @@ -106,7 +106,7 @@ class KafkaApisTest { private val controller: KafkaController = mock(classOf[KafkaController]) private val forwardingManager: ForwardingManager = mock(classOf[ForwardingManager]) private val autoTopicCreationManager: AutoTopicCreationManager = mock(classOf[AutoTopicCreationManager]) - private val brokerEpochManager: BrokerEpochManager = mock(classOf[BrokerEpochManager]) + private val brokerEpochManager: ZkBrokerEpochManager = mock(classOf[ZkBrokerEpochManager]) private val kafkaPrincipalSerde = new KafkaPrincipalSerde { override def serialize(principal: KafkaPrincipal): Array[Byte] = Utils.utf8(principal.toString) diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/metadata/MetadataRequestBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/metadata/MetadataRequestBenchmark.java index 34055009b3045..9b907ef7f9edd 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/metadata/MetadataRequestBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/metadata/MetadataRequestBenchmark.java @@ -23,7 +23,7 @@ import kafka.network.RequestChannel; import kafka.network.RequestConvertToJson; import kafka.server.AutoTopicCreationManager; -import kafka.server.BrokerEpochManager; +import kafka.server.ZkBrokerEpochManager; import kafka.server.BrokerFeatures; import kafka.server.BrokerTopicStats; import kafka.server.ClientQuotaManager; @@ -182,7 +182,7 @@ private KafkaApis createKafkaApis() { return new KafkaApisBuilder(). setRequestChannel(requestChannel). setMetadataSupport(new ZkSupport(adminManager, kafkaController, kafkaZkClient, - Option.empty(), metadataCache, new BrokerEpochManager(metadataCache, kafkaController, Option.empty()))). + Option.empty(), metadataCache, new ZkBrokerEpochManager(metadataCache, kafkaController, Option.empty()))). setReplicaManager(replicaManager). setGroupCoordinator(groupCoordinator). setTxnCoordinator(transactionCoordinator). diff --git a/metadata/src/main/java/org/apache/kafka/controller/MigrationControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/MigrationControlManager.java index 7f3f700576887..7f477785129c1 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/MigrationControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/MigrationControlManager.java @@ -28,7 +28,7 @@ public class MigrationControlManager { zkMigrationState = new TimelineObject<>(snapshotRegistry, ZkMigrationState.NONE); } - public ZkMigrationState zkMigrationState() { + ZkMigrationState zkMigrationState() { return zkMigrationState.get(); } diff --git a/metadata/src/main/java/org/apache/kafka/metadata/migration/BrokersRpcClient.java b/metadata/src/main/java/org/apache/kafka/metadata/migration/BrokersRpcClient.java deleted file mode 100644 index 727498ec64c14..0000000000000 --- a/metadata/src/main/java/org/apache/kafka/metadata/migration/BrokersRpcClient.java +++ /dev/null @@ -1,37 +0,0 @@ -/* - * 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.metadata.migration; - -import org.apache.kafka.image.MetadataDelta; -import org.apache.kafka.image.MetadataImage; - -public interface BrokersRpcClient { - - void startup(); - - void shutdown(); - - void publishMetadata(MetadataImage image); - - void sendRPCsToBrokersFromMetadataDelta(MetadataDelta delta, - MetadataImage image, - int controllerEpoch); - - void sendRPCsToBrokersFromMetadataImage(MetadataImage image, int controllerEpoch); - - void clear(); -} From 028b3e6202ebe3f80771c66416f7f71834a8e9ea Mon Sep 17 00:00:00 2001 From: David Arthur Date: Thu, 5 Jan 2023 15:59:35 -0500 Subject: [PATCH 23/30] Fixup after merge from trunk --- build.gradle | 2 +- core/src/main/scala/kafka/server/ControllerServer.scala | 2 +- .../main/java/org/apache/kafka/controller/QuorumController.java | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/build.gradle b/build.gradle index 776bbbcc699fa..fb5756492d1fd 100644 --- a/build.gradle +++ b/build.gradle @@ -1046,7 +1046,7 @@ project(':core') { task genTopicConfigDocs(type: JavaExec) { classpath = sourceSets.main.runtimeClasspath - mainClass = 'kafka.log.LogConfig' + mainClass = 'org.apache.kafka.server.log.internals.LogConfig' if( !generatedDocsDir.exists() ) { generatedDocsDir.mkdirs() } standardOutput = new File(generatedDocsDir, "topic_config.html").newOutputStream() } diff --git a/core/src/main/scala/kafka/server/ControllerServer.scala b/core/src/main/scala/kafka/server/ControllerServer.scala index 465cdb36970c9..4270d9448d223 100644 --- a/core/src/main/scala/kafka/server/ControllerServer.scala +++ b/core/src/main/scala/kafka/server/ControllerServer.scala @@ -243,7 +243,7 @@ class ControllerServer( val zkClient = KafkaZkClient.createZkClient("KRaft Migration", time, config, KafkaServer.zkClientConfigFromKafkaConfig(config)) val migrationClient = new ZkMigrationClient(zkClient) val rpcClient: LegacyPropagator = new MigrationPropagator(config.nodeId, config, () => - config.interBrokerProtocolVersion) + controller.asInstanceOf[QuorumController].metadataVersion()) val migrationDriver = new KRaftMigrationDriver( config.nodeId, controller.asInstanceOf[QuorumController].zkRecordConsumer(), diff --git a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java index 04fccc5deec30..7a0cd2f681850 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java +++ b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java @@ -2099,7 +2099,7 @@ public int curClaimEpoch() { } // Visible for testing - MetadataVersion metadataVersion() { + public MetadataVersion metadataVersion() { return featureControl.metadataVersion(); } From c39b37df0bdaf5e669524f7dd11378eeeafcf7dc Mon Sep 17 00:00:00 2001 From: "Colin P. McCabe" Date: Fri, 6 Jan 2023 10:33:43 -0800 Subject: [PATCH 24/30] Fix un-thread-safe access to QC fields, fix some variable names --- .../kafka/migration/MigrationPropagator.scala | 10 +++++++--- .../main/scala/kafka/server/ControllerServer.scala | 7 +++---- .../apache/kafka/controller/QuorumController.java | 5 ----- .../metadata/migration/KRaftMigrationDriver.java | 14 +++++++++----- .../kafka/metadata/migration/LegacyPropagator.java | 3 +++ 5 files changed, 22 insertions(+), 17 deletions(-) diff --git a/core/src/main/scala/kafka/migration/MigrationPropagator.scala b/core/src/main/scala/kafka/migration/MigrationPropagator.scala index 9c61ffd7aefc3..308d8f06f152f 100644 --- a/core/src/main/scala/kafka/migration/MigrationPropagator.scala +++ b/core/src/main/scala/kafka/migration/MigrationPropagator.scala @@ -32,10 +32,10 @@ import scala.jdk.CollectionConverters._ class MigrationPropagator( nodeId: Int, - config: KafkaConfig, - metadataVersionProvider: () => MetadataVersion, + config: KafkaConfig ) extends LegacyPropagator { @volatile private var _image = MetadataImage.EMPTY + @volatile private var metadataVersion = MetadataVersion.IBP_3_4_IV0 val stateChangeLogger = new StateChangeLogger(nodeId, inControllerContext = true, None) val channelManager = new ControllerChannelManager( () => _image.highestOffsetAndEpoch().epoch(), @@ -48,7 +48,7 @@ class MigrationPropagator( val requestBatch = new MigrationPropagatorBatch( config, metadataProvider, - metadataVersionProvider, + () => metadataVersion, channelManager, stateChangeLogger ) @@ -201,4 +201,8 @@ class MigrationPropagator( override def clear(): Unit = { requestBatch.clear() } + + override def setMetadataVersion(newMetadataVersion: MetadataVersion): Unit = { + metadataVersion = newMetadataVersion + } } diff --git a/core/src/main/scala/kafka/server/ControllerServer.scala b/core/src/main/scala/kafka/server/ControllerServer.scala index 4270d9448d223..e27e46eed432a 100644 --- a/core/src/main/scala/kafka/server/ControllerServer.scala +++ b/core/src/main/scala/kafka/server/ControllerServer.scala @@ -242,13 +242,12 @@ class ControllerServer( if (config.migrationEnabled) { val zkClient = KafkaZkClient.createZkClient("KRaft Migration", time, config, KafkaServer.zkClientConfigFromKafkaConfig(config)) val migrationClient = new ZkMigrationClient(zkClient) - val rpcClient: LegacyPropagator = new MigrationPropagator(config.nodeId, config, () => - controller.asInstanceOf[QuorumController].metadataVersion()) + val propagator: LegacyPropagator = new MigrationPropagator(config.nodeId, config) val migrationDriver = new KRaftMigrationDriver( config.nodeId, controller.asInstanceOf[QuorumController].zkRecordConsumer(), migrationClient, - rpcClient, + propagator, publisher => sharedServer.loader.installPublishers(java.util.Collections.singletonList(publisher)), sharedServer.faultHandlerFactory.build( "zk migration", @@ -257,7 +256,7 @@ class ControllerServer( ) ) migrationDriver.start() - migrationSupport = Some(ControllerMigrationSupport(zkClient, migrationDriver, rpcClient)) + migrationSupport = Some(ControllerMigrationSupport(zkClient, migrationDriver, propagator)) } quotaManagers = QuotaFactory.instantiate(config, diff --git a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java index 7a0cd2f681850..bf2f20e91249f 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java +++ b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java @@ -2098,11 +2098,6 @@ public int curClaimEpoch() { return curClaimEpoch; } - // Visible for testing - public MetadataVersion metadataVersion() { - return featureControl.metadataVersion(); - } - @Override public void close() throws InterruptedException { queue.close(); diff --git a/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java b/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java index f5f220be12b3d..391cf55a29395 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java @@ -28,6 +28,7 @@ import org.apache.kafka.queue.KafkaEventQueue; import org.apache.kafka.raft.LeaderAndEpoch; import org.apache.kafka.raft.OffsetAndEpoch; +import org.apache.kafka.server.common.MetadataVersion; import org.apache.kafka.server.fault.FaultHandler; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -53,7 +54,7 @@ public class KRaftMigrationDriver implements MetadataPublisher { private final Logger log; private final int nodeId; private final MigrationClient zkMigrationClient; - private final LegacyPropagator rpcClient; + private final LegacyPropagator propagator; private final ZkRecordConsumer zkRecordConsumer; private final KafkaEventQueue eventQueue; private final FaultHandler faultHandler; @@ -71,14 +72,14 @@ public KRaftMigrationDriver( int nodeId, ZkRecordConsumer zkRecordConsumer, MigrationClient zkMigrationClient, - LegacyPropagator rpcClient, + LegacyPropagator propagator, Consumer initialZkLoadHandler, FaultHandler faultHandler ) { this.nodeId = nodeId; this.zkRecordConsumer = zkRecordConsumer; this.zkMigrationClient = zkMigrationClient; - this.rpcClient = rpcClient; + this.propagator = propagator; this.time = Time.SYSTEM; this.log = LoggerFactory.getLogger(KRaftMigrationDriver.class); this.migrationState = MigrationState.UNINITIALIZED; @@ -444,7 +445,7 @@ public void run() throws Exception { // Ignore sending RPCs to the brokers since we're no longer in the state. if (migrationState == MigrationState.KRAFT_CONTROLLER_TO_BROKER_COMM) { if (image.highestOffsetAndEpoch().compareTo(migrationLeadershipState.offsetAndEpoch()) >= 0) { - rpcClient.sendRPCsToBrokersFromMetadataImage(image, migrationLeadershipState.zkControllerEpoch()); + propagator.sendRPCsToBrokersFromMetadataImage(image, migrationLeadershipState.zkControllerEpoch()); // Migration leadership state doesn't change since we're not doing any Zk writes. transitionTo(MigrationState.DUAL_WRITE); } @@ -474,6 +475,9 @@ public void run() throws Exception { "mode. Ignoring the change to be replicated to Zookeeper"); return; } + if (delta.featuresDelta() != null) { + propagator.setMetadataVersion(image.features().metadataVersion()); + } if (image.highestOffsetAndEpoch().compareTo(migrationLeadershipState.offsetAndEpoch()) >= 0) { if (delta.topicsDelta() != null) { @@ -498,7 +502,7 @@ public void run() throws Exception { apply("Write MetadataDelta to Zk", state -> zkMigrationClient.writeMetadataDeltaToZookeeper(delta, image, state)); // TODO: Unhappy path: Probably relinquish leadership and let new controller // retry the write? - rpcClient.sendRPCsToBrokersFromMetadataDelta(delta, image, + propagator.sendRPCsToBrokersFromMetadataDelta(delta, image, migrationLeadershipState.zkControllerEpoch()); } else { String metadataType = isSnapshot ? "snapshot" : "delta"; diff --git a/metadata/src/main/java/org/apache/kafka/metadata/migration/LegacyPropagator.java b/metadata/src/main/java/org/apache/kafka/metadata/migration/LegacyPropagator.java index c2bd4e6f3d0a1..6fe707099a3c9 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/migration/LegacyPropagator.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/migration/LegacyPropagator.java @@ -18,6 +18,7 @@ import org.apache.kafka.image.MetadataDelta; import org.apache.kafka.image.MetadataImage; +import org.apache.kafka.server.common.MetadataVersion; public interface LegacyPropagator { @@ -34,4 +35,6 @@ void sendRPCsToBrokersFromMetadataDelta(MetadataDelta delta, void sendRPCsToBrokersFromMetadataImage(MetadataImage image, int zkControllerEpoch); void clear(); + + void setMetadataVersion(MetadataVersion metadataVersion); } From 960aed57a883ebe929bab702d094c1c163ebeb6e Mon Sep 17 00:00:00 2001 From: "Colin P. McCabe" Date: Fri, 6 Jan 2023 10:38:08 -0800 Subject: [PATCH 25/30] fix checkstyle --- .../apache/kafka/metadata/migration/KRaftMigrationDriver.java | 1 - 1 file changed, 1 deletion(-) diff --git a/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java b/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java index 391cf55a29395..8070ae9a2e44e 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java @@ -28,7 +28,6 @@ import org.apache.kafka.queue.KafkaEventQueue; import org.apache.kafka.raft.LeaderAndEpoch; import org.apache.kafka.raft.OffsetAndEpoch; -import org.apache.kafka.server.common.MetadataVersion; import org.apache.kafka.server.fault.FaultHandler; import org.slf4j.Logger; import org.slf4j.LoggerFactory; From 358f535902e29c77d890bdc989d78872efd23895 Mon Sep 17 00:00:00 2001 From: David Arthur Date: Fri, 6 Jan 2023 14:20:08 -0500 Subject: [PATCH 26/30] Revert default exception handler --- .../src/main/java/org/apache/kafka/queue/EventQueue.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/server-common/src/main/java/org/apache/kafka/queue/EventQueue.java b/server-common/src/main/java/org/apache/kafka/queue/EventQueue.java index 9fbf1e6dfa8f3..d0c752e641da9 100644 --- a/server-common/src/main/java/org/apache/kafka/queue/EventQueue.java +++ b/server-common/src/main/java/org/apache/kafka/queue/EventQueue.java @@ -42,9 +42,7 @@ interface Event { * scheduled because the event queue has already been closed. * Otherwise, it will be whatever exception was thrown by run(). */ - default void handleException(Throwable e) { - throw new RuntimeException("Unhandled exception in EventQueue when running " + this, e); - } + default void handleException(Throwable e) {} } abstract class FailureLoggingEvent implements Event { From c5aa43bfab430a1b52c313da2e5a769a5937ee16 Mon Sep 17 00:00:00 2001 From: David Arthur Date: Fri, 6 Jan 2023 14:30:20 -0500 Subject: [PATCH 27/30] Pin metadata.version to 3.4, fix some tests --- core/src/main/scala/kafka/migration/MigrationPropagator.scala | 2 -- .../main/scala/kafka/migration/MigrationPropagatorBatch.scala | 3 +-- core/src/main/scala/kafka/server/ControllerServer.scala | 3 +-- core/src/main/scala/kafka/server/KafkaConfig.scala | 4 +++- .../java/org/apache/kafka/controller/QuorumController.java | 2 +- 5 files changed, 6 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/kafka/migration/MigrationPropagator.scala b/core/src/main/scala/kafka/migration/MigrationPropagator.scala index 9c61ffd7aefc3..691e02a9228e3 100644 --- a/core/src/main/scala/kafka/migration/MigrationPropagator.scala +++ b/core/src/main/scala/kafka/migration/MigrationPropagator.scala @@ -33,7 +33,6 @@ import scala.jdk.CollectionConverters._ class MigrationPropagator( nodeId: Int, config: KafkaConfig, - metadataVersionProvider: () => MetadataVersion, ) extends LegacyPropagator { @volatile private var _image = MetadataImage.EMPTY val stateChangeLogger = new StateChangeLogger(nodeId, inControllerContext = true, None) @@ -48,7 +47,6 @@ class MigrationPropagator( val requestBatch = new MigrationPropagatorBatch( config, metadataProvider, - metadataVersionProvider, channelManager, stateChangeLogger ) diff --git a/core/src/main/scala/kafka/migration/MigrationPropagatorBatch.scala b/core/src/main/scala/kafka/migration/MigrationPropagatorBatch.scala index 7c4952c9382ef..8096843794501 100644 --- a/core/src/main/scala/kafka/migration/MigrationPropagatorBatch.scala +++ b/core/src/main/scala/kafka/migration/MigrationPropagatorBatch.scala @@ -29,13 +29,12 @@ import scala.jdk.CollectionConverters._ sealed class MigrationPropagatorBatch( config: KafkaConfig, metadataProvider: () => ControllerChannelContext, - metadataVersionProvider: () => MetadataVersion, controllerChannelManager: ControllerChannelManager, stateChangeLogger: StateChangeLogger ) extends AbstractControllerBrokerRequestBatch( config, metadataProvider, - metadataVersionProvider, + () => MetadataVersion.IBP_3_4_IV0, stateChangeLogger, kraftController = true, ) { diff --git a/core/src/main/scala/kafka/server/ControllerServer.scala b/core/src/main/scala/kafka/server/ControllerServer.scala index 4270d9448d223..184ba80da922a 100644 --- a/core/src/main/scala/kafka/server/ControllerServer.scala +++ b/core/src/main/scala/kafka/server/ControllerServer.scala @@ -242,8 +242,7 @@ class ControllerServer( if (config.migrationEnabled) { val zkClient = KafkaZkClient.createZkClient("KRaft Migration", time, config, KafkaServer.zkClientConfigFromKafkaConfig(config)) val migrationClient = new ZkMigrationClient(zkClient) - val rpcClient: LegacyPropagator = new MigrationPropagator(config.nodeId, config, () => - controller.asInstanceOf[QuorumController].metadataVersion()) + val rpcClient: LegacyPropagator = new MigrationPropagator(config.nodeId, config) val migrationDriver = new KRaftMigrationDriver( config.nodeId, controller.asInstanceOf[QuorumController].zkRecordConsumer(), diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 5903a495a4747..92bf9567bf5dd 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -2193,7 +2193,9 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami if (migrationEnabled) { validateNonEmptyQuorumVotersForKRaft() require(controllerListenerNames.nonEmpty, - s"${KafkaConfig.ControllerListenerNamesProp} must not be empty when running in ZK migration mode: ${controllerListenerNames.asJava}") + s"${KafkaConfig.ControllerListenerNamesProp} must not be empty when running in ZooKeeper migration mode: ${controllerListenerNames.asJava}") + require(interBrokerProtocolVersion.isMigrationSupported, s"Cannot enable ZooKeeper migration without setting " + + s"'${KafkaConfig.InterBrokerProtocolVersionProp}' to 3.4 or higher") } else { // controller listener names must be empty when not in KRaft mode require(controllerListenerNames.isEmpty, diff --git a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java index 7a0cd2f681850..04fccc5deec30 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java +++ b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java @@ -2099,7 +2099,7 @@ public int curClaimEpoch() { } // Visible for testing - public MetadataVersion metadataVersion() { + MetadataVersion metadataVersion() { return featureControl.metadataVersion(); } From 9d051dd618706aacf5ab4334493f9b406a7ae5d6 Mon Sep 17 00:00:00 2001 From: David Arthur Date: Sun, 8 Jan 2023 14:45:19 -0500 Subject: [PATCH 28/30] Fix failing integration tests --- .../main/scala/kafka/server/KafkaConfig.scala | 6 ++++++ .../junit/RaftClusterInvocationContext.java | 17 +++++++++++++++-- .../test/junit/ZkClusterInvocationContext.java | 1 - .../KafkaServerKRaftRegistrationTest.scala | 3 +++ .../scala/unit/kafka/server/KafkaApisTest.scala | 2 +- .../scala/unit/kafka/zk/EmbeddedZookeeper.scala | 9 ++++++--- 6 files changed, 31 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 92bf9567bf5dd..8097f82fa2cf9 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -2088,6 +2088,12 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami throw new ConfigException(s"Missing configuration `${KafkaConfig.NodeIdProp}` which is required " + s"when `process.roles` is defined (i.e. when running in KRaft mode).") } + if (migrationEnabled) { + if (zkConnect == null) { + throw new ConfigException(s"Missing required configuration `${KafkaConfig.ZkConnectProp}` which has no default value. " + + s"`${KafkaConfig.ZkConnectProp}` is required because `${KafkaConfig.MigrationEnabledProp} is set to true.") + } + } } require(logRollTimeMillis >= 1, "log.roll.ms must be greater than or equal to 1") require(logRollTimeJitterMillis >= 0, "log.roll.jitter.ms must be greater than or equal to 0") diff --git a/core/src/test/java/kafka/test/junit/RaftClusterInvocationContext.java b/core/src/test/java/kafka/test/junit/RaftClusterInvocationContext.java index f7eb0a501798e..e5702cf95fcf8 100644 --- a/core/src/test/java/kafka/test/junit/RaftClusterInvocationContext.java +++ b/core/src/test/java/kafka/test/junit/RaftClusterInvocationContext.java @@ -25,6 +25,7 @@ import kafka.test.ClusterInstance; import kafka.testkit.KafkaClusterTestKit; import kafka.testkit.TestKitNodes; +import kafka.zk.EmbeddedZookeeper; import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.admin.Admin; import org.apache.kafka.common.network.ListenerName; @@ -66,11 +67,13 @@ public class RaftClusterInvocationContext implements TestTemplateInvocationConte private final ClusterConfig clusterConfig; private final AtomicReference clusterReference; + private final AtomicReference zkReference; private final boolean isCoResident; public RaftClusterInvocationContext(ClusterConfig clusterConfig, boolean isCoResident) { this.clusterConfig = clusterConfig; this.clusterReference = new AtomicReference<>(); + this.zkReference = new AtomicReference<>(); this.isCoResident = isCoResident; } @@ -84,7 +87,7 @@ public String getDisplayName(int invocationIndex) { @Override public List getAdditionalExtensions() { - RaftClusterInstance clusterInstance = new RaftClusterInstance(clusterReference, clusterConfig); + RaftClusterInstance clusterInstance = new RaftClusterInstance(clusterReference, zkReference, clusterConfig); return Arrays.asList( (BeforeTestExecutionCallback) context -> { TestKitNodes nodes = new TestKitNodes.Builder(). @@ -98,6 +101,11 @@ public List getAdditionalExtensions() { }); KafkaClusterTestKit.Builder builder = new KafkaClusterTestKit.Builder(nodes); + if (Boolean.parseBoolean(clusterConfig.serverProperties().getProperty("zookeeper.metadata.migration.enable", "false"))) { + zkReference.set(new EmbeddedZookeeper()); + builder.setConfigProp("zookeeper.connect", String.format("localhost:%d", zkReference.get().port())); + } + // Copy properties into the TestKit builder clusterConfig.serverProperties().forEach((key, value) -> builder.setConfigProp(key.toString(), value.toString())); // KAFKA-12512 need to pass security protocol and listener name here @@ -120,13 +128,15 @@ public List getAdditionalExtensions() { public static class RaftClusterInstance implements ClusterInstance { private final AtomicReference clusterReference; + private final AtomicReference zkReference; private final ClusterConfig clusterConfig; final AtomicBoolean started = new AtomicBoolean(false); final AtomicBoolean stopped = new AtomicBoolean(false); private final ConcurrentLinkedQueue admins = new ConcurrentLinkedQueue<>(); - RaftClusterInstance(AtomicReference clusterReference, ClusterConfig clusterConfig) { + RaftClusterInstance(AtomicReference clusterReference, AtomicReference zkReference, ClusterConfig clusterConfig) { this.clusterReference = clusterReference; + this.zkReference = zkReference; this.clusterConfig = clusterConfig; } @@ -247,6 +257,9 @@ public void stop() { if (stopped.compareAndSet(false, true)) { admins.forEach(admin -> Utils.closeQuietly(admin, "admin")); Utils.closeQuietly(clusterReference.get(), "cluster"); + if (zkReference.get() != null) { + Utils.closeQuietly(zkReference.get(), "zk"); + } } } diff --git a/core/src/test/java/kafka/test/junit/ZkClusterInvocationContext.java b/core/src/test/java/kafka/test/junit/ZkClusterInvocationContext.java index 0d9375dbd7c2c..5154982e47219 100644 --- a/core/src/test/java/kafka/test/junit/ZkClusterInvocationContext.java +++ b/core/src/test/java/kafka/test/junit/ZkClusterInvocationContext.java @@ -339,6 +339,5 @@ private KafkaServer findBrokerOrThrow(int brokerId) { private Stream servers() { return JavaConverters.asJavaCollection(clusterReference.get().servers()).stream(); } - } } diff --git a/core/src/test/scala/integration/kafka/server/KafkaServerKRaftRegistrationTest.scala b/core/src/test/scala/integration/kafka/server/KafkaServerKRaftRegistrationTest.scala index 9737a0ea0596c..f961cd4507eaa 100644 --- a/core/src/test/scala/integration/kafka/server/KafkaServerKRaftRegistrationTest.scala +++ b/core/src/test/scala/integration/kafka/server/KafkaServerKRaftRegistrationTest.scala @@ -20,6 +20,7 @@ package kafka.server import kafka.test.ClusterInstance import kafka.test.annotation.{ClusterTest, Type} import kafka.test.junit.ClusterTestExtensions +import kafka.test.junit.ZkClusterInvocationContext.ZkClusterInstance import kafka.testkit.{KafkaClusterTestKit, TestKitNodes} import org.apache.kafka.common.Uuid import org.apache.kafka.raft.RaftConfig @@ -55,6 +56,7 @@ class KafkaServerKRaftRegistrationTest { setNumBrokerNodes(0). setNumControllerNodes(1).build()) .setConfigProp(KafkaConfig.MigrationEnabledProp, "true") + .setConfigProp(KafkaConfig.ZkConnectProp, zkCluster.asInstanceOf[ZkClusterInstance].getUnderlying.zkConnect) .build() try { kraftCluster.format() @@ -79,6 +81,7 @@ class KafkaServerKRaftRegistrationTest { case t: Throwable => fail("Had some other error waiting for brokers", t) } } finally { + zkCluster.stop() kraftCluster.close() } } diff --git a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala index 4244edda0f20c..f80a4abf35f8d 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala @@ -106,7 +106,6 @@ class KafkaApisTest { private val controller: KafkaController = mock(classOf[KafkaController]) private val forwardingManager: ForwardingManager = mock(classOf[ForwardingManager]) private val autoTopicCreationManager: AutoTopicCreationManager = mock(classOf[AutoTopicCreationManager]) - private val brokerEpochManager: ZkBrokerEpochManager = mock(classOf[ZkBrokerEpochManager]) private val kafkaPrincipalSerde = new KafkaPrincipalSerde { override def serialize(principal: KafkaPrincipal): Array[Byte] = Utils.utf8(principal.toString) @@ -117,6 +116,7 @@ class KafkaApisTest { private val brokerId = 1 // KRaft tests should override this with a KRaftMetadataCache private var metadataCache: MetadataCache = MetadataCache.zkMetadataCache(brokerId, MetadataVersion.latest()) + private val brokerEpochManager: ZkBrokerEpochManager = new ZkBrokerEpochManager(metadataCache, controller, None) private val clientQuotaManager: ClientQuotaManager = mock(classOf[ClientQuotaManager]) private val clientRequestQuotaManager: ClientRequestQuotaManager = mock(classOf[ClientRequestQuotaManager]) private val clientControllerQuotaManager: ControllerMutationQuotaManager = mock(classOf[ControllerMutationQuotaManager]) diff --git a/core/src/test/scala/unit/kafka/zk/EmbeddedZookeeper.scala b/core/src/test/scala/unit/kafka/zk/EmbeddedZookeeper.scala index 28b592eaf7af8..a1424a269cecd 100755 --- a/core/src/test/scala/unit/kafka/zk/EmbeddedZookeeper.scala +++ b/core/src/test/scala/unit/kafka/zk/EmbeddedZookeeper.scala @@ -20,10 +20,12 @@ package kafka.zk import org.apache.zookeeper.server.ZooKeeperServer import org.apache.zookeeper.server.NIOServerCnxnFactory import kafka.utils.{CoreUtils, Logging, TestUtils} -import java.net.InetSocketAddress +import java.net.InetSocketAddress import org.apache.kafka.common.utils.Utils +import java.io.Closeable + /** * ZooKeeperServer wrapper that starts the server with temporary directories during construction and deletes * the directories when `shutdown()` is called. @@ -34,7 +36,7 @@ import org.apache.kafka.common.utils.Utils // This should be named EmbeddedZooKeeper for consistency with other classes, but since this is widely used by other // projects (even though it's internal), we keep the name as it is until we have a publicly supported test library for // others to use. -class EmbeddedZookeeper() extends Logging { +class EmbeddedZookeeper() extends Closeable with Logging { val snapshotDir = TestUtils.tempDir() val logDir = TestUtils.tempDir() @@ -65,5 +67,6 @@ class EmbeddedZookeeper() extends Logging { Utils.delete(logDir) Utils.delete(snapshotDir) } - + + override def close(): Unit = shutdown() } From c1ddfcaf23cc8122d76bdafa0f7f85568a4c922d Mon Sep 17 00:00:00 2001 From: David Arthur Date: Sun, 8 Jan 2023 16:47:58 -0500 Subject: [PATCH 29/30] Close ZK client before KRaftMigrationDriver in shutdown --- core/src/main/scala/kafka/server/ControllerServer.scala | 6 +++--- .../kafka/metadata/migration/KRaftMigrationDriver.java | 2 ++ 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/kafka/server/ControllerServer.scala b/core/src/main/scala/kafka/server/ControllerServer.scala index e27e46eed432a..c856089af3e97 100644 --- a/core/src/main/scala/kafka/server/ControllerServer.scala +++ b/core/src/main/scala/kafka/server/ControllerServer.scala @@ -58,15 +58,15 @@ case class ControllerMigrationSupport( brokersRpcClient: LegacyPropagator ) { def shutdown(logging: Logging): Unit = { - if (migrationDriver != null) { - CoreUtils.swallow(migrationDriver.close(), logging) - } if (zkClient != null) { CoreUtils.swallow(zkClient.close(), logging) } if (brokersRpcClient != null) { CoreUtils.swallow(brokersRpcClient.shutdown(), logging) } + if (migrationDriver != null) { + CoreUtils.swallow(migrationDriver.close(), logging) + } } } diff --git a/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java b/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java index 8070ae9a2e44e..f3bdeb8150742 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java @@ -95,6 +95,8 @@ public void start() { } public void shutdown() throws InterruptedException { + eventQueue.beginShutdown("KRaftMigrationDriver#shutdown"); + log.debug("Shutting down KRaftMigrationDriver"); eventQueue.close(); } From 894a917351983ab4516caa2a924cae3b329680fb Mon Sep 17 00:00:00 2001 From: "Colin P. McCabe" Date: Sun, 8 Jan 2023 21:51:17 -0800 Subject: [PATCH 30/30] fix checkstyle --- checkstyle/import-control-core.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/checkstyle/import-control-core.xml b/checkstyle/import-control-core.xml index 4042cba402fdd..5a92c44d7c87e 100644 --- a/checkstyle/import-control-core.xml +++ b/checkstyle/import-control-core.xml @@ -89,6 +89,7 @@ +