From 2d4be265feaeaf433a77b296e0d8e3f1e2db79d3 Mon Sep 17 00:00:00 2001 From: David Arthur Date: Wed, 7 Apr 2021 13:04:18 -0400 Subject: [PATCH 01/20] Add new AllocateProducerIds RPC and support for ZK-mode --- .../apache/kafka/common/protocol/ApiKeys.java | 3 +- .../common/requests/AbstractRequest.java | 2 + .../common/requests/AbstractResponse.java | 2 + .../requests/AllocateProducerIdsRequest.java | 72 +++++++ .../requests/AllocateProducerIdsResponse.java | 63 ++++++ .../message/AllocateProducerIdsRequest.json | 29 +++ .../message/AllocateProducerIdsResponse.json | 32 +++ .../kafka/controller/KafkaController.scala | 90 +++++++- .../transaction/ProducerIdManager.scala | 192 +++++++++--------- .../kafka/network/RequestConvertToJson.scala | 2 + .../main/scala/kafka/server/KafkaApis.scala | 16 ++ .../main/scala/kafka/server/KafkaServer.scala | 22 +- core/src/main/scala/kafka/zk/ZkData.scala | 31 ++- .../test/java/kafka/test/ClusterInstance.java | 2 + .../junit/RaftClusterInvocationContext.java | 5 + .../junit/ZkClusterInvocationContext.java | 12 ++ .../ProducerIdsIntegrationTest.scala | 82 ++++++++ .../transaction/ProducerIdManagerTest.scala | 101 ++++----- .../common/metadata/ProducerIdRecord.json | 29 +++ 19 files changed, 625 insertions(+), 162 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/common/requests/AllocateProducerIdsRequest.java create mode 100644 clients/src/main/java/org/apache/kafka/common/requests/AllocateProducerIdsResponse.java create mode 100644 clients/src/main/resources/common/message/AllocateProducerIdsRequest.json create mode 100644 clients/src/main/resources/common/message/AllocateProducerIdsResponse.json create mode 100644 core/src/test/scala/integration/kafka/coordinator/transaction/ProducerIdsIntegrationTest.scala create mode 100644 metadata/src/main/resources/common/metadata/ProducerIdRecord.json diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java b/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java index 8ec2d02ea3de6..bd28aa26aa977 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java @@ -107,7 +107,8 @@ public enum ApiKeys { BROKER_HEARTBEAT(ApiMessageType.BROKER_HEARTBEAT, true, RecordBatch.MAGIC_VALUE_V0, false), UNREGISTER_BROKER(ApiMessageType.UNREGISTER_BROKER, false, RecordBatch.MAGIC_VALUE_V0, true), DESCRIBE_TRANSACTIONS(ApiMessageType.DESCRIBE_TRANSACTIONS), - LIST_TRANSACTIONS(ApiMessageType.LIST_TRANSACTIONS); + LIST_TRANSACTIONS(ApiMessageType.LIST_TRANSACTIONS), + ALLOCATE_PRODUCER_IDS(ApiMessageType.ALLOCATE_PRODUCER_IDS, true, false); private static final Map> APIS_BY_LISTENER = new EnumMap<>(ApiMessageType.ListenerType.class); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java index 7802293c9bbfd..29ca3a17646d4 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java @@ -288,6 +288,8 @@ private static AbstractRequest doParseRequest(ApiKeys apiKey, short apiVersion, return DescribeTransactionsRequest.parse(buffer, apiVersion); case LIST_TRANSACTIONS: return ListTransactionsRequest.parse(buffer, apiVersion); + case ALLOCATE_PRODUCER_IDS: + return AllocateProducerIdsRequest.parse(buffer, apiVersion); default: throw new AssertionError(String.format("ApiKey %s is not currently handled in `parseRequest`, the " + "code should be updated to do so.", apiKey)); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java index 5f7b88f269405..47f2b3c7f3099 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java @@ -245,6 +245,8 @@ public static AbstractResponse parseResponse(ApiKeys apiKey, ByteBuffer response return DescribeTransactionsResponse.parse(responseBuffer, version); case LIST_TRANSACTIONS: return ListTransactionsResponse.parse(responseBuffer, version); + case ALLOCATE_PRODUCER_IDS: + return AllocateProducerIdsResponse.parse(responseBuffer, version); default: throw new AssertionError(String.format("ApiKey %s is not currently handled in `parseResponse`, the " + "code should be updated to do so.", apiKey)); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AllocateProducerIdsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/AllocateProducerIdsRequest.java new file mode 100644 index 0000000000000..7938f92df56d9 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/AllocateProducerIdsRequest.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.common.requests; + +import org.apache.kafka.common.message.AllocateProducerIdsRequestData; +import org.apache.kafka.common.message.AllocateProducerIdsResponseData; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.ByteBufferAccessor; +import org.apache.kafka.common.protocol.Errors; + +import java.nio.ByteBuffer; + +public class AllocateProducerIdsRequest extends AbstractRequest { + private final AllocateProducerIdsRequestData data; + + public AllocateProducerIdsRequest(AllocateProducerIdsRequestData data, short version) { + super(ApiKeys.ALLOCATE_PRODUCER_IDS, version); + this.data = data; + } + + @Override + public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) { + return new AllocateProducerIdsResponse(new AllocateProducerIdsResponseData() + .setThrottleTimeMs(throttleTimeMs) + .setErrorCode(Errors.forException(e).code())); + } + + @Override + public AllocateProducerIdsRequestData data() { + return data; + } + + public static class Builder extends AbstractRequest.Builder { + + private final AllocateProducerIdsRequestData data; + + public Builder(AllocateProducerIdsRequestData data) { + super(ApiKeys.ALLOCATE_PRODUCER_IDS); + this.data = data; + } + + @Override + public AllocateProducerIdsRequest build(short version) { + return new AllocateProducerIdsRequest(data, version); + } + + @Override + public String toString() { + return data.toString(); + } + } + + public static AllocateProducerIdsRequest parse(ByteBuffer buffer, short version) { + return new AllocateProducerIdsRequest(new AllocateProducerIdsRequestData( + new ByteBufferAccessor(buffer), version), version); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AllocateProducerIdsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/AllocateProducerIdsResponse.java new file mode 100644 index 0000000000000..5d48c39e8019d --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/AllocateProducerIdsResponse.java @@ -0,0 +1,63 @@ +/* + * 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.requests; + +import org.apache.kafka.common.message.AllocateProducerIdsResponseData; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.ByteBufferAccessor; +import org.apache.kafka.common.protocol.Errors; + +import java.nio.ByteBuffer; +import java.util.Collections; +import java.util.Map; + +public class AllocateProducerIdsResponse extends AbstractResponse { + + private final AllocateProducerIdsResponseData data; + + public AllocateProducerIdsResponse(AllocateProducerIdsResponseData data) { + super(ApiKeys.ALLOCATE_PRODUCER_IDS); + this.data = data; + } + + @Override + public AllocateProducerIdsResponseData data() { + return data; + } + + /** + * The number of each type of error in the response, including {@link Errors#NONE} and top-level errors as well as + * more specifically scoped errors (such as topic or partition-level errors). + * + * @return A count of errors. + */ + @Override + public Map errorCounts() { + return Collections.singletonMap(Errors.forCode(data.errorCode()), 1); + } + + @Override + public int throttleTimeMs() { + return data.throttleTimeMs(); + } + + public static AllocateProducerIdsResponse parse(ByteBuffer buffer, short version) { + return new AllocateProducerIdsResponse(new AllocateProducerIdsResponseData( + new ByteBufferAccessor(buffer), version)); + } +} diff --git a/clients/src/main/resources/common/message/AllocateProducerIdsRequest.json b/clients/src/main/resources/common/message/AllocateProducerIdsRequest.json new file mode 100644 index 0000000000000..0cfa494291a36 --- /dev/null +++ b/clients/src/main/resources/common/message/AllocateProducerIdsRequest.json @@ -0,0 +1,29 @@ +// 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 implie +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 67, + "type": "request", + "listeners": ["controller", "zkBroker"], + "name": "AllocateProducerIdsRequest", + "validVersions": "0", + "flexibleVersions": "0+", + "fields": [ + { "name": "BrokerId", "type": "int32", "versions": "0+", "entityType": "brokerId", + "about": "The ID of the requesting broker" }, + { "name": "BrokerEpoch", "type": "int64", "versions": "0+", "default": "-1", + "about": "The epoch of the requesting broker" } + ] +} diff --git a/clients/src/main/resources/common/message/AllocateProducerIdsResponse.json b/clients/src/main/resources/common/message/AllocateProducerIdsResponse.json new file mode 100644 index 0000000000000..c3e2ae1e3059a --- /dev/null +++ b/clients/src/main/resources/common/message/AllocateProducerIdsResponse.json @@ -0,0 +1,32 @@ +// 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. + +{ + "apiKey": 67, + "type": "response", + "name": "AllocateProducerIdsResponse", + "validVersions": "0", + "flexibleVersions": "0+", + "fields": [ + { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", + "about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." }, + { "name": "ErrorCode", "type": "int16", "versions": "0+", + "about": "The top level response error code" }, + { "name": "ProducerIdStart", "type": "int64", "versions": "0+", + "about": "The first producer ID in this range, inclusive"}, + { "name": "ProducerIdLen", "type": "int32", "versions": "0+", + "about": "The number of producer IDs in this range"} + ] +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index b17a6e8e6f7da..557f816d0f2cd 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -18,13 +18,13 @@ package kafka.controller import java.util import java.util.concurrent.TimeUnit - import kafka.admin.AdminOperationException import kafka.api._ import kafka.common._ import kafka.controller.KafkaController.AlterIsrCallback import kafka.cluster.Broker import kafka.controller.KafkaController.{AlterReassignmentsCallback, ElectLeadersCallback, ListReassignmentsCallback, UpdateFeaturesCallback} +import kafka.coordinator.transaction.{ProducerIdBlock, ProducerIdManager} import kafka.metrics.{KafkaMetricsGroup, KafkaTimer} import kafka.server._ import kafka.utils._ @@ -37,9 +37,8 @@ import org.apache.kafka.common.ElectionType import org.apache.kafka.common.KafkaException import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.errors.{BrokerNotAvailableException, ControllerMovedException, StaleBrokerEpochException} -import org.apache.kafka.common.message.{AlterIsrRequestData, AlterIsrResponseData} +import org.apache.kafka.common.message.{AllocateProducerIdsRequestData, AllocateProducerIdsResponseData, AlterIsrRequestData, AlterIsrResponseData, UpdateFeaturesRequestData} import org.apache.kafka.common.feature.{Features, FinalizedVersionRange} -import org.apache.kafka.common.message.UpdateFeaturesRequestData import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.{AbstractControlRequest, ApiError, LeaderAndIsrResponse, UpdateFeaturesRequest, UpdateMetadataResponse} @@ -2376,6 +2375,84 @@ class KafkaController(val config: KafkaConfig, } } + def allocateProducerIds(allocateProducerIdsRequest: AllocateProducerIdsRequestData, + callback: AllocateProducerIdsResponseData => Unit): Unit = { + + def eventManagerCallback(results: Either[Errors, (Long, Long)]): Unit = { + results match { + case Left(error) => callback.apply(new AllocateProducerIdsResponseData().setErrorCode(error.code)) + case Right((start, end)) => callback.apply( + new AllocateProducerIdsResponseData() + .setProducerIdStart(start) + .setProducerIdLen(Math.toIntExact(end - start + 1))) // ends are inclusive + } + } + eventManager.put(AllocateProducerIds(allocateProducerIdsRequest.brokerId, + allocateProducerIdsRequest.brokerEpoch, eventManagerCallback)) + } + + def processAllocateProducerIds(brokerId: Int, brokerEpoch: Long, callback: Either[Errors, (Long, Long)] => Unit): Unit = { + // Handle a few short-circuits + if (!isActive) { + callback.apply(Left(Errors.NOT_CONTROLLER)) + return + } + + val brokerEpochOpt = controllerContext.liveBrokerIdAndEpochs.get(brokerId) + if (brokerEpochOpt.isEmpty) { + warn(s"Ignoring AllocateProducerIds due to unknown broker $brokerId") + callback.apply(Left(Errors.STALE_BROKER_EPOCH)) + return + } + + if (!brokerEpochOpt.contains(brokerEpoch)) { + warn(s"Ignoring AllocateProducerIds due to stale broker epoch $brokerEpoch for broker $brokerId") + callback.apply(Left(Errors.STALE_BROKER_EPOCH)) + return + } + + // Get or create the existing PID block from ZK and attempt to update it. We retry in a loop here since other + // brokers may be generating PID blocks during a rolling upgrade + val maxAttempts = 1000 + for (_ <- 1 to maxAttempts) { + // refresh current producerId block from zookeeper again + val (dataOpt, zkVersion) = zkClient.getDataAndVersion(ProducerIdBlockZNode.path) + + // generate the new producerId block + val newProducerIdBlock = dataOpt match { + case Some(data) => + val currProducerIdBlock = ProducerIdBlockZNode.parseProducerIdBlockData(data) + debug(s"Read current producerId block $currProducerIdBlock, Zk path version $zkVersion") + + if (currProducerIdBlock.blockEndId > Long.MaxValue - ProducerIdManager.PidBlockSize) { + // we have exhausted all producerIds (wow!), treat it as a fatal error + fatal(s"Exhausted all producerIds as the next block's end producerId is will has exceeded long type limit (current block end producerId is ${currProducerIdBlock.blockEndId})") + callback.apply(Left(Errors.UNKNOWN_SERVER_ERROR)) + return + } + + ProducerIdBlock(brokerId, currProducerIdBlock.blockEndId + 1L, currProducerIdBlock.blockEndId + ProducerIdManager.PidBlockSize) + case None => + debug(s"There is no producerId block yet (Zk path version $zkVersion), creating the first block") + ProducerIdBlock(brokerId, 0L, ProducerIdManager.PidBlockSize - 1) + } + + val newProducerIdBlockData = ProducerIdBlockZNode.generateProducerIdBlockJson(newProducerIdBlock) + + // try to write the new producerId block into zookeeper + val (succeeded, version) = zkClient.conditionalUpdatePath(ProducerIdBlockZNode.path, newProducerIdBlockData, zkVersion, None) + + if (succeeded) { + debug(s"Acquired new producerId block $newProducerIdBlock by writing to Zk with path version $version") + callback.apply(Right((newProducerIdBlock.blockStartId, newProducerIdBlock.blockEndId))) + return + } + } + + debug(s"Failed to allocate new producerId block after $maxAttempts attempts.") + callback.apply(Left(Errors.UNKNOWN_SERVER_ERROR)) + } + private def processControllerChange(): Unit = { maybeResign() } @@ -2454,6 +2531,8 @@ class KafkaController(val config: KafkaConfig, processIsrChangeNotification() case AlterIsrReceived(brokerId, brokerEpoch, isrsToAlter, callback) => processAlterIsr(brokerId, brokerEpoch, isrsToAlter, callback) + case AllocateProducerIds(brokerId, brokerEpoch, callback) => + processAllocateProducerIds(brokerId, brokerEpoch, callback) case Startup => processStartup() } @@ -2747,6 +2826,11 @@ case class UpdateFeatures(request: UpdateFeaturesRequest, override def preempt(): Unit = {} } +case class AllocateProducerIds(brokerId: Int, brokerEpoch: Long, callback: Either[Errors, (Long, Long)] => Unit) extends ControllerEvent { + override def state: ControllerState = ControllerState.Idle + override def preempt(): Unit = {} +} + // Used only in test cases abstract class MockEvent(val state: ControllerState) extends ControllerEvent { diff --git a/core/src/main/scala/kafka/coordinator/transaction/ProducerIdManager.scala b/core/src/main/scala/kafka/coordinator/transaction/ProducerIdManager.scala index 8d3f542c2b725..cf7de03015b65 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/ProducerIdManager.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/ProducerIdManager.scala @@ -16,48 +16,33 @@ */ package kafka.coordinator.transaction -import java.nio.charset.StandardCharsets - -import kafka.utils.{Json, Logging} -import kafka.zk.{KafkaZkClient, ProducerIdBlockZNode} +import kafka.server.{BrokerToControllerChannelManager, ControllerRequestCompletionHandler} +import kafka.utils.Logging +import org.apache.kafka.clients.ClientResponse import org.apache.kafka.common.KafkaException +import org.apache.kafka.common.message.AllocateProducerIdsRequestData +import org.apache.kafka.common.protocol.Errors +import org.apache.kafka.common.requests.{AllocateProducerIdsRequest, AllocateProducerIdsResponse} -import scala.jdk.CollectionConverters._ +import java.util.concurrent.{ArrayBlockingQueue, TimeUnit} +import java.util.concurrent.atomic.AtomicBoolean +import scala.util.{Failure, Success, Try} /** * ProducerIdManager is the part of the transaction coordinator that provides ProducerIds in a unique way * such that the same producerId will not be assigned twice across multiple transaction coordinators. * - * ProducerIds are managed via ZooKeeper, where the latest producerId block is written on the corresponding ZK - * path by the manager who claims the block, where the written block_start and block_end are both inclusive. + * ProducerIds are managed by the controller. When requesting a new range of IDs, we are guaranteed to receive + * a unique block. The block start and block end are inclusive. */ -object ProducerIdManager extends Logging { - val CurrentVersion: Long = 1L - val PidBlockSize: Long = 1000L - - def generateProducerIdBlockJson(producerIdBlock: ProducerIdBlock): Array[Byte] = { - Json.encodeAsBytes(Map("version" -> CurrentVersion, - "broker" -> producerIdBlock.brokerId, - "block_start" -> producerIdBlock.blockStartId.toString, - "block_end" -> producerIdBlock.blockEndId.toString).asJava - ) - } +object ProducerIdManager { + val PidBlockSize = 1000L - def parseProducerIdBlockData(jsonData: Array[Byte]): ProducerIdBlock = { - try { - Json.parseBytes(jsonData).map(_.asJsonObject).flatMap { js => - val brokerId = js("broker").to[Int] - val blockStart = js("block_start").to[String].toLong - val blockEnd = js("block_end").to[String].toLong - Some(ProducerIdBlock(brokerId, blockStart, blockEnd)) - }.getOrElse(throw new KafkaException(s"Failed to parse the producerId block json $jsonData")) - } catch { - case e: java.lang.NumberFormatException => - // this should never happen: the written data has exceeded long type limit - fatal(s"Read jason data $jsonData contains producerIds that have exceeded long type limit") - throw e - } - } + // Max time to wait on the next PID to be available + val PidTimeoutMs = 60000 + + // Once we reach this percentage of PIDs consumed from the current block, trigger a fetch of the next block + val PidPrefetchThreshold = 0.90 } case class ProducerIdBlock(brokerId: Int, blockStartId: Long, blockEndId: Long) { @@ -68,6 +53,8 @@ case class ProducerIdBlock(brokerId: Int, blockStartId: Long, blockEndId: Long) producerIdBlockInfo.append(",blockEndProducerId:" + blockEndId + ")") producerIdBlockInfo.toString() } + + val blockSize: Long = blockEndId - blockStartId + 1 // inclusive } trait ProducerIdGenerator { @@ -75,87 +62,98 @@ trait ProducerIdGenerator { def shutdown() : Unit = {} } -class ProducerIdManager(val brokerId: Int, val zkClient: KafkaZkClient) extends ProducerIdGenerator with Logging { +class ProducerIdManager(brokerId: Int, + brokerEpochSupplier: () => Long, + controllerChannel: BrokerToControllerChannelManager) extends ProducerIdGenerator with Logging { this.logIdent = "[ProducerId Manager " + brokerId + "]: " - private var currentProducerIdBlock: ProducerIdBlock = null - private var nextProducerId: Long = -1L + private val nextProducerIdBlock = new ArrayBlockingQueue[Try[ProducerIdBlock]](1) + private val requestInFlight = new AtomicBoolean(false) + private var currentProducerIdBlock: ProducerIdBlock = ProducerIdBlock(brokerId, 0L, 0L) + private var nextProducerId: Long = 0L - // grab the first block of producerIds - this synchronized { - getNewProducerIdBlock() - nextProducerId = currentProducerIdBlock.blockStartId - } + // Send an initial request to get the first block + maybeRequestNextBlock() - private def getNewProducerIdBlock(): Unit = { - var zkWriteComplete = false - while (!zkWriteComplete) { - // refresh current producerId block from zookeeper again - val (dataOpt, zkVersion) = zkClient.getDataAndVersion(ProducerIdBlockZNode.path) - - // generate the new producerId block - currentProducerIdBlock = dataOpt match { - case Some(data) => - val currProducerIdBlock = ProducerIdManager.parseProducerIdBlockData(data) - debug(s"Read current producerId block $currProducerIdBlock, Zk path version $zkVersion") - - if (currProducerIdBlock.blockEndId > Long.MaxValue - ProducerIdManager.PidBlockSize) { - // we have exhausted all producerIds (wow!), treat it as a fatal error - fatal(s"Exhausted all producerIds as the next block's end producerId is will has exceeded long type limit (current block end producerId is ${currProducerIdBlock.blockEndId})") - throw new KafkaException("Have exhausted all producerIds.") - } + override def generateProducerId(): Long = { + this synchronized { + // Advance the ID + nextProducerId += 1 - ProducerIdBlock(brokerId, currProducerIdBlock.blockEndId + 1L, currProducerIdBlock.blockEndId + ProducerIdManager.PidBlockSize) - case None => - debug(s"There is no producerId block yet (Zk path version $zkVersion), creating the first block") - ProducerIdBlock(brokerId, 0L, ProducerIdManager.PidBlockSize - 1) + // Check if we need to fetch the next block + if (nextProducerId >= (currentProducerIdBlock.blockStartId + currentProducerIdBlock.blockSize * ProducerIdManager.PidPrefetchThreshold)) { + maybeRequestNextBlock() } - val newProducerIdBlockData = ProducerIdManager.generateProducerIdBlockJson(currentProducerIdBlock) - - // try to write the new producerId block into zookeeper - val (succeeded, version) = zkClient.conditionalUpdatePath(ProducerIdBlockZNode.path, - newProducerIdBlockData, zkVersion, Some(checkProducerIdBlockZkData)) - zkWriteComplete = succeeded - - if (zkWriteComplete) - info(s"Acquired new producerId block $currentProducerIdBlock by writing to Zk with path version $version") + // If we've exhausted the current block, grab the next block (waiting if necessary) + if (nextProducerId > currentProducerIdBlock.blockEndId) { + val block = nextProducerIdBlock.poll(ProducerIdManager.PidTimeoutMs, TimeUnit.MILLISECONDS) + if (block == null) { + throw new KafkaException(s"sTimed out waiting for next block of Producer IDs after ${ProducerIdManager.PidTimeoutMs}ms.") + } else { + block match { + case Success(nextBlock) => + currentProducerIdBlock = nextBlock + nextProducerId = currentProducerIdBlock.blockStartId + case Failure(t) => throw t + } + } + } + nextProducerId } } - private def checkProducerIdBlockZkData(zkClient: KafkaZkClient, path: String, expectedData: Array[Byte]): (Boolean, Int) = { - try { - val expectedPidBlock = ProducerIdManager.parseProducerIdBlockData(expectedData) - zkClient.getDataAndVersion(ProducerIdBlockZNode.path) match { - case (Some(data), zkVersion) => - val currProducerIdBLock = ProducerIdManager.parseProducerIdBlockData(data) - (currProducerIdBLock == expectedPidBlock, zkVersion) - case (None, _) => (false, -1) - } - } catch { - case e: Exception => - warn(s"Error while checking for producerId block Zk data on path $path: expected data " + - s"${new String(expectedData, StandardCharsets.UTF_8)}", e) - (false, -1) + + private def maybeRequestNextBlock(): Unit = { + if (nextProducerIdBlock.isEmpty && requestInFlight.compareAndSet(false, true)) { + sendRequest() } } - def generateProducerId(): Long = { - this synchronized { - // grab a new block of producerIds if this block has been exhausted - if (nextProducerId > currentProducerIdBlock.blockEndId) { - getNewProducerIdBlock() - nextProducerId = currentProducerIdBlock.blockStartId + 1 - } else { - nextProducerId += 1 + private[transaction] def sendRequest(): Unit = { + val message = new AllocateProducerIdsRequestData() + .setBrokerEpoch(brokerEpochSupplier.apply()) + .setBrokerId(brokerId) + + val request = new AllocateProducerIdsRequest.Builder(message) + debug("Requesting next Producer ID block") + controllerChannel.sendRequest(request, new ControllerRequestCompletionHandler() { + override def onComplete(response: ClientResponse): Unit = { + val message = response.responseBody().asInstanceOf[AllocateProducerIdsResponse] + handleAllocateProducerIdsResponse(message) } - nextProducerId - 1 - } + override def onTimeout(): Unit = { + warn("Encountered unexpected timeout when requesting AllocateProducerIds from the controller, trying again.") + requestInFlight.set(false) + maybeRequestNextBlock() + } + }) } - override def shutdown(): Unit = { - info(s"Shutdown complete: last producerId assigned $nextProducerId") + private[transaction] def handleAllocateProducerIdsResponse(response: AllocateProducerIdsResponse): Unit = { + requestInFlight.set(false) + val data = response.data + Errors.forCode(data.errorCode()) match { + case Errors.NONE => + debug(s"Got next producer ID block from controller $data") + // Do some sanity checks on the response + if (data.producerIdStart() < currentProducerIdBlock.blockEndId) { + nextProducerIdBlock.put(Failure(new KafkaException( + s"Producer ID block is not monotonic with current block: current=$currentProducerIdBlock response=$data"))) + } else if (data.producerIdStart() < 0 || data.producerIdLen() < 0 || data.producerIdStart() > Long.MaxValue - data.producerIdLen()) { + nextProducerIdBlock.put(Failure(new KafkaException(s"Producer ID block includes invalid ID range: $data"))) + } else { + nextProducerIdBlock.put( + Success(ProducerIdBlock(brokerId, data.producerIdStart(), data.producerIdStart() + data.producerIdLen() - 1))) + } + case Errors.STALE_BROKER_EPOCH => + warn("Our broker epoch was stale, trying again.") + maybeRequestNextBlock() + case e: Errors => + warn("Had an unknown error from the controller, giving up.") + nextProducerIdBlock.put(Failure(e.exception())) + } } } diff --git a/core/src/main/scala/kafka/network/RequestConvertToJson.scala b/core/src/main/scala/kafka/network/RequestConvertToJson.scala index c0fbd68e75bf2..bb8e327b1890a 100644 --- a/core/src/main/scala/kafka/network/RequestConvertToJson.scala +++ b/core/src/main/scala/kafka/network/RequestConvertToJson.scala @@ -29,6 +29,7 @@ object RequestConvertToJson { request match { case req: AddOffsetsToTxnRequest => AddOffsetsToTxnRequestDataJsonConverter.write(req.data, request.version) case req: AddPartitionsToTxnRequest => AddPartitionsToTxnRequestDataJsonConverter.write(req.data, request.version) + case req: AllocateProducerIdsRequest => AllocateProducerIdsRequestDataJsonConverter.write(req.data, request.version) case req: AlterClientQuotasRequest => AlterClientQuotasRequestDataJsonConverter.write(req.data, request.version) case req: AlterConfigsRequest => AlterConfigsRequestDataJsonConverter.write(req.data, request.version) case req: AlterIsrRequest => AlterIsrRequestDataJsonConverter.write(req.data, request.version) @@ -103,6 +104,7 @@ object RequestConvertToJson { response match { case res: AddOffsetsToTxnResponse => AddOffsetsToTxnResponseDataJsonConverter.write(res.data, version) case res: AddPartitionsToTxnResponse => AddPartitionsToTxnResponseDataJsonConverter.write(res.data, version) + case res: AllocateProducerIdsResponse => AllocateProducerIdsResponseDataJsonConverter.write(res.data, version) case res: AlterClientQuotasResponse => AlterClientQuotasResponseDataJsonConverter.write(res.data, version) case res: AlterConfigsResponse => AlterConfigsResponseDataJsonConverter.write(res.data, version) case res: AlterIsrResponse => AlterIsrResponseDataJsonConverter.write(res.data, version) diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 5aad1f6cc7d1b..81809b5f7188f 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -222,6 +222,7 @@ class KafkaApis(val requestChannel: RequestChannel, case ApiKeys.UNREGISTER_BROKER => maybeForwardToController(request, handleUnregisterBrokerRequest) case ApiKeys.DESCRIBE_TRANSACTIONS => handleDescribeTransactionsRequest(request) case ApiKeys.LIST_TRANSACTIONS => handleListTransactionsRequest(request) + case ApiKeys.ALLOCATE_PRODUCER_IDS => handleAllocateProducerIdsRequest(request) case _ => throw new IllegalStateException(s"No handler for request api key ${request.header.apiKey}") } } catch { @@ -3349,6 +3350,21 @@ class KafkaApis(val requestChannel: RequestChannel, new ListTransactionsResponse(response.setThrottleTimeMs(requestThrottleMs))) } + def handleAllocateProducerIdsRequest(request: RequestChannel.Request): Unit = { + val zkSupport = metadataSupport.requireZkOrThrow(KafkaApis.shouldNeverReceive(request)) + authHelper.authorizeClusterOperation(request, CLUSTER_ACTION) + + val allocateProducerIdsRequest = request.body[AllocateProducerIdsRequest] + + if (!zkSupport.controller.isActive) + requestHelper.sendResponseExemptThrottle(request, allocateProducerIdsRequest.getErrorResponse( + AbstractResponse.DEFAULT_THROTTLE_TIME, Errors.NOT_CONTROLLER.exception)) + else + zkSupport.controller.allocateProducerIds(allocateProducerIdsRequest.data, producerIdsResponse => + requestHelper.sendResponseExemptThrottle(request, new AllocateProducerIdsResponse(producerIdsResponse)) + ) + } + private def updateRecordConversionStats(request: RequestChannel.Request, tp: TopicPartition, conversionStats: RecordConversionStats): Unit = { diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 47c8aee038f10..9c4098cc54e1d 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -257,17 +257,18 @@ class KafkaServer( tokenCache = new DelegationTokenCache(ScramMechanism.mechanismNames) credentialProvider = new CredentialProvider(ScramMechanism.mechanismNames, tokenCache) + val brokerToControllerManager = BrokerToControllerChannelManager( + controllerNodeProvider = MetadataCacheControllerNodeProvider(config, metadataCache), + time = time, + metrics = metrics, + config = config, + channelName = "forwarding", + threadNamePrefix = threadNamePrefix, + retryTimeoutMs = config.requestTimeoutMs.longValue) + brokerToControllerManager.start() + /* start forwarding manager */ if (enableForwarding) { - val brokerToControllerManager = BrokerToControllerChannelManager( - controllerNodeProvider = MetadataCacheControllerNodeProvider(config, metadataCache), - time = time, - metrics = metrics, - config = config, - channelName = "forwarding", - threadNamePrefix = threadNamePrefix, - retryTimeoutMs = config.requestTimeoutMs.longValue) - brokerToControllerManager.start() this.forwardingManager = Some(ForwardingManager(brokerToControllerManager)) clientToControllerChannelManager = Some(brokerToControllerManager) } @@ -333,7 +334,8 @@ class KafkaServer( /* start transaction coordinator, with a separate background thread scheduler for transaction expiration and log loading */ // Hardcode Time.SYSTEM for now as some Streams tests fail otherwise, it would be good to fix the underlying issue transactionCoordinator = TransactionCoordinator(config, replicaManager, new KafkaScheduler(threads = 1, threadNamePrefix = "transaction-log-manager-"), - () => new ProducerIdManager(config.brokerId, zkClient), metrics, metadataCache, Time.SYSTEM) + () => new ProducerIdManager(config.brokerId, brokerEpochSupplier = () => kafkaController.brokerEpoch, brokerToControllerManager), + metrics, metadataCache, Time.SYSTEM) transactionCoordinator.startup( () => zkClient.getTopicPartitionCount(Topic.TRANSACTION_STATE_TOPIC_NAME).getOrElse(config.transactionTopicPartitions)) diff --git a/core/src/main/scala/kafka/zk/ZkData.scala b/core/src/main/scala/kafka/zk/ZkData.scala index 4941ece5163cc..7246dc27b9fd3 100644 --- a/core/src/main/scala/kafka/zk/ZkData.scala +++ b/core/src/main/scala/kafka/zk/ZkData.scala @@ -19,13 +19,13 @@ package kafka.zk import java.nio.charset.StandardCharsets.UTF_8 import java.util import java.util.Properties - import com.fasterxml.jackson.annotation.JsonProperty import com.fasterxml.jackson.core.JsonProcessingException import kafka.api.{ApiVersion, KAFKA_0_10_0_IV1, KAFKA_2_7_IV0, LeaderAndIsr} import kafka.cluster.{Broker, EndPoint} import kafka.common.{NotificationHandler, ZkNodeChangeNotificationListener} import kafka.controller.{IsrChangeNotificationHandler, LeaderIsrAndControllerEpoch, ReplicaAssignment} +import kafka.coordinator.transaction.ProducerIdBlock import kafka.security.authorizer.AclAuthorizer.VersionedAcls import kafka.security.authorizer.AclEntry import kafka.server.{ConfigType, DelegationTokenManager} @@ -766,7 +766,33 @@ object BrokerSequenceIdZNode { } object ProducerIdBlockZNode { + val CurrentVersion: Long = 1L + def path = "/latest_producer_id_block" + + def generateProducerIdBlockJson(producerIdBlock: ProducerIdBlock): Array[Byte] = { + Json.encodeAsBytes(Map("version" -> CurrentVersion, + "broker" -> producerIdBlock.brokerId, + "block_start" -> producerIdBlock.blockStartId.toString, + "block_end" -> producerIdBlock.blockEndId.toString).asJava + ) + } + + def parseProducerIdBlockData(jsonData: Array[Byte]): ProducerIdBlock = { + val jsonDataAsString = jsonData.map(_.toChar).mkString + try { + Json.parseBytes(jsonData).map(_.asJsonObject).flatMap { js => + val brokerId = js("broker").to[Int] + val blockStart = js("block_start").to[String].toLong + val blockEnd = js("block_end").to[String].toLong + Some(ProducerIdBlock(brokerId, blockStart, blockEnd)) + }.getOrElse(throw new KafkaException(s"Failed to parse the producerId block json $jsonDataAsString")) + } catch { + case e: java.lang.NumberFormatException => + // this should never happen: the written data has exceeded long type limit + throw new KafkaException(s"Read jason data $jsonDataAsString contains producerIds that have exceeded long type limit", e) + } + } } object DelegationTokenAuthZNode { @@ -937,6 +963,9 @@ object FeatureZNode { } } + + + object ZkData { // Important: it is necessary to add any new top level Zookeeper path to the Seq diff --git a/core/src/test/java/kafka/test/ClusterInstance.java b/core/src/test/java/kafka/test/ClusterInstance.java index cac986fdc317c..021db5a34381e 100644 --- a/core/src/test/java/kafka/test/ClusterInstance.java +++ b/core/src/test/java/kafka/test/ClusterInstance.java @@ -94,4 +94,6 @@ default Admin createAdminClient() { void start(); void stop(); + + void rollingBrokerRestart(); } diff --git a/core/src/test/java/kafka/test/junit/RaftClusterInvocationContext.java b/core/src/test/java/kafka/test/junit/RaftClusterInvocationContext.java index aa3fa057b080d..ce9f60d24ae02 100644 --- a/core/src/test/java/kafka/test/junit/RaftClusterInvocationContext.java +++ b/core/src/test/java/kafka/test/junit/RaftClusterInvocationContext.java @@ -192,5 +192,10 @@ public void stop() { } } } + + @Override + public void rollingBrokerRestart() { + throw new UnsupportedOperationException("Restarting Raft servers is not yet supported."); + } } } \ No newline at end of file diff --git a/core/src/test/java/kafka/test/junit/ZkClusterInvocationContext.java b/core/src/test/java/kafka/test/junit/ZkClusterInvocationContext.java index 62cc80df27771..996118aedb269 100644 --- a/core/src/test/java/kafka/test/junit/ZkClusterInvocationContext.java +++ b/core/src/test/java/kafka/test/junit/ZkClusterInvocationContext.java @@ -248,5 +248,17 @@ public void stop() { clusterReference.get().tearDown(); } } + + @Override + public void rollingBrokerRestart() { + if (started.get()) { + for (int i = 0; i < clusterReference.get().brokerCount(); i++) { + clusterReference.get().killBroker(i); + } + clusterReference.get().restartDeadBrokers(); + } else { + throw new IllegalStateException("Tried to restart brokers but the cluster has not been started!"); + } + } } } diff --git a/core/src/test/scala/integration/kafka/coordinator/transaction/ProducerIdsIntegrationTest.scala b/core/src/test/scala/integration/kafka/coordinator/transaction/ProducerIdsIntegrationTest.scala new file mode 100644 index 0000000000000..7f4f2949cb40f --- /dev/null +++ b/core/src/test/scala/integration/kafka/coordinator/transaction/ProducerIdsIntegrationTest.scala @@ -0,0 +1,82 @@ +/** + * 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 integration.kafka.coordinator.transaction + +import integration.kafka.server.IntegrationTestUtils +import kafka.network.SocketServer +import kafka.server.KafkaConfig +import kafka.test.annotation.{ClusterTest, Type} +import kafka.test.junit.ClusterTestExtensions +import kafka.test.{ClusterConfig, ClusterInstance} +import org.apache.kafka.common.message.InitProducerIdRequestData +import org.apache.kafka.common.network.ListenerName +import org.apache.kafka.common.record.RecordBatch +import org.apache.kafka.common.requests.{InitProducerIdRequest, InitProducerIdResponse} +import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue} +import org.junit.jupiter.api.BeforeEach +import org.junit.jupiter.api.extension.ExtendWith + +import java.util.stream.{Collectors, IntStream} + +@ExtendWith(value = Array(classOf[ClusterTestExtensions])) +class ProducerIdsIntegrationTest { + + @BeforeEach + def setup(clusterConfig: ClusterConfig): Unit = { + clusterConfig.serverProperties().put(KafkaConfig.TransactionsTopicPartitionsProp, "1") + clusterConfig.serverProperties().put(KafkaConfig.TransactionsTopicReplicationFactorProp, "3") + } + + @ClusterTest(clusterType = Type.ZK, brokers = 3) + def testNonOverlapping(clusterInstance: ClusterInstance): Unit = { + val ids = clusterInstance.brokerSocketServers().stream().flatMap( broker => { + IntStream.range(0, 1001).parallel().mapToObj( _ => nextProducerId(broker, clusterInstance.clientListener())) + }).collect(Collectors.toSet[Long]) + + assertEquals(3003, ids.size) + + val expectedIds = Set(0L, 999L, 1000L, 1999L, 2000L, 2999L, 3000L, 4000L, 5000L) + val idsAsString = expectedIds.mkString(", ") + expectedIds.foreach { id => + assertTrue(ids.contains(id), s"Expected to see $id in $idsAsString") + } + } + + @ClusterTest(clusterType = Type.ZK, brokers = 1) + def testNewBlockOnRestart(clusterInstance: ClusterInstance): Unit = { + val id0 = nextProducerId(clusterInstance.anyBrokerSocketServer(), clusterInstance.clientListener()) + clusterInstance.rollingBrokerRestart() + val id1 = nextProducerId(clusterInstance.anyBrokerSocketServer(), clusterInstance.clientListener()) + assertEquals(0, id0) + assertEquals(1000, id1) + } + + private def nextProducerId(broker: SocketServer, listener: ListenerName): Long = { + val data = new InitProducerIdRequestData() + .setProducerEpoch(RecordBatch.NO_PRODUCER_EPOCH) + .setProducerId(RecordBatch.NO_PRODUCER_ID) + .setTransactionalId(null) + .setTransactionTimeoutMs(10) + val request = new InitProducerIdRequest.Builder(data).build() + + val response = IntegrationTestUtils.connectAndReceive[InitProducerIdResponse](request, + destination = broker, + listenerName = listener) + response.data().producerId() + } +} diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/ProducerIdManagerTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/ProducerIdManagerTest.scala index be3e8144f6859..8deca4e01c6ce 100644 --- a/core/src/test/scala/unit/kafka/coordinator/transaction/ProducerIdManagerTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/transaction/ProducerIdManagerTest.scala @@ -16,71 +16,72 @@ */ package kafka.coordinator.transaction -import kafka.zk.KafkaZkClient +import kafka.server.BrokerToControllerChannelManager import org.apache.kafka.common.KafkaException -import org.easymock.{Capture, EasyMock} -import org.junit.jupiter.api.{AfterEach, Test} +import org.apache.kafka.common.message.AllocateProducerIdsResponseData +import org.apache.kafka.common.protocol.Errors +import org.apache.kafka.common.requests.AllocateProducerIdsResponse +import org.easymock.EasyMock import org.junit.jupiter.api.Assertions._ +import org.junit.jupiter.api.Test +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.{EnumSource, ValueSource} -class ProducerIdManagerTest { - - private val zkClient: KafkaZkClient = EasyMock.createNiceMock(classOf[KafkaZkClient]) +import java.util.stream.IntStream - @AfterEach - def tearDown(): Unit = { - EasyMock.reset(zkClient) - } - - @Test - def testGetProducerId(): Unit = { - var zkVersion: Option[Int] = None - var data: Array[Byte] = null - EasyMock.expect(zkClient.getDataAndVersion(EasyMock.anyString)).andAnswer(() => - zkVersion.map(Some(data) -> _).getOrElse(None, 0)).anyTimes() +class ProducerIdManagerTest { - val capturedVersion: Capture[Int] = EasyMock.newCapture() - val capturedData: Capture[Array[Byte]] = EasyMock.newCapture() - EasyMock.expect(zkClient.conditionalUpdatePath(EasyMock.anyString(), - EasyMock.capture(capturedData), - EasyMock.capture(capturedVersion), - EasyMock.anyObject[Option[(KafkaZkClient, String, Array[Byte]) => (Boolean, Int)]]) - ).andAnswer(() => { - val newZkVersion = capturedVersion.getValue + 1 - zkVersion = Some(newZkVersion) - data = capturedData.getValue - (true, newZkVersion) - }).anyTimes() + var brokerToController: BrokerToControllerChannelManager = EasyMock.niceMock(classOf[BrokerToControllerChannelManager]) - EasyMock.replay(zkClient) + // Mutable test implementation that lets us easily set the idStart and error + class MockProducerIdManager(val brokerId: Int, var idStart: Long, val idLen: Int, var error: Errors = Errors.NONE) + extends ProducerIdManager(brokerId, () => 1, brokerToController) { - val manager1 = new ProducerIdManager(0, zkClient) - val manager2 = new ProducerIdManager(1, zkClient) + override private[transaction] def sendRequest(): Unit = { + if (error == Errors.NONE) { + handleAllocateProducerIdsResponse(new AllocateProducerIdsResponse( + new AllocateProducerIdsResponseData().setProducerIdStart(idStart).setProducerIdLen(idLen))) + idStart += idLen + } else { + handleAllocateProducerIdsResponse(new AllocateProducerIdsResponse( + new AllocateProducerIdsResponseData().setErrorCode(error.code))) + } + } + } - val pid1 = manager1.generateProducerId() - val pid2 = manager2.generateProducerId() + @ParameterizedTest + @ValueSource(ints = Array(1, 2, 1000)) + def testDifferentBlockLengths(idBlockLen: Int): Unit = { + val manager = new MockProducerIdManager(0, 0, idBlockLen) - assertEquals(0, pid1) - assertEquals(ProducerIdManager.PidBlockSize, pid2) + IntStream.range(0, idBlockLen * 2).forEach { i => + assertEquals(i, manager.generateProducerId()) + } + } - for (i <- 1L until ProducerIdManager.PidBlockSize) - assertEquals(pid1 + i, manager1.generateProducerId()) + @ParameterizedTest + @EnumSource(value = classOf[Errors], names = Array("UNKNOWN_SERVER_ERROR", "INVALID_REQUEST")) + def testUnrecoverableErrors(error: Errors): Unit = { + val manager = new MockProducerIdManager(0, 0, 1) + assertEquals(0, manager.generateProducerId()) - for (i <- 1L until ProducerIdManager.PidBlockSize) - assertEquals(pid2 + i, manager2.generateProducerId()) + manager.error = error + assertThrows(classOf[Throwable], () => manager.generateProducerId()) - assertEquals(pid2 + ProducerIdManager.PidBlockSize, manager1.generateProducerId()) - assertEquals(pid2 + ProducerIdManager.PidBlockSize * 2, manager2.generateProducerId()) + manager.error = Errors.NONE + assertEquals(1, manager.generateProducerId()) } @Test - def testExceedProducerIdLimit(): Unit = { - EasyMock.expect(zkClient.getDataAndVersion(EasyMock.anyString)).andAnswer(() => { - val json = ProducerIdManager.generateProducerIdBlockJson( - ProducerIdBlock(0, Long.MaxValue - ProducerIdManager.PidBlockSize, Long.MaxValue)) - (Some(json), 0) - }).anyTimes() - EasyMock.replay(zkClient) - assertThrows(classOf[KafkaException], () => new ProducerIdManager(0, zkClient)) + def testInvalidRanges(): Unit = { + var manager = new MockProducerIdManager(0, -1, 10) + assertThrows(classOf[KafkaException], () => manager.generateProducerId()) + + manager = new MockProducerIdManager(0, 0, -1) + assertThrows(classOf[KafkaException], () => manager.generateProducerId()) + + manager = new MockProducerIdManager(0, Long.MaxValue-1, 10) + assertThrows(classOf[KafkaException], () => manager.generateProducerId()) } } diff --git a/metadata/src/main/resources/common/metadata/ProducerIdRecord.json b/metadata/src/main/resources/common/metadata/ProducerIdRecord.json new file mode 100644 index 0000000000000..ea6e16f3292a4 --- /dev/null +++ b/metadata/src/main/resources/common/metadata/ProducerIdRecord.json @@ -0,0 +1,29 @@ +// 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. + +{ + "apiKey": 15, + "type": "metadata", + "name": "ProducerIdRecord", + "validVersions": "0", + "fields": [ + { "name": "BrokerId", "type": "int32", "versions": "0+", "entityType": "brokerId", + "about": "The ID of the requesting broker" }, + { "name": "BrokerEpoch", "type": "int64", "versions": "0+", "default": "-1", + "about": "The epoch of the requesting broker" }, + { "name": "ProducerIdEnd", "type": "int64", "versions": "0+", + "about": "The highest producer ID that has been generated"} + ] +} From f8ef8a4ef6d3942a1be1b65bf227cb53b7a80037 Mon Sep 17 00:00:00 2001 From: David Arthur Date: Thu, 8 Apr 2021 13:38:54 -0400 Subject: [PATCH 02/20] Fix ApiKeysTest throttling test --- .../test/java/org/apache/kafka/common/protocol/ApiKeysTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/clients/src/test/java/org/apache/kafka/common/protocol/ApiKeysTest.java b/clients/src/test/java/org/apache/kafka/common/protocol/ApiKeysTest.java index 17d2e1ce26e43..3c66b211bec4f 100644 --- a/clients/src/test/java/org/apache/kafka/common/protocol/ApiKeysTest.java +++ b/clients/src/test/java/org/apache/kafka/common/protocol/ApiKeysTest.java @@ -62,7 +62,7 @@ public void testAlterIsrIsClusterAction() { public void testResponseThrottleTime() { Set authenticationKeys = EnumSet.of(ApiKeys.SASL_HANDSHAKE, ApiKeys.SASL_AUTHENTICATE); // Newer protocol apis include throttle time ms even for cluster actions - Set clusterActionsWithThrottleTimeMs = EnumSet.of(ApiKeys.ALTER_ISR); + Set clusterActionsWithThrottleTimeMs = EnumSet.of(ApiKeys.ALTER_ISR, ApiKeys.ALLOCATE_PRODUCER_IDS); for (ApiKeys apiKey: ApiKeys.zkBrokerApis()) { Schema responseSchema = apiKey.messageType.responseSchemas()[apiKey.latestVersion()]; BoundField throttleTimeField = responseSchema.get("throttle_time_ms"); From 2c802520642671119f1583df755529fc9adab129 Mon Sep 17 00:00:00 2001 From: David Arthur Date: Thu, 8 Apr 2021 13:53:35 -0400 Subject: [PATCH 03/20] Actually use throttling in KafkaApis for AllocateProducerIds RPC --- core/src/main/scala/kafka/server/KafkaApis.scala | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 81809b5f7188f..641e9592a980c 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -3357,11 +3357,12 @@ class KafkaApis(val requestChannel: RequestChannel, val allocateProducerIdsRequest = request.body[AllocateProducerIdsRequest] if (!zkSupport.controller.isActive) - requestHelper.sendResponseExemptThrottle(request, allocateProducerIdsRequest.getErrorResponse( - AbstractResponse.DEFAULT_THROTTLE_TIME, Errors.NOT_CONTROLLER.exception)) + requestHelper.sendResponseMaybeThrottle(request, throttleTimeMs => + allocateProducerIdsRequest.getErrorResponse(throttleTimeMs, Errors.NOT_CONTROLLER.exception)) else zkSupport.controller.allocateProducerIds(allocateProducerIdsRequest.data, producerIdsResponse => - requestHelper.sendResponseExemptThrottle(request, new AllocateProducerIdsResponse(producerIdsResponse)) + requestHelper.sendResponseMaybeThrottle(request, throttleTimeMs => + new AllocateProducerIdsResponse(producerIdsResponse.setThrottleTimeMs(throttleTimeMs))) ) } From 8243d9cdcc6d6fa9114e73903939df6fa94c49b1 Mon Sep 17 00:00:00 2001 From: David Arthur Date: Thu, 8 Apr 2021 17:08:00 -0400 Subject: [PATCH 04/20] Rely on request timeouts --- .../kafka/controller/KafkaController.scala | 12 ++++------ .../transaction/ProducerIdManager.scala | 24 +++++++++---------- .../main/scala/kafka/server/KafkaServer.scala | 9 +++++-- .../transaction/ProducerIdManagerTest.scala | 2 +- 4 files changed, 25 insertions(+), 22 deletions(-) diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index 557f816d0f2cd..01e23087e53ec 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -2413,8 +2413,8 @@ class KafkaController(val config: KafkaConfig, // Get or create the existing PID block from ZK and attempt to update it. We retry in a loop here since other // brokers may be generating PID blocks during a rolling upgrade - val maxAttempts = 1000 - for (_ <- 1 to maxAttempts) { + var zkWriteComplete = false + while (!zkWriteComplete) { // refresh current producerId block from zookeeper again val (dataOpt, zkVersion) = zkClient.getDataAndVersion(ProducerIdBlockZNode.path) @@ -2441,16 +2441,14 @@ class KafkaController(val config: KafkaConfig, // try to write the new producerId block into zookeeper val (succeeded, version) = zkClient.conditionalUpdatePath(ProducerIdBlockZNode.path, newProducerIdBlockData, zkVersion, None) + zkWriteComplete = succeeded - if (succeeded) { - debug(s"Acquired new producerId block $newProducerIdBlock by writing to Zk with path version $version") + if (zkWriteComplete) { + info(s"Acquired new producerId block $newProducerIdBlock by writing to Zk with path version $version") callback.apply(Right((newProducerIdBlock.blockStartId, newProducerIdBlock.blockEndId))) return } } - - debug(s"Failed to allocate new producerId block after $maxAttempts attempts.") - callback.apply(Left(Errors.UNKNOWN_SERVER_ERROR)) } private def processControllerChange(): Unit = { diff --git a/core/src/main/scala/kafka/coordinator/transaction/ProducerIdManager.scala b/core/src/main/scala/kafka/coordinator/transaction/ProducerIdManager.scala index cf7de03015b65..036866139648c 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/ProducerIdManager.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/ProducerIdManager.scala @@ -38,9 +38,6 @@ import scala.util.{Failure, Success, Try} object ProducerIdManager { val PidBlockSize = 1000L - // Max time to wait on the next PID to be available - val PidTimeoutMs = 60000 - // Once we reach this percentage of PIDs consumed from the current block, trigger a fetch of the next block val PidPrefetchThreshold = 0.90 } @@ -64,7 +61,8 @@ trait ProducerIdGenerator { class ProducerIdManager(brokerId: Int, brokerEpochSupplier: () => Long, - controllerChannel: BrokerToControllerChannelManager) extends ProducerIdGenerator with Logging { + controllerChannel: BrokerToControllerChannelManager, + maxWaitMs: Int) extends ProducerIdGenerator with Logging { this.logIdent = "[ProducerId Manager " + brokerId + "]: " @@ -78,7 +76,6 @@ class ProducerIdManager(brokerId: Int, override def generateProducerId(): Long = { this synchronized { - // Advance the ID nextProducerId += 1 // Check if we need to fetch the next block @@ -88,9 +85,9 @@ class ProducerIdManager(brokerId: Int, // If we've exhausted the current block, grab the next block (waiting if necessary) if (nextProducerId > currentProducerIdBlock.blockEndId) { - val block = nextProducerIdBlock.poll(ProducerIdManager.PidTimeoutMs, TimeUnit.MILLISECONDS) + val block = nextProducerIdBlock.poll(maxWaitMs, TimeUnit.MILLISECONDS) if (block == null) { - throw new KafkaException(s"sTimed out waiting for next block of Producer IDs after ${ProducerIdManager.PidTimeoutMs}ms.") + throw Errors.REQUEST_TIMED_OUT.exception("Timed out waiting for next producer ID block") } else { block match { case Success(nextBlock) => @@ -124,11 +121,7 @@ class ProducerIdManager(brokerId: Int, handleAllocateProducerIdsResponse(message) } - override def onTimeout(): Unit = { - warn("Encountered unexpected timeout when requesting AllocateProducerIds from the controller, trying again.") - requestInFlight.set(false) - maybeRequestNextBlock() - } + override def onTimeout(): Unit = handleTimeout() }) } @@ -156,4 +149,11 @@ class ProducerIdManager(brokerId: Int, nextProducerIdBlock.put(Failure(e.exception())) } } + + private[transaction] def handleTimeout(): Unit = { + warn("Timed out when requesting AllocateProducerIds from the controller.") + requestInFlight.set(false) + nextProducerIdBlock.put(Failure(Errors.REQUEST_TIMED_OUT.exception)) + maybeRequestNextBlock() + } } diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 9c4098cc54e1d..b01bed3e4cd22 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -333,9 +333,14 @@ class KafkaServer( /* start transaction coordinator, with a separate background thread scheduler for transaction expiration and log loading */ // Hardcode Time.SYSTEM for now as some Streams tests fail otherwise, it would be good to fix the underlying issue + val producerIdManagerSupplier = () => new ProducerIdManager( + config.brokerId, + brokerEpochSupplier = () => kafkaController.brokerEpoch, + brokerToControllerManager, + config.requestTimeoutMs + ) transactionCoordinator = TransactionCoordinator(config, replicaManager, new KafkaScheduler(threads = 1, threadNamePrefix = "transaction-log-manager-"), - () => new ProducerIdManager(config.brokerId, brokerEpochSupplier = () => kafkaController.brokerEpoch, brokerToControllerManager), - metrics, metadataCache, Time.SYSTEM) + producerIdManagerSupplier, metrics, metadataCache, Time.SYSTEM) transactionCoordinator.startup( () => zkClient.getTopicPartitionCount(Topic.TRANSACTION_STATE_TOPIC_NAME).getOrElse(config.transactionTopicPartitions)) diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/ProducerIdManagerTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/ProducerIdManagerTest.scala index 8deca4e01c6ce..95c25a6fe5884 100644 --- a/core/src/test/scala/unit/kafka/coordinator/transaction/ProducerIdManagerTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/transaction/ProducerIdManagerTest.scala @@ -35,7 +35,7 @@ class ProducerIdManagerTest { // Mutable test implementation that lets us easily set the idStart and error class MockProducerIdManager(val brokerId: Int, var idStart: Long, val idLen: Int, var error: Errors = Errors.NONE) - extends ProducerIdManager(brokerId, () => 1, brokerToController) { + extends ProducerIdManager(brokerId, () => 1, brokerToController, 100) { override private[transaction] def sendRequest(): Unit = { if (error == Errors.NONE) { From 16cc8bf539acf67dd80daf1ee12d4e539b02ce89 Mon Sep 17 00:00:00 2001 From: David Arthur Date: Tue, 11 May 2021 16:08:50 -0400 Subject: [PATCH 05/20] Move data class to common module. This ProducerIdsBlock class is used by the broker, controller, and eventually the KRaft controller. Moving to the server-common module lets us reuse it. --- .../kafka/controller/KafkaController.scala | 12 ++-- .../transaction/ProducerIdManager.scala | 29 +++------ core/src/main/scala/kafka/zk/ZkData.scala | 12 ++-- .../ProducerIdsIntegrationTest.scala | 4 +- .../transaction/ProducerIdManagerTest.scala | 6 +- .../kafka/server/common/ProducerIdsBlock.java | 61 +++++++++++++++++++ 6 files changed, 86 insertions(+), 38 deletions(-) create mode 100644 server-common/src/main/java/org/apache/kafka/server/common/ProducerIdsBlock.java diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index 01e23087e53ec..3a2906c2fb3a7 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -24,7 +24,6 @@ import kafka.common._ import kafka.controller.KafkaController.AlterIsrCallback import kafka.cluster.Broker import kafka.controller.KafkaController.{AlterReassignmentsCallback, ElectLeadersCallback, ListReassignmentsCallback, UpdateFeaturesCallback} -import kafka.coordinator.transaction.{ProducerIdBlock, ProducerIdManager} import kafka.metrics.{KafkaMetricsGroup, KafkaTimer} import kafka.server._ import kafka.utils._ @@ -43,6 +42,7 @@ import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.{AbstractControlRequest, ApiError, LeaderAndIsrResponse, UpdateFeaturesRequest, UpdateMetadataResponse} import org.apache.kafka.common.utils.{Time, Utils} +import org.apache.kafka.server.common.ProducerIdsBlock import org.apache.zookeeper.KeeperException import org.apache.zookeeper.KeeperException.Code @@ -2424,17 +2424,17 @@ class KafkaController(val config: KafkaConfig, val currProducerIdBlock = ProducerIdBlockZNode.parseProducerIdBlockData(data) debug(s"Read current producerId block $currProducerIdBlock, Zk path version $zkVersion") - if (currProducerIdBlock.blockEndId > Long.MaxValue - ProducerIdManager.PidBlockSize) { + if (currProducerIdBlock.producerIdEnd > Long.MaxValue - ProducerIdsBlock.PRODUCER_ID_BLOCK_SIZE) { // we have exhausted all producerIds (wow!), treat it as a fatal error - fatal(s"Exhausted all producerIds as the next block's end producerId is will has exceeded long type limit (current block end producerId is ${currProducerIdBlock.blockEndId})") + fatal(s"Exhausted all producerIds as the next block's end producerId is will has exceeded long type limit (current block end producerId is ${currProducerIdBlock.producerIdEnd})") callback.apply(Left(Errors.UNKNOWN_SERVER_ERROR)) return } - ProducerIdBlock(brokerId, currProducerIdBlock.blockEndId + 1L, currProducerIdBlock.blockEndId + ProducerIdManager.PidBlockSize) + new ProducerIdsBlock(brokerId, currProducerIdBlock.producerIdEnd + 1L, ProducerIdsBlock.PRODUCER_ID_BLOCK_SIZE) case None => debug(s"There is no producerId block yet (Zk path version $zkVersion), creating the first block") - ProducerIdBlock(brokerId, 0L, ProducerIdManager.PidBlockSize - 1) + new ProducerIdsBlock(brokerId, 0L, ProducerIdsBlock.PRODUCER_ID_BLOCK_SIZE) } val newProducerIdBlockData = ProducerIdBlockZNode.generateProducerIdBlockJson(newProducerIdBlock) @@ -2445,7 +2445,7 @@ class KafkaController(val config: KafkaConfig, if (zkWriteComplete) { info(s"Acquired new producerId block $newProducerIdBlock by writing to Zk with path version $version") - callback.apply(Right((newProducerIdBlock.blockStartId, newProducerIdBlock.blockEndId))) + callback.apply(Right((newProducerIdBlock.producerIdStart, newProducerIdBlock.producerIdEnd))) return } } diff --git a/core/src/main/scala/kafka/coordinator/transaction/ProducerIdManager.scala b/core/src/main/scala/kafka/coordinator/transaction/ProducerIdManager.scala index 036866139648c..38dc6280bb15e 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/ProducerIdManager.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/ProducerIdManager.scala @@ -23,6 +23,7 @@ import org.apache.kafka.common.KafkaException import org.apache.kafka.common.message.AllocateProducerIdsRequestData import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.{AllocateProducerIdsRequest, AllocateProducerIdsResponse} +import org.apache.kafka.server.common.ProducerIdsBlock import java.util.concurrent.{ArrayBlockingQueue, TimeUnit} import java.util.concurrent.atomic.AtomicBoolean @@ -36,24 +37,10 @@ import scala.util.{Failure, Success, Try} * a unique block. The block start and block end are inclusive. */ object ProducerIdManager { - val PidBlockSize = 1000L - // Once we reach this percentage of PIDs consumed from the current block, trigger a fetch of the next block val PidPrefetchThreshold = 0.90 } -case class ProducerIdBlock(brokerId: Int, blockStartId: Long, blockEndId: Long) { - override def toString: String = { - val producerIdBlockInfo = new StringBuilder - producerIdBlockInfo.append("(brokerId:" + brokerId) - producerIdBlockInfo.append(",blockStartProducerId:" + blockStartId) - producerIdBlockInfo.append(",blockEndProducerId:" + blockEndId + ")") - producerIdBlockInfo.toString() - } - - val blockSize: Long = blockEndId - blockStartId + 1 // inclusive -} - trait ProducerIdGenerator { def generateProducerId(): Long def shutdown() : Unit = {} @@ -66,9 +53,9 @@ class ProducerIdManager(brokerId: Int, this.logIdent = "[ProducerId Manager " + brokerId + "]: " - private val nextProducerIdBlock = new ArrayBlockingQueue[Try[ProducerIdBlock]](1) + private val nextProducerIdBlock = new ArrayBlockingQueue[Try[ProducerIdsBlock]](1) private val requestInFlight = new AtomicBoolean(false) - private var currentProducerIdBlock: ProducerIdBlock = ProducerIdBlock(brokerId, 0L, 0L) + private var currentProducerIdBlock: ProducerIdsBlock = ProducerIdsBlock.EMPTY private var nextProducerId: Long = 0L // Send an initial request to get the first block @@ -79,12 +66,12 @@ class ProducerIdManager(brokerId: Int, nextProducerId += 1 // Check if we need to fetch the next block - if (nextProducerId >= (currentProducerIdBlock.blockStartId + currentProducerIdBlock.blockSize * ProducerIdManager.PidPrefetchThreshold)) { + if (nextProducerId >= (currentProducerIdBlock.producerIdStart + currentProducerIdBlock.producerIdLen * ProducerIdManager.PidPrefetchThreshold)) { maybeRequestNextBlock() } // If we've exhausted the current block, grab the next block (waiting if necessary) - if (nextProducerId > currentProducerIdBlock.blockEndId) { + if (nextProducerId > currentProducerIdBlock.producerIdEnd) { val block = nextProducerIdBlock.poll(maxWaitMs, TimeUnit.MILLISECONDS) if (block == null) { throw Errors.REQUEST_TIMED_OUT.exception("Timed out waiting for next producer ID block") @@ -92,7 +79,7 @@ class ProducerIdManager(brokerId: Int, block match { case Success(nextBlock) => currentProducerIdBlock = nextBlock - nextProducerId = currentProducerIdBlock.blockStartId + nextProducerId = currentProducerIdBlock.producerIdStart case Failure(t) => throw t } } @@ -132,14 +119,14 @@ class ProducerIdManager(brokerId: Int, case Errors.NONE => debug(s"Got next producer ID block from controller $data") // Do some sanity checks on the response - if (data.producerIdStart() < currentProducerIdBlock.blockEndId) { + if (data.producerIdStart() < currentProducerIdBlock.producerIdEnd) { nextProducerIdBlock.put(Failure(new KafkaException( s"Producer ID block is not monotonic with current block: current=$currentProducerIdBlock response=$data"))) } else if (data.producerIdStart() < 0 || data.producerIdLen() < 0 || data.producerIdStart() > Long.MaxValue - data.producerIdLen()) { nextProducerIdBlock.put(Failure(new KafkaException(s"Producer ID block includes invalid ID range: $data"))) } else { nextProducerIdBlock.put( - Success(ProducerIdBlock(brokerId, data.producerIdStart(), data.producerIdStart() + data.producerIdLen() - 1))) + Success(new ProducerIdsBlock(brokerId, data.producerIdStart(), data.producerIdLen()))) } case Errors.STALE_BROKER_EPOCH => warn("Our broker epoch was stale, trying again.") diff --git a/core/src/main/scala/kafka/zk/ZkData.scala b/core/src/main/scala/kafka/zk/ZkData.scala index 7246dc27b9fd3..6fac96f30d217 100644 --- a/core/src/main/scala/kafka/zk/ZkData.scala +++ b/core/src/main/scala/kafka/zk/ZkData.scala @@ -25,7 +25,6 @@ import kafka.api.{ApiVersion, KAFKA_0_10_0_IV1, KAFKA_2_7_IV0, LeaderAndIsr} import kafka.cluster.{Broker, EndPoint} import kafka.common.{NotificationHandler, ZkNodeChangeNotificationListener} import kafka.controller.{IsrChangeNotificationHandler, LeaderIsrAndControllerEpoch, ReplicaAssignment} -import kafka.coordinator.transaction.ProducerIdBlock import kafka.security.authorizer.AclAuthorizer.VersionedAcls import kafka.security.authorizer.AclEntry import kafka.server.{ConfigType, DelegationTokenManager} @@ -40,6 +39,7 @@ import org.apache.kafka.common.resource.{PatternType, ResourcePattern, ResourceT import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.security.token.delegation.{DelegationToken, TokenInformation} import org.apache.kafka.common.utils.{SecurityUtils, Time} +import org.apache.kafka.server.common.ProducerIdsBlock import org.apache.zookeeper.ZooDefs import org.apache.zookeeper.data.{ACL, Stat} @@ -770,22 +770,22 @@ object ProducerIdBlockZNode { def path = "/latest_producer_id_block" - def generateProducerIdBlockJson(producerIdBlock: ProducerIdBlock): Array[Byte] = { + def generateProducerIdBlockJson(producerIdBlock: ProducerIdsBlock): Array[Byte] = { Json.encodeAsBytes(Map("version" -> CurrentVersion, "broker" -> producerIdBlock.brokerId, - "block_start" -> producerIdBlock.blockStartId.toString, - "block_end" -> producerIdBlock.blockEndId.toString).asJava + "block_start" -> producerIdBlock.producerIdStart.toString, + "block_end" -> producerIdBlock.producerIdEnd.toString).asJava ) } - def parseProducerIdBlockData(jsonData: Array[Byte]): ProducerIdBlock = { + def parseProducerIdBlockData(jsonData: Array[Byte]): ProducerIdsBlock = { val jsonDataAsString = jsonData.map(_.toChar).mkString try { Json.parseBytes(jsonData).map(_.asJsonObject).flatMap { js => val brokerId = js("broker").to[Int] val blockStart = js("block_start").to[String].toLong val blockEnd = js("block_end").to[String].toLong - Some(ProducerIdBlock(brokerId, blockStart, blockEnd)) + Some(new ProducerIdsBlock(brokerId, blockStart, Math.toIntExact(blockEnd - blockStart + 1))) }.getOrElse(throw new KafkaException(s"Failed to parse the producerId block json $jsonDataAsString")) } catch { case e: java.lang.NumberFormatException => diff --git a/core/src/test/scala/integration/kafka/coordinator/transaction/ProducerIdsIntegrationTest.scala b/core/src/test/scala/integration/kafka/coordinator/transaction/ProducerIdsIntegrationTest.scala index 7f4f2949cb40f..1edac84c39344 100644 --- a/core/src/test/scala/integration/kafka/coordinator/transaction/ProducerIdsIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/coordinator/transaction/ProducerIdsIntegrationTest.scala @@ -15,9 +15,9 @@ * limitations under the License. */ -package integration.kafka.coordinator.transaction +package kafka.coordinator.transaction -import integration.kafka.server.IntegrationTestUtils +import kafka.server.IntegrationTestUtils import kafka.network.SocketServer import kafka.server.KafkaConfig import kafka.test.annotation.{ClusterTest, Type} diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/ProducerIdManagerTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/ProducerIdManagerTest.scala index 95c25a6fe5884..8b0917f4506be 100644 --- a/core/src/test/scala/unit/kafka/coordinator/transaction/ProducerIdManagerTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/transaction/ProducerIdManagerTest.scala @@ -50,11 +50,11 @@ class ProducerIdManagerTest { } @ParameterizedTest - @ValueSource(ints = Array(1, 2, 1000)) - def testDifferentBlockLengths(idBlockLen: Int): Unit = { + @ValueSource(ints = Array(1, 2, 10)) + def testContiguousIds(idBlockLen: Int): Unit = { val manager = new MockProducerIdManager(0, 0, idBlockLen) - IntStream.range(0, idBlockLen * 2).forEach { i => + IntStream.range(0, idBlockLen * 3).forEach { i => assertEquals(i, manager.generateProducerId()) } } diff --git a/server-common/src/main/java/org/apache/kafka/server/common/ProducerIdsBlock.java b/server-common/src/main/java/org/apache/kafka/server/common/ProducerIdsBlock.java new file mode 100644 index 0000000000000..085640abeae7a --- /dev/null +++ b/server-common/src/main/java/org/apache/kafka/server/common/ProducerIdsBlock.java @@ -0,0 +1,61 @@ +package org.apache.kafka.server.common; + +import java.util.Objects; + +/** + * Holds a range of Producer IDs used for Transactional and EOS producers. + */ +public class ProducerIdsBlock { + public static final int PRODUCER_ID_BLOCK_SIZE = 1000; + + public static final ProducerIdsBlock EMPTY = new ProducerIdsBlock(-1, 0, 0); + + private final int brokerId; + private final long producerIdStart; + private final int producerIdLen; + + public ProducerIdsBlock(int brokerId, long producerIdStart, int producerIdLen) { + this.brokerId = brokerId; + this.producerIdStart = producerIdStart; + this.producerIdLen = producerIdLen; + } + + public int brokerId() { + return brokerId; + } + + public long producerIdStart() { + return producerIdStart; + } + + public int producerIdLen() { + return producerIdLen; + } + + public long producerIdEnd() { + return producerIdStart + producerIdLen - 1; + } + + + @Override + public String toString() { + return "ProducerIdsBlock{" + + "brokerId=" + brokerId + + ", producerIdStart=" + producerIdStart + + ", producerIdLen=" + producerIdLen + + '}'; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + ProducerIdsBlock that = (ProducerIdsBlock) o; + return brokerId == that.brokerId && producerIdStart == that.producerIdStart && producerIdLen == that.producerIdLen; + } + + @Override + public int hashCode() { + return Objects.hash(brokerId, producerIdStart, producerIdLen); + } +} From e89c4bbc26fd80d3cc212531fbecf3ac20858e29 Mon Sep 17 00:00:00 2001 From: David Arthur Date: Tue, 11 May 2021 16:15:41 -0400 Subject: [PATCH 06/20] Use new ProducerIdsBlock as callback type in controller --- .../message/AllocateProducerIdsResponse.json | 2 +- .../kafka/controller/KafkaController.scala | 15 ++++++++------- .../kafka/server/common/ProducerIdsBlock.java | 17 +++++++++++++++++ 3 files changed, 26 insertions(+), 8 deletions(-) diff --git a/clients/src/main/resources/common/message/AllocateProducerIdsResponse.json b/clients/src/main/resources/common/message/AllocateProducerIdsResponse.json index c3e2ae1e3059a..b6b5948f7d00d 100644 --- a/clients/src/main/resources/common/message/AllocateProducerIdsResponse.json +++ b/clients/src/main/resources/common/message/AllocateProducerIdsResponse.json @@ -29,4 +29,4 @@ { "name": "ProducerIdLen", "type": "int32", "versions": "0+", "about": "The number of producer IDs in this range"} ] -} \ No newline at end of file +} diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index 3a2906c2fb3a7..33ad434f5e83d 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -2378,20 +2378,20 @@ class KafkaController(val config: KafkaConfig, def allocateProducerIds(allocateProducerIdsRequest: AllocateProducerIdsRequestData, callback: AllocateProducerIdsResponseData => Unit): Unit = { - def eventManagerCallback(results: Either[Errors, (Long, Long)]): Unit = { + def eventManagerCallback(results: Either[Errors, ProducerIdsBlock]): Unit = { results match { case Left(error) => callback.apply(new AllocateProducerIdsResponseData().setErrorCode(error.code)) - case Right((start, end)) => callback.apply( + case Right(pidBlock) => callback.apply( new AllocateProducerIdsResponseData() - .setProducerIdStart(start) - .setProducerIdLen(Math.toIntExact(end - start + 1))) // ends are inclusive + .setProducerIdStart(pidBlock.producerIdStart()) + .setProducerIdLen(pidBlock.producerIdLen())) } } eventManager.put(AllocateProducerIds(allocateProducerIdsRequest.brokerId, allocateProducerIdsRequest.brokerEpoch, eventManagerCallback)) } - def processAllocateProducerIds(brokerId: Int, brokerEpoch: Long, callback: Either[Errors, (Long, Long)] => Unit): Unit = { + def processAllocateProducerIds(brokerId: Int, brokerEpoch: Long, callback: Either[Errors, ProducerIdsBlock] => Unit): Unit = { // Handle a few short-circuits if (!isActive) { callback.apply(Left(Errors.NOT_CONTROLLER)) @@ -2445,7 +2445,7 @@ class KafkaController(val config: KafkaConfig, if (zkWriteComplete) { info(s"Acquired new producerId block $newProducerIdBlock by writing to Zk with path version $version") - callback.apply(Right((newProducerIdBlock.producerIdStart, newProducerIdBlock.producerIdEnd))) + callback.apply(Right(newProducerIdBlock)) return } } @@ -2824,7 +2824,8 @@ case class UpdateFeatures(request: UpdateFeaturesRequest, override def preempt(): Unit = {} } -case class AllocateProducerIds(brokerId: Int, brokerEpoch: Long, callback: Either[Errors, (Long, Long)] => Unit) extends ControllerEvent { +case class AllocateProducerIds(brokerId: Int, brokerEpoch: Long, callback: Either[Errors, ProducerIdsBlock] => Unit) + extends ControllerEvent { override def state: ControllerState = ControllerState.Idle override def preempt(): Unit = {} } diff --git a/server-common/src/main/java/org/apache/kafka/server/common/ProducerIdsBlock.java b/server-common/src/main/java/org/apache/kafka/server/common/ProducerIdsBlock.java index 085640abeae7a..cfeb09a93a2c2 100644 --- a/server-common/src/main/java/org/apache/kafka/server/common/ProducerIdsBlock.java +++ b/server-common/src/main/java/org/apache/kafka/server/common/ProducerIdsBlock.java @@ -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 org.apache.kafka.server.common; import java.util.Objects; From 18615267e6ccba49e33551ff5ba40b8f6754176f Mon Sep 17 00:00:00 2001 From: David Arthur Date: Tue, 11 May 2021 16:20:36 -0400 Subject: [PATCH 07/20] Clean up a few things --- .../transaction/ProducerIdManager.scala | 2 +- core/src/main/scala/kafka/zk/ZkData.scala | 3 - .../junit/RaftClusterInvocationContext.java | 70 +++++++++---------- .../common/metadata/ProducerIdRecord.json | 29 -------- .../kafka/server/common/ProducerIdsBlock.java | 2 + 5 files changed, 38 insertions(+), 68 deletions(-) delete mode 100644 metadata/src/main/resources/common/metadata/ProducerIdRecord.json diff --git a/core/src/main/scala/kafka/coordinator/transaction/ProducerIdManager.scala b/core/src/main/scala/kafka/coordinator/transaction/ProducerIdManager.scala index 38dc6280bb15e..cf1b598a6d72c 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/ProducerIdManager.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/ProducerIdManager.scala @@ -34,7 +34,7 @@ import scala.util.{Failure, Success, Try} * such that the same producerId will not be assigned twice across multiple transaction coordinators. * * ProducerIds are managed by the controller. When requesting a new range of IDs, we are guaranteed to receive - * a unique block. The block start and block end are inclusive. + * a unique block. */ object ProducerIdManager { // Once we reach this percentage of PIDs consumed from the current block, trigger a fetch of the next block diff --git a/core/src/main/scala/kafka/zk/ZkData.scala b/core/src/main/scala/kafka/zk/ZkData.scala index 6fac96f30d217..0f6db4a5d3dc1 100644 --- a/core/src/main/scala/kafka/zk/ZkData.scala +++ b/core/src/main/scala/kafka/zk/ZkData.scala @@ -963,9 +963,6 @@ object FeatureZNode { } } - - - object ZkData { // Important: it is necessary to add any new top level Zookeeper path to the Seq diff --git a/core/src/test/java/kafka/test/junit/RaftClusterInvocationContext.java b/core/src/test/java/kafka/test/junit/RaftClusterInvocationContext.java index 409e7d6311bb3..8a0a9d01e7e57 100644 --- a/core/src/test/java/kafka/test/junit/RaftClusterInvocationContext.java +++ b/core/src/test/java/kafka/test/junit/RaftClusterInvocationContext.java @@ -66,36 +66,36 @@ public RaftClusterInvocationContext(ClusterConfig clusterConfig) { @Override public String getDisplayName(int invocationIndex) { String clusterDesc = clusterConfig.nameTags().entrySet().stream() - .map(Object::toString) - .collect(Collectors.joining(", ")); + .map(Object::toString) + .collect(Collectors.joining(", ")); return String.format("[%d] Type=Raft, %s", invocationIndex, clusterDesc); } @Override public List getAdditionalExtensions() { return Arrays.asList( - (BeforeTestExecutionCallback) context -> { - KafkaClusterTestKit.Builder builder = new KafkaClusterTestKit.Builder( - new TestKitNodes.Builder(). - setNumBrokerNodes(clusterConfig.numBrokers()). - setNumControllerNodes(clusterConfig.numControllers()).build()); - - // 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 - KafkaClusterTestKit cluster = builder.build(); - clusterReference.set(cluster); - cluster.format(); - cluster.startup(); - kafka.utils.TestUtils.waitUntilTrue( - () -> cluster.brokers().get(0).currentState() == BrokerState.RUNNING, - () -> "Broker never made it to RUNNING state.", - org.apache.kafka.test.TestUtils.DEFAULT_MAX_WAIT_MS, - 100L); - }, - (AfterTestExecutionCallback) context -> clusterReference.get().close(), - new ClusterInstanceParameterResolver(new RaftClusterInstance(clusterReference, clusterConfig)), - new GenericParameterResolver<>(clusterConfig, ClusterConfig.class) + (BeforeTestExecutionCallback) context -> { + KafkaClusterTestKit.Builder builder = new KafkaClusterTestKit.Builder( + new TestKitNodes.Builder(). + setNumBrokerNodes(clusterConfig.numBrokers()). + setNumControllerNodes(clusterConfig.numControllers()).build()); + + // 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 + KafkaClusterTestKit cluster = builder.build(); + clusterReference.set(cluster); + cluster.format(); + cluster.startup(); + kafka.utils.TestUtils.waitUntilTrue( + () -> cluster.brokers().get(0).currentState() == BrokerState.RUNNING, + () -> "Broker never made it to RUNNING state.", + org.apache.kafka.test.TestUtils.DEFAULT_MAX_WAIT_MS, + 100L); + }, + (AfterTestExecutionCallback) context -> clusterReference.get().close(), + new ClusterInstanceParameterResolver(new RaftClusterInstance(clusterReference, clusterConfig)), + new GenericParameterResolver<>(clusterConfig, ClusterConfig.class) ); } @@ -119,8 +119,8 @@ public String bootstrapServers() { @Override public Collection brokerSocketServers() { return clusterReference.get().brokers().values().stream() - .map(BrokerServer::socketServer) - .collect(Collectors.toList()); + .map(BrokerServer::socketServer) + .collect(Collectors.toList()); } @Override @@ -131,24 +131,24 @@ public ListenerName clientListener() { @Override public Collection controllerSocketServers() { return clusterReference.get().controllers().values().stream() - .map(ControllerServer::socketServer) - .collect(Collectors.toList()); + .map(ControllerServer::socketServer) + .collect(Collectors.toList()); } @Override public SocketServer anyBrokerSocketServer() { return clusterReference.get().brokers().values().stream() - .map(BrokerServer::socketServer) - .findFirst() - .orElseThrow(() -> new RuntimeException("No broker SocketServers found")); + .map(BrokerServer::socketServer) + .findFirst() + .orElseThrow(() -> new RuntimeException("No broker SocketServers found")); } @Override public SocketServer anyControllerSocketServer() { return clusterReference.get().controllers().values().stream() - .map(ControllerServer::socketServer) - .findFirst() - .orElseThrow(() -> new RuntimeException("No controller SocketServers found")); + .map(ControllerServer::socketServer) + .findFirst() + .orElseThrow(() -> new RuntimeException("No controller SocketServers found")); } @Override @@ -198,4 +198,4 @@ public void rollingBrokerRestart() { throw new UnsupportedOperationException("Restarting Raft servers is not yet supported."); } } -} \ No newline at end of file +} diff --git a/metadata/src/main/resources/common/metadata/ProducerIdRecord.json b/metadata/src/main/resources/common/metadata/ProducerIdRecord.json deleted file mode 100644 index ea6e16f3292a4..0000000000000 --- a/metadata/src/main/resources/common/metadata/ProducerIdRecord.json +++ /dev/null @@ -1,29 +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. - -{ - "apiKey": 15, - "type": "metadata", - "name": "ProducerIdRecord", - "validVersions": "0", - "fields": [ - { "name": "BrokerId", "type": "int32", "versions": "0+", "entityType": "brokerId", - "about": "The ID of the requesting broker" }, - { "name": "BrokerEpoch", "type": "int64", "versions": "0+", "default": "-1", - "about": "The epoch of the requesting broker" }, - { "name": "ProducerIdEnd", "type": "int64", "versions": "0+", - "about": "The highest producer ID that has been generated"} - ] -} diff --git a/server-common/src/main/java/org/apache/kafka/server/common/ProducerIdsBlock.java b/server-common/src/main/java/org/apache/kafka/server/common/ProducerIdsBlock.java index cfeb09a93a2c2..8a0fd8469f7f9 100644 --- a/server-common/src/main/java/org/apache/kafka/server/common/ProducerIdsBlock.java +++ b/server-common/src/main/java/org/apache/kafka/server/common/ProducerIdsBlock.java @@ -21,6 +21,8 @@ /** * Holds a range of Producer IDs used for Transactional and EOS producers. + * + * The start and end of the ID block are inclusive. */ public class ProducerIdsBlock { public static final int PRODUCER_ID_BLOCK_SIZE = 1000; From aede4fc01b451d1a5da604604437f82b557d2175 Mon Sep 17 00:00:00 2001 From: David Arthur Date: Tue, 11 May 2021 16:38:54 -0400 Subject: [PATCH 08/20] Remove unwanted whitespace --- .../junit/RaftClusterInvocationContext.java | 64 +++++++++---------- 1 file changed, 32 insertions(+), 32 deletions(-) diff --git a/core/src/test/java/kafka/test/junit/RaftClusterInvocationContext.java b/core/src/test/java/kafka/test/junit/RaftClusterInvocationContext.java index 8a0a9d01e7e57..4fda43ac172ba 100644 --- a/core/src/test/java/kafka/test/junit/RaftClusterInvocationContext.java +++ b/core/src/test/java/kafka/test/junit/RaftClusterInvocationContext.java @@ -74,28 +74,28 @@ public String getDisplayName(int invocationIndex) { @Override public List getAdditionalExtensions() { return Arrays.asList( - (BeforeTestExecutionCallback) context -> { - KafkaClusterTestKit.Builder builder = new KafkaClusterTestKit.Builder( - new TestKitNodes.Builder(). - setNumBrokerNodes(clusterConfig.numBrokers()). - setNumControllerNodes(clusterConfig.numControllers()).build()); - - // 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 - KafkaClusterTestKit cluster = builder.build(); - clusterReference.set(cluster); - cluster.format(); - cluster.startup(); - kafka.utils.TestUtils.waitUntilTrue( - () -> cluster.brokers().get(0).currentState() == BrokerState.RUNNING, - () -> "Broker never made it to RUNNING state.", - org.apache.kafka.test.TestUtils.DEFAULT_MAX_WAIT_MS, - 100L); - }, - (AfterTestExecutionCallback) context -> clusterReference.get().close(), - new ClusterInstanceParameterResolver(new RaftClusterInstance(clusterReference, clusterConfig)), - new GenericParameterResolver<>(clusterConfig, ClusterConfig.class) + (BeforeTestExecutionCallback) context -> { + KafkaClusterTestKit.Builder builder = new KafkaClusterTestKit.Builder( + new TestKitNodes.Builder(). + setNumBrokerNodes(clusterConfig.numBrokers()). + setNumControllerNodes(clusterConfig.numControllers()).build()); + + // 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 + KafkaClusterTestKit cluster = builder.build(); + clusterReference.set(cluster); + cluster.format(); + cluster.startup(); + kafka.utils.TestUtils.waitUntilTrue( + () -> cluster.brokers().get(0).currentState() == BrokerState.RUNNING, + () -> "Broker never made it to RUNNING state.", + org.apache.kafka.test.TestUtils.DEFAULT_MAX_WAIT_MS, + 100L); + }, + (AfterTestExecutionCallback) context -> clusterReference.get().close(), + new ClusterInstanceParameterResolver(new RaftClusterInstance(clusterReference, clusterConfig)), + new GenericParameterResolver<>(clusterConfig, ClusterConfig.class) ); } @@ -119,8 +119,8 @@ public String bootstrapServers() { @Override public Collection brokerSocketServers() { return clusterReference.get().brokers().values().stream() - .map(BrokerServer::socketServer) - .collect(Collectors.toList()); + .map(BrokerServer::socketServer) + .collect(Collectors.toList()); } @Override @@ -131,24 +131,24 @@ public ListenerName clientListener() { @Override public Collection controllerSocketServers() { return clusterReference.get().controllers().values().stream() - .map(ControllerServer::socketServer) - .collect(Collectors.toList()); + .map(ControllerServer::socketServer) + .collect(Collectors.toList()); } @Override public SocketServer anyBrokerSocketServer() { return clusterReference.get().brokers().values().stream() - .map(BrokerServer::socketServer) - .findFirst() - .orElseThrow(() -> new RuntimeException("No broker SocketServers found")); + .map(BrokerServer::socketServer) + .findFirst() + .orElseThrow(() -> new RuntimeException("No broker SocketServers found")); } @Override public SocketServer anyControllerSocketServer() { return clusterReference.get().controllers().values().stream() - .map(ControllerServer::socketServer) - .findFirst() - .orElseThrow(() -> new RuntimeException("No controller SocketServers found")); + .map(ControllerServer::socketServer) + .findFirst() + .orElseThrow(() -> new RuntimeException("No controller SocketServers found")); } @Override From 1ff23984a7555d35a96aae64f125abec9bfc60f4 Mon Sep 17 00:00:00 2001 From: David Arthur Date: Wed, 12 May 2021 13:04:19 -0400 Subject: [PATCH 09/20] Add IBP 3.0-IV0 for gating this new RPC --- .../src/main/scala/kafka/api/ApiVersion.scala | 13 ++- .../kafka/controller/KafkaController.scala | 47 ++-------- .../transaction/ProducerIdManager.scala | 91 ++++++++++++++++++- .../main/scala/kafka/server/KafkaServer.scala | 21 +++-- .../test/java/kafka/test/ClusterConfig.java | 20 +++- .../kafka/test/annotation/ClusterTest.java | 1 + .../test/junit/ClusterTestExtensions.java | 6 +- .../junit/ZkClusterInvocationContext.java | 7 +- .../ProducerIdsIntegrationTest.scala | 53 ++++++++++- .../transaction/ProducerIdManagerTest.scala | 59 +++++++++++- .../integration/KafkaServerTestHarness.scala | 13 ++- 11 files changed, 271 insertions(+), 60 deletions(-) diff --git a/core/src/main/scala/kafka/api/ApiVersion.scala b/core/src/main/scala/kafka/api/ApiVersion.scala index 879373787365f..724bc9894045a 100644 --- a/core/src/main/scala/kafka/api/ApiVersion.scala +++ b/core/src/main/scala/kafka/api/ApiVersion.scala @@ -112,7 +112,9 @@ object ApiVersion { // Flexible versioning on ListOffsets, WriteTxnMarkers and OffsetsForLeaderEpoch. Also adds topic IDs (KIP-516) KAFKA_2_8_IV0, // Introduced topic IDs to LeaderAndIsr and UpdateMetadata requests/responses (KIP-516) - KAFKA_2_8_IV1 + KAFKA_2_8_IV1, + // Introduce AllocateProducerIds (KIP-730) + KAFKA_3_0_IV0 ) // Map keys are the union of the short and full versions @@ -197,6 +199,8 @@ sealed trait ApiVersion extends Ordered[ApiVersion] { def isAlterIsrSupported: Boolean = this >= KAFKA_2_7_IV2 + def isAllocateProducerIdsSupported: Boolean = this >= KAFKA_3_0_IV0 + override def compare(that: ApiVersion): Int = ApiVersion.orderingByVersion.compare(this, that) @@ -447,6 +451,13 @@ case object KAFKA_2_8_IV1 extends DefaultApiVersion { val id: Int = 32 } +case object KAFKA_3_0_IV0 extends DefaultApiVersion { + val shortVersion: String = "3.0" + val subVersion = "IV0" + val recordVersion = RecordVersion.V2 + val id: Int = 33 +} + object ApiVersionValidator extends Validator { override def ensureValid(name: String, value: Any): Unit = { diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index 33ad434f5e83d..df6d3d6d67648 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -24,6 +24,7 @@ import kafka.common._ import kafka.controller.KafkaController.AlterIsrCallback import kafka.cluster.Broker import kafka.controller.KafkaController.{AlterReassignmentsCallback, ElectLeadersCallback, ListReassignmentsCallback, UpdateFeaturesCallback} +import kafka.coordinator.transaction.ZkProducerIdManager import kafka.metrics.{KafkaMetricsGroup, KafkaTimer} import kafka.server._ import kafka.utils._ @@ -49,7 +50,7 @@ import org.apache.zookeeper.KeeperException.Code import scala.collection.{Map, Seq, Set, immutable, mutable} import scala.collection.mutable.ArrayBuffer import scala.jdk.CollectionConverters._ -import scala.util.{Failure, Try} +import scala.util.{Failure, Success, Try} sealed trait ElectionTrigger final case object AutoTriggered extends ElectionTrigger @@ -2411,43 +2412,15 @@ class KafkaController(val config: KafkaConfig, return } - // Get or create the existing PID block from ZK and attempt to update it. We retry in a loop here since other - // brokers may be generating PID blocks during a rolling upgrade - var zkWriteComplete = false - while (!zkWriteComplete) { - // refresh current producerId block from zookeeper again - val (dataOpt, zkVersion) = zkClient.getDataAndVersion(ProducerIdBlockZNode.path) - - // generate the new producerId block - val newProducerIdBlock = dataOpt match { - case Some(data) => - val currProducerIdBlock = ProducerIdBlockZNode.parseProducerIdBlockData(data) - debug(s"Read current producerId block $currProducerIdBlock, Zk path version $zkVersion") - - if (currProducerIdBlock.producerIdEnd > Long.MaxValue - ProducerIdsBlock.PRODUCER_ID_BLOCK_SIZE) { - // we have exhausted all producerIds (wow!), treat it as a fatal error - fatal(s"Exhausted all producerIds as the next block's end producerId is will has exceeded long type limit (current block end producerId is ${currProducerIdBlock.producerIdEnd})") - callback.apply(Left(Errors.UNKNOWN_SERVER_ERROR)) - return - } - - new ProducerIdsBlock(brokerId, currProducerIdBlock.producerIdEnd + 1L, ProducerIdsBlock.PRODUCER_ID_BLOCK_SIZE) - case None => - debug(s"There is no producerId block yet (Zk path version $zkVersion), creating the first block") - new ProducerIdsBlock(brokerId, 0L, ProducerIdsBlock.PRODUCER_ID_BLOCK_SIZE) - } - - val newProducerIdBlockData = ProducerIdBlockZNode.generateProducerIdBlockJson(newProducerIdBlock) - - // try to write the new producerId block into zookeeper - val (succeeded, version) = zkClient.conditionalUpdatePath(ProducerIdBlockZNode.path, newProducerIdBlockData, zkVersion, None) - zkWriteComplete = succeeded + val maybeNewProducerIdsBlock = try { + Try(ZkProducerIdManager.getNewProducerIdBlock(brokerId, zkClient, this)) + } catch { + case ke: KafkaException => Failure(ke) + } - if (zkWriteComplete) { - info(s"Acquired new producerId block $newProducerIdBlock by writing to Zk with path version $version") - callback.apply(Right(newProducerIdBlock)) - return - } + maybeNewProducerIdsBlock match { + case Failure(exception) => callback.apply(Left(Errors.forException(exception))) + case Success(newProducerIdBlock) => callback.apply(Right(newProducerIdBlock)) } } diff --git a/core/src/main/scala/kafka/coordinator/transaction/ProducerIdManager.scala b/core/src/main/scala/kafka/coordinator/transaction/ProducerIdManager.scala index cf1b598a6d72c..64d1133f831f2 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/ProducerIdManager.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/ProducerIdManager.scala @@ -18,6 +18,7 @@ package kafka.coordinator.transaction import kafka.server.{BrokerToControllerChannelManager, ControllerRequestCompletionHandler} import kafka.utils.Logging +import kafka.zk.{KafkaZkClient, ProducerIdBlockZNode} import org.apache.kafka.clients.ClientResponse import org.apache.kafka.common.KafkaException import org.apache.kafka.common.message.AllocateProducerIdsRequestData @@ -36,9 +37,23 @@ import scala.util.{Failure, Success, Try} * ProducerIds are managed by the controller. When requesting a new range of IDs, we are guaranteed to receive * a unique block. */ -object ProducerIdManager { + +object ProducerIdGenerator { // Once we reach this percentage of PIDs consumed from the current block, trigger a fetch of the next block val PidPrefetchThreshold = 0.90 + + // Creates a ProducerIdGenerate that directly interfaces with ZooKeeper, IBP < 3.0-IV0 + def apply(brokerId: Int, zkClient: KafkaZkClient): ZkProducerIdManager = { + new ZkProducerIdManager(brokerId, zkClient) + } + + // Creates a ProducerIdGenerate that uses AllocateProducerIds RPC, IBP >= 3.0-IV0 + def apply(brokerId: Int, + brokerEpochSupplier: () => Long, + controllerChannel: BrokerToControllerChannelManager, + maxWaitMs: Int): ProducerIdManager = { + new ProducerIdManager(brokerId, brokerEpochSupplier, controllerChannel, maxWaitMs) + } } trait ProducerIdGenerator { @@ -46,6 +61,78 @@ trait ProducerIdGenerator { def shutdown() : Unit = {} } +object ZkProducerIdManager { + def getNewProducerIdBlock(brokerId: Int, zkClient: KafkaZkClient, logger: Logging): ProducerIdsBlock = { + // Get or create the existing PID block from ZK and attempt to update it. We retry in a loop here since other + // brokers may be generating PID blocks during a rolling upgrade + var zkWriteComplete = false + while (!zkWriteComplete) { + // refresh current producerId block from zookeeper again + val (dataOpt, zkVersion) = zkClient.getDataAndVersion(ProducerIdBlockZNode.path) + + // generate the new producerId block + val newProducerIdBlock = dataOpt match { + case Some(data) => + val currProducerIdBlock = ProducerIdBlockZNode.parseProducerIdBlockData(data) + logger.debug(s"Read current producerId block $currProducerIdBlock, Zk path version $zkVersion") + + if (currProducerIdBlock.producerIdEnd > Long.MaxValue - ProducerIdsBlock.PRODUCER_ID_BLOCK_SIZE) { + // we have exhausted all producerIds (wow!), treat it as a fatal error + logger.fatal(s"Exhausted all producerIds as the next block's end producerId is will has exceeded long type limit (current block end producerId is ${currProducerIdBlock.producerIdEnd})") + throw new KafkaException("Have exhausted all producerIds.") + } + + new ProducerIdsBlock(brokerId, currProducerIdBlock.producerIdEnd + 1L, ProducerIdsBlock.PRODUCER_ID_BLOCK_SIZE) + case None => + logger.debug(s"There is no producerId block yet (Zk path version $zkVersion), creating the first block") + new ProducerIdsBlock(brokerId, 0L, ProducerIdsBlock.PRODUCER_ID_BLOCK_SIZE) + } + + val newProducerIdBlockData = ProducerIdBlockZNode.generateProducerIdBlockJson(newProducerIdBlock) + + // try to write the new producerId block into zookeeper + val (succeeded, version) = zkClient.conditionalUpdatePath(ProducerIdBlockZNode.path, newProducerIdBlockData, zkVersion, None) + zkWriteComplete = succeeded + + if (zkWriteComplete) { + logger.info(s"Acquired new producerId block $newProducerIdBlock by writing to Zk with path version $version") + return newProducerIdBlock + } + } + throw new IllegalStateException() + } +} + +class ZkProducerIdManager(brokerId: Int, + zkClient: KafkaZkClient) extends ProducerIdGenerator with Logging { + + private var currentProducerIdBlock: ProducerIdsBlock = ProducerIdsBlock.EMPTY + private var nextProducerId: Long = -1L + + // grab the first block of producerIds + this synchronized { + getNewProducerIdBlock() + nextProducerId = currentProducerIdBlock.producerIdStart + } + + private def getNewProducerIdBlock(): Unit = { + currentProducerIdBlock = ZkProducerIdManager.getNewProducerIdBlock(brokerId, zkClient, this) + } + + def generateProducerId(): Long = { + this synchronized { + // grab a new block of producerIds if this block has been exhausted + if (nextProducerId > currentProducerIdBlock.producerIdEnd) { + getNewProducerIdBlock() + nextProducerId = currentProducerIdBlock.producerIdStart + 1 + } else { + nextProducerId += 1 + } + nextProducerId - 1 + } + } +} + class ProducerIdManager(brokerId: Int, brokerEpochSupplier: () => Long, controllerChannel: BrokerToControllerChannelManager, @@ -66,7 +153,7 @@ class ProducerIdManager(brokerId: Int, nextProducerId += 1 // Check if we need to fetch the next block - if (nextProducerId >= (currentProducerIdBlock.producerIdStart + currentProducerIdBlock.producerIdLen * ProducerIdManager.PidPrefetchThreshold)) { + if (nextProducerId >= (currentProducerIdBlock.producerIdStart + currentProducerIdBlock.producerIdLen * ProducerIdGenerator.PidPrefetchThreshold)) { maybeRequestNextBlock() } diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index db650dab4f3ab..b99b36caebf18 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -26,7 +26,7 @@ import kafka.cluster.Broker import kafka.common.{GenerateBrokerIdException, InconsistentBrokerIdException, InconsistentClusterIdException} import kafka.controller.KafkaController import kafka.coordinator.group.GroupCoordinator -import kafka.coordinator.transaction.{ProducerIdManager, TransactionCoordinator} +import kafka.coordinator.transaction.{ProducerIdGenerator, TransactionCoordinator} import kafka.log.LogManager import kafka.metrics.{KafkaMetricsReporter, KafkaYammerMetrics} import kafka.network.SocketServer @@ -331,16 +331,21 @@ class KafkaServer( groupCoordinator = GroupCoordinator(config, replicaManager, Time.SYSTEM, metrics) groupCoordinator.startup(() => zkClient.getTopicPartitionCount(Topic.GROUP_METADATA_TOPIC_NAME).getOrElse(config.offsetsTopicPartitions)) + /* create producer ids manager */ + val producerIdManager = if (config.interBrokerProtocolVersion.isAllocateProducerIdsSupported) { + ProducerIdGenerator( + config.brokerId, + brokerEpochSupplier = () => kafkaController.brokerEpoch, + brokerToControllerManager, + config.requestTimeoutMs + ) + } else { + ProducerIdGenerator(config.brokerId, zkClient) + } /* start transaction coordinator, with a separate background thread scheduler for transaction expiration and log loading */ // Hardcode Time.SYSTEM for now as some Streams tests fail otherwise, it would be good to fix the underlying issue - val producerIdManagerSupplier = () => new ProducerIdManager( - config.brokerId, - brokerEpochSupplier = () => kafkaController.brokerEpoch, - brokerToControllerManager, - config.requestTimeoutMs - ) transactionCoordinator = TransactionCoordinator(config, replicaManager, new KafkaScheduler(threads = 1, threadNamePrefix = "transaction-log-manager-"), - producerIdManagerSupplier, metrics, metadataCache, Time.SYSTEM) + () => producerIdManager, metrics, metadataCache, Time.SYSTEM) transactionCoordinator.startup( () => zkClient.getTopicPartitionCount(Topic.TRANSACTION_STATE_TOPIC_NAME).getOrElse(config.transactionTopicPartitions)) diff --git a/core/src/test/java/kafka/test/ClusterConfig.java b/core/src/test/java/kafka/test/ClusterConfig.java index e7e8bf5a0234a..7c14ed252698a 100644 --- a/core/src/test/java/kafka/test/ClusterConfig.java +++ b/core/src/test/java/kafka/test/ClusterConfig.java @@ -40,6 +40,7 @@ public class ClusterConfig { private final SecurityProtocol securityProtocol; private final String listenerName; private final File trustStoreFile; + private final String ibp; private final Properties serverProperties = new Properties(); private final Properties producerProperties = new Properties(); @@ -49,7 +50,8 @@ public class ClusterConfig { private final Properties saslClientProperties = new Properties(); ClusterConfig(Type type, int brokers, int controllers, String name, boolean autoStart, - SecurityProtocol securityProtocol, String listenerName, File trustStoreFile) { + SecurityProtocol securityProtocol, String listenerName, File trustStoreFile, + String ibp) { this.type = type; this.brokers = brokers; this.controllers = controllers; @@ -58,6 +60,7 @@ public class ClusterConfig { this.securityProtocol = securityProtocol; this.listenerName = listenerName; this.trustStoreFile = trustStoreFile; + this.ibp = ibp; } public Type clusterType() { @@ -116,16 +119,21 @@ public Optional trustStoreFile() { return Optional.ofNullable(trustStoreFile); } + public Optional ibp() { + return Optional.ofNullable(ibp); + } + public Map nameTags() { Map tags = new LinkedHashMap<>(3); name().ifPresent(name -> tags.put("Name", name)); + ibp().ifPresent(ibp -> tags.put("IBP", ibp)); tags.put("Security", securityProtocol.name()); listenerName().ifPresent(listener -> tags.put("Listener", listener)); return tags; } public ClusterConfig copyOf() { - ClusterConfig copy = new ClusterConfig(type, brokers, controllers, name, autoStart, securityProtocol, listenerName, trustStoreFile); + ClusterConfig copy = new ClusterConfig(type, brokers, controllers, name, autoStart, securityProtocol, listenerName, trustStoreFile, ibp); copy.serverProperties.putAll(serverProperties); copy.producerProperties.putAll(producerProperties); copy.consumerProperties.putAll(consumerProperties); @@ -151,6 +159,7 @@ public static class Builder { private SecurityProtocol securityProtocol; private String listenerName; private File trustStoreFile; + private String ibp; Builder(Type type, int brokers, int controllers, boolean autoStart, SecurityProtocol securityProtocol) { this.type = type; @@ -200,8 +209,13 @@ public Builder trustStoreFile(File trustStoreFile) { return this; } + public Builder ibp(String ibp) { + this.ibp = ibp; + return this; + } + public ClusterConfig build() { - return new ClusterConfig(type, brokers, controllers, name, autoStart, securityProtocol, listenerName, trustStoreFile); + return new ClusterConfig(type, brokers, controllers, name, autoStart, securityProtocol, listenerName, trustStoreFile, ibp); } } } diff --git a/core/src/test/java/kafka/test/annotation/ClusterTest.java b/core/src/test/java/kafka/test/annotation/ClusterTest.java index 687255c3c4705..11336ab87a15f 100644 --- a/core/src/test/java/kafka/test/annotation/ClusterTest.java +++ b/core/src/test/java/kafka/test/annotation/ClusterTest.java @@ -40,5 +40,6 @@ String name() default ""; SecurityProtocol securityProtocol() default SecurityProtocol.PLAINTEXT; String listener() default ""; + String ibp() default ""; ClusterConfigProperty[] serverProperties() default {}; } diff --git a/core/src/test/java/kafka/test/junit/ClusterTestExtensions.java b/core/src/test/java/kafka/test/junit/ClusterTestExtensions.java index 94eefbd91cb81..293f00b035ca5 100644 --- a/core/src/test/java/kafka/test/junit/ClusterTestExtensions.java +++ b/core/src/test/java/kafka/test/junit/ClusterTestExtensions.java @@ -183,7 +183,7 @@ private void processClusterTest(ExtensionContext context, ClusterTest annot, Clu if (!annot.name().isEmpty()) { builder.name(annot.name()); } else { - builder.name(context.getDisplayName()); + builder.name(context.getRequiredTestMethod().getName()); } if (!annot.listener().isEmpty()) { builder.listenerName(annot.listener()); @@ -194,6 +194,10 @@ private void processClusterTest(ExtensionContext context, ClusterTest annot, Clu properties.put(property.key(), property.value()); } + if (!annot.ibp().isEmpty()) { + builder.ibp(annot.ibp()); + } + ClusterConfig config = builder.build(); config.serverProperties().putAll(properties); type.invocationContexts(config, testInvocations); diff --git a/core/src/test/java/kafka/test/junit/ZkClusterInvocationContext.java b/core/src/test/java/kafka/test/junit/ZkClusterInvocationContext.java index 1b629330d687e..cd3ff94f2e84a 100644 --- a/core/src/test/java/kafka/test/junit/ZkClusterInvocationContext.java +++ b/core/src/test/java/kafka/test/junit/ZkClusterInvocationContext.java @@ -19,6 +19,7 @@ import kafka.api.IntegrationTestHarness; import kafka.network.SocketServer; +import kafka.server.KafkaConfig; import kafka.server.KafkaServer; import kafka.utils.TestUtils; import org.apache.kafka.clients.admin.Admin; @@ -91,7 +92,9 @@ public List getAdditionalExtensions() { IntegrationTestHarness cluster = new IntegrationTestHarness() { @Override public Properties serverConfig() { - return clusterConfig.serverProperties(); + Properties props = clusterConfig.serverProperties(); + clusterConfig.ibp().ifPresent(ibp -> props.put(KafkaConfig.InterBrokerProtocolVersionProp(), ibp)); + return props; } @Override @@ -255,7 +258,7 @@ public void rollingBrokerRestart() { for (int i = 0; i < clusterReference.get().brokerCount(); i++) { clusterReference.get().killBroker(i); } - clusterReference.get().restartDeadBrokers(); + clusterReference.get().restartDeadBrokers(true); } else { throw new IllegalStateException("Tried to restart brokers but the cluster has not been started!"); } diff --git a/core/src/test/scala/integration/kafka/coordinator/transaction/ProducerIdsIntegrationTest.scala b/core/src/test/scala/integration/kafka/coordinator/transaction/ProducerIdsIntegrationTest.scala index 1edac84c39344..ce61be7c44b16 100644 --- a/core/src/test/scala/integration/kafka/coordinator/transaction/ProducerIdsIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/coordinator/transaction/ProducerIdsIntegrationTest.scala @@ -17,10 +17,9 @@ package kafka.coordinator.transaction -import kafka.server.IntegrationTestUtils import kafka.network.SocketServer -import kafka.server.KafkaConfig -import kafka.test.annotation.{ClusterTest, Type} +import kafka.server.{IntegrationTestUtils, KafkaConfig} +import kafka.test.annotation.{AutoStart, ClusterTest, ClusterTests, Type} import kafka.test.junit.ClusterTestExtensions import kafka.test.{ClusterConfig, ClusterInstance} import org.apache.kafka.common.message.InitProducerIdRequestData @@ -31,7 +30,11 @@ import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue} import org.junit.jupiter.api.BeforeEach import org.junit.jupiter.api.extension.ExtendWith +import java.net.SocketException +import java.util.concurrent.atomic.AtomicBoolean +import java.util.concurrent.{CompletableFuture, Executors} import java.util.stream.{Collectors, IntStream} +import scala.collection.mutable.ArrayBuffer @ExtendWith(value = Array(classOf[ClusterTestExtensions])) class ProducerIdsIntegrationTest { @@ -42,7 +45,10 @@ class ProducerIdsIntegrationTest { clusterConfig.serverProperties().put(KafkaConfig.TransactionsTopicReplicationFactorProp, "3") } - @ClusterTest(clusterType = Type.ZK, brokers = 3) + @ClusterTests(Array( + new ClusterTest(clusterType = Type.ZK, brokers = 3, ibp = "2.8"), + new ClusterTest(clusterType = Type.ZK, brokers = 3, ibp = "3.0-IV0") + )) def testNonOverlapping(clusterInstance: ClusterInstance): Unit = { val ids = clusterInstance.brokerSocketServers().stream().flatMap( broker => { IntStream.range(0, 1001).parallel().mapToObj( _ => nextProducerId(broker, clusterInstance.clientListener())) @@ -66,6 +72,45 @@ class ProducerIdsIntegrationTest { assertEquals(1000, id1) } + @ClusterTest(clusterType = Type.ZK, brokers = 3, autoStart = AutoStart.NO) + def testRollingUpgrade(clusterInstance: ClusterInstance): Unit = { + clusterInstance.config().serverProperties().put(KafkaConfig.InterBrokerProtocolVersionProp, "2.8") + clusterInstance.start() + + // Create a thread that continuously tries to get next producer ID + val running = new AtomicBoolean(true) + val doneLatch = new CompletableFuture[ArrayBuffer[Long]]() + Executors.newSingleThreadExecutor().submit(new Runnable() { + override def run(): Unit = { + val collectedIds = ArrayBuffer[Long]() + while (running.get) { + clusterInstance.brokerSocketServers().stream().forEach( broker => { + try { + collectedIds += nextProducerId(broker, clusterInstance.clientListener()) + } catch { + case _: SocketException => // Expected during rolling restart + case t: Throwable => // Not expected + doneLatch.completeExceptionally(t) + return + } + }) + } + doneLatch.complete(collectedIds) + } + }) + + + Thread.sleep(100) + + clusterInstance.config().serverProperties().put(KafkaConfig.InterBrokerProtocolVersionProp, "3.0") + clusterInstance.rollingBrokerRestart() + + running.set(false) + val ids = doneLatch.get() + assertEquals(ids.size, ids.distinct.size) // ensure no duplicates + clusterInstance.stop() + } + private def nextProducerId(broker: SocketServer, listener: ListenerName): Long = { val data = new InitProducerIdRequestData() .setProducerEpoch(RecordBatch.NO_PRODUCER_EPOCH) diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/ProducerIdManagerTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/ProducerIdManagerTest.scala index 8b0917f4506be..a49ce19d1002e 100644 --- a/core/src/test/scala/unit/kafka/coordinator/transaction/ProducerIdManagerTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/transaction/ProducerIdManagerTest.scala @@ -17,11 +17,13 @@ package kafka.coordinator.transaction import kafka.server.BrokerToControllerChannelManager +import kafka.zk.{KafkaZkClient, ProducerIdBlockZNode} import org.apache.kafka.common.KafkaException import org.apache.kafka.common.message.AllocateProducerIdsResponseData import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.AllocateProducerIdsResponse -import org.easymock.EasyMock +import org.apache.kafka.server.common.ProducerIdsBlock +import org.easymock.{Capture, EasyMock} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Test import org.junit.jupiter.params.ParameterizedTest @@ -32,6 +34,7 @@ import java.util.stream.IntStream class ProducerIdManagerTest { var brokerToController: BrokerToControllerChannelManager = EasyMock.niceMock(classOf[BrokerToControllerChannelManager]) + val zkClient: KafkaZkClient = EasyMock.createNiceMock(classOf[KafkaZkClient]) // Mutable test implementation that lets us easily set the idStart and error class MockProducerIdManager(val brokerId: Int, var idStart: Long, val idLen: Int, var error: Errors = Errors.NONE) @@ -49,6 +52,60 @@ class ProducerIdManagerTest { } } + @Test + def testGetProducerIdZk(): Unit = { + var zkVersion: Option[Int] = None + var data: Array[Byte] = null + EasyMock.expect(zkClient.getDataAndVersion(EasyMock.anyString)).andAnswer(() => + zkVersion.map(Some(data) -> _).getOrElse(None, 0)).anyTimes() + + val capturedVersion: Capture[Int] = EasyMock.newCapture() + val capturedData: Capture[Array[Byte]] = EasyMock.newCapture() + EasyMock.expect(zkClient.conditionalUpdatePath(EasyMock.anyString(), + EasyMock.capture(capturedData), + EasyMock.capture(capturedVersion), + EasyMock.anyObject[Option[(KafkaZkClient, String, Array[Byte]) => (Boolean, Int)]]) + ).andAnswer(() => { + val newZkVersion = capturedVersion.getValue + 1 + zkVersion = Some(newZkVersion) + data = capturedData.getValue + (true, newZkVersion) + }).anyTimes() + + EasyMock.replay(zkClient) + + val manager1 = new ZkProducerIdManager(0, zkClient) + val manager2 = new ZkProducerIdManager(1, zkClient) + + val pid1 = manager1.generateProducerId() + val pid2 = manager2.generateProducerId() + + assertEquals(0, pid1) + assertEquals(ProducerIdsBlock.PRODUCER_ID_BLOCK_SIZE, pid2) + + for (i <- 1L until ProducerIdsBlock.PRODUCER_ID_BLOCK_SIZE) + assertEquals(pid1 + i, manager1.generateProducerId()) + + for (i <- 1L until ProducerIdsBlock.PRODUCER_ID_BLOCK_SIZE) + assertEquals(pid2 + i, manager2.generateProducerId()) + + assertEquals(pid2 + ProducerIdsBlock.PRODUCER_ID_BLOCK_SIZE, manager1.generateProducerId()) + assertEquals(pid2 + ProducerIdsBlock.PRODUCER_ID_BLOCK_SIZE * 2, manager2.generateProducerId()) + + EasyMock.reset(zkClient) + } + + @Test + def testExceedProducerIdLimitZk(): Unit = { + EasyMock.expect(zkClient.getDataAndVersion(EasyMock.anyString)).andAnswer(() => { + val json = ProducerIdBlockZNode.generateProducerIdBlockJson( + new ProducerIdsBlock(0, Long.MaxValue - ProducerIdsBlock.PRODUCER_ID_BLOCK_SIZE, ProducerIdsBlock.PRODUCER_ID_BLOCK_SIZE)) + (Some(json), 0) + }).anyTimes() + EasyMock.replay(zkClient) + assertThrows(classOf[KafkaException], () => new ZkProducerIdManager(0, zkClient)) + } + @ParameterizedTest @ValueSource(ints = Array(1, 2, 10)) def testContiguousIds(idBlockLen: Int): Unit = { diff --git a/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala b/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala index 5369fdd66fae7..6fc18f0d10c68 100755 --- a/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala +++ b/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala @@ -161,8 +161,19 @@ abstract class KafkaServerTestHarness extends ZooKeeperTestHarness { /** * Restart any dead brokers */ - def restartDeadBrokers(): Unit = { + def restartDeadBrokers(reconfigure: Boolean = false): Unit = { + if (reconfigure) { + instanceConfigs = null + } for(i <- servers.indices if !alive(i)) { + if (reconfigure) { + servers(i) = TestUtils.createServer( + configs(i), + time = brokerTime(configs(i).brokerId), + threadNamePrefix = None, + enableForwarding + ) + } servers(i).startup() alive(i) = true } From 19154529d95227eb7182b6e166c300e586c03f34 Mon Sep 17 00:00:00 2001 From: David Arthur Date: Wed, 12 May 2021 13:24:13 -0400 Subject: [PATCH 10/20] Ensure the broker to controller channel is shutdown --- core/src/main/scala/kafka/server/KafkaServer.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index b99b36caebf18..98cadca4c3c22 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -266,11 +266,11 @@ class KafkaServer( threadNamePrefix = threadNamePrefix, retryTimeoutMs = config.requestTimeoutMs.longValue) brokerToControllerManager.start() + clientToControllerChannelManager = Some(brokerToControllerManager) /* start forwarding manager */ if (enableForwarding) { this.forwardingManager = Some(ForwardingManager(brokerToControllerManager)) - clientToControllerChannelManager = Some(brokerToControllerManager) } val apiVersionManager = ApiVersionManager( From d5ceeab56f1188d4df4abca2fd58f47c819d967c Mon Sep 17 00:00:00 2001 From: David Arthur Date: Wed, 12 May 2021 14:54:37 -0400 Subject: [PATCH 11/20] Remove redundant test case --- .../ProducerIdsIntegrationTest.scala | 22 +++++-------------- 1 file changed, 5 insertions(+), 17 deletions(-) diff --git a/core/src/test/scala/integration/kafka/coordinator/transaction/ProducerIdsIntegrationTest.scala b/core/src/test/scala/integration/kafka/coordinator/transaction/ProducerIdsIntegrationTest.scala index ce61be7c44b16..1f213fd2a4558 100644 --- a/core/src/test/scala/integration/kafka/coordinator/transaction/ProducerIdsIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/coordinator/transaction/ProducerIdsIntegrationTest.scala @@ -30,7 +30,7 @@ import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue} import org.junit.jupiter.api.BeforeEach import org.junit.jupiter.api.extension.ExtendWith -import java.net.SocketException +import java.io.IOException import java.util.concurrent.atomic.AtomicBoolean import java.util.concurrent.{CompletableFuture, Executors} import java.util.stream.{Collectors, IntStream} @@ -63,17 +63,8 @@ class ProducerIdsIntegrationTest { } } - @ClusterTest(clusterType = Type.ZK, brokers = 1) - def testNewBlockOnRestart(clusterInstance: ClusterInstance): Unit = { - val id0 = nextProducerId(clusterInstance.anyBrokerSocketServer(), clusterInstance.clientListener()) - clusterInstance.rollingBrokerRestart() - val id1 = nextProducerId(clusterInstance.anyBrokerSocketServer(), clusterInstance.clientListener()) - assertEquals(0, id0) - assertEquals(1000, id1) - } - @ClusterTest(clusterType = Type.ZK, brokers = 3, autoStart = AutoStart.NO) - def testRollingUpgrade(clusterInstance: ClusterInstance): Unit = { + def testBumpIBP(clusterInstance: ClusterInstance): Unit = { clusterInstance.config().serverProperties().put(KafkaConfig.InterBrokerProtocolVersionProp, "2.8") clusterInstance.start() @@ -88,8 +79,8 @@ class ProducerIdsIntegrationTest { try { collectedIds += nextProducerId(broker, clusterInstance.clientListener()) } catch { - case _: SocketException => // Expected during rolling restart - case t: Throwable => // Not expected + case _: IOException => // Expected during rolling restart + case t: Throwable => doneLatch.completeExceptionally(t) return } @@ -99,15 +90,12 @@ class ProducerIdsIntegrationTest { } }) - - Thread.sleep(100) - clusterInstance.config().serverProperties().put(KafkaConfig.InterBrokerProtocolVersionProp, "3.0") clusterInstance.rollingBrokerRestart() running.set(false) val ids = doneLatch.get() - assertEquals(ids.size, ids.distinct.size) // ensure no duplicates + assertEquals(ids.size, ids.distinct.size, "Found duplicate producer IDs") clusterInstance.stop() } From e72431b60971848eebcb13161db9a5e68de86640 Mon Sep 17 00:00:00 2001 From: David Arthur Date: Wed, 12 May 2021 15:39:05 -0400 Subject: [PATCH 12/20] Add per-broker override to testkit --- .../test/java/kafka/test/ClusterConfig.java | 6 ++ .../junit/RaftClusterInvocationContext.java | 10 +++- .../junit/ZkClusterInvocationContext.java | 10 ++++ .../test/java/kafka/testkit/BrokerNode.java | 8 +++ .../kafka/testkit/KafkaClusterTestKit.java | 1 + .../ProducerIdsIntegrationTest.scala | 58 ++++++------------- 6 files changed, 49 insertions(+), 44 deletions(-) diff --git a/core/src/test/java/kafka/test/ClusterConfig.java b/core/src/test/java/kafka/test/ClusterConfig.java index 7c14ed252698a..20b74cf43244b 100644 --- a/core/src/test/java/kafka/test/ClusterConfig.java +++ b/core/src/test/java/kafka/test/ClusterConfig.java @@ -21,6 +21,7 @@ import org.apache.kafka.common.security.auth.SecurityProtocol; import java.io.File; +import java.util.HashMap; import java.util.LinkedHashMap; import java.util.Map; import java.util.Optional; @@ -48,6 +49,7 @@ public class ClusterConfig { private final Properties adminClientProperties = new Properties(); private final Properties saslServerProperties = new Properties(); private final Properties saslClientProperties = new Properties(); + private final Map perBrokerOverrideProperties = new HashMap<>(); ClusterConfig(Type type, int brokers, int controllers, String name, boolean autoStart, SecurityProtocol securityProtocol, String listenerName, File trustStoreFile, @@ -123,6 +125,10 @@ public Optional ibp() { return Optional.ofNullable(ibp); } + public Properties brokerServerProperties(int brokerId) { + return perBrokerOverrideProperties.computeIfAbsent(brokerId, __ -> new Properties()); + } + public Map nameTags() { Map tags = new LinkedHashMap<>(3); name().ifPresent(name -> tags.put("Name", name)); diff --git a/core/src/test/java/kafka/test/junit/RaftClusterInvocationContext.java b/core/src/test/java/kafka/test/junit/RaftClusterInvocationContext.java index 4fda43ac172ba..fc6b55736c3fc 100644 --- a/core/src/test/java/kafka/test/junit/RaftClusterInvocationContext.java +++ b/core/src/test/java/kafka/test/junit/RaftClusterInvocationContext.java @@ -75,10 +75,14 @@ public String getDisplayName(int invocationIndex) { public List getAdditionalExtensions() { return Arrays.asList( (BeforeTestExecutionCallback) context -> { - KafkaClusterTestKit.Builder builder = new KafkaClusterTestKit.Builder( - new TestKitNodes.Builder(). + TestKitNodes nodes = new TestKitNodes.Builder(). setNumBrokerNodes(clusterConfig.numBrokers()). - setNumControllerNodes(clusterConfig.numControllers()).build()); + setNumControllerNodes(clusterConfig.numControllers()).build(); + nodes.brokerNodes().forEach((brokerId, brokerNode) -> { + clusterConfig.brokerServerProperties(brokerId).forEach( + (key, value) -> brokerNode.propertyOverrides().put(key.toString(), value.toString())); + }); + KafkaClusterTestKit.Builder builder = new KafkaClusterTestKit.Builder(nodes); // Copy properties into the TestKit builder clusterConfig.serverProperties().forEach((key, value) -> builder.setConfigProp(key.toString(), value.toString())); diff --git a/core/src/test/java/kafka/test/junit/ZkClusterInvocationContext.java b/core/src/test/java/kafka/test/junit/ZkClusterInvocationContext.java index cd3ff94f2e84a..9c208d1a88cd9 100644 --- a/core/src/test/java/kafka/test/junit/ZkClusterInvocationContext.java +++ b/core/src/test/java/kafka/test/junit/ZkClusterInvocationContext.java @@ -33,6 +33,7 @@ import org.junit.jupiter.api.extension.TestTemplateInvocationContext; import scala.Option; import scala.collection.JavaConverters; +import scala.collection.Seq; import scala.compat.java8.OptionConverters; import java.io.File; @@ -90,6 +91,15 @@ public List getAdditionalExtensions() { // This is what tests normally extend from to start a cluster, here we create it anonymously and // configure the cluster using values from ClusterConfig IntegrationTestHarness cluster = new IntegrationTestHarness() { + + @Override + public void modifyConfigs(Seq props) { + super.modifyConfigs(props); + for (int i = 0; i < props.length(); i++) { + props.apply(i).putAll(clusterConfig.brokerServerProperties(i)); + } + } + @Override public Properties serverConfig() { Properties props = clusterConfig.serverProperties(); diff --git a/core/src/test/java/kafka/testkit/BrokerNode.java b/core/src/test/java/kafka/testkit/BrokerNode.java index 0b5859404b346..32bd51b4b56dc 100644 --- a/core/src/test/java/kafka/testkit/BrokerNode.java +++ b/core/src/test/java/kafka/testkit/BrokerNode.java @@ -21,7 +21,9 @@ import java.util.ArrayList; import java.util.Collections; +import java.util.HashMap; import java.util.List; +import java.util.Map; public class BrokerNode implements TestKitNode { public static class Builder { @@ -68,6 +70,7 @@ public BrokerNode build() { private final Uuid incarnationId; private final String metadataDirectory; private final List logDataDirectories; + private final Map propertyOverrides; BrokerNode(int id, Uuid incarnationId, @@ -77,6 +80,7 @@ public BrokerNode build() { this.incarnationId = incarnationId; this.metadataDirectory = metadataDirectory; this.logDataDirectories = new ArrayList<>(logDataDirectories); + this.propertyOverrides = new HashMap<>(); } @Override @@ -96,4 +100,8 @@ public String metadataDirectory() { public List logDataDirectories() { return logDataDirectories; } + + public Map propertyOverrides() { + return propertyOverrides; + } } diff --git a/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java b/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java index 87083a240c2d0..bf2789c640fa9 100644 --- a/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java +++ b/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java @@ -220,6 +220,7 @@ metaProperties, config, new MetadataRecordSerde(), metadataPartition, KafkaRaftS // Just like above, we set a placeholder voter list here until we // find out what ports the controllers picked. props.put(RaftConfig.QUORUM_VOTERS_CONFIG, uninitializedQuorumVotersString); + props.putAll(node.propertyOverrides()); KafkaConfig config = new KafkaConfig(props, false, Option.empty()); String threadNamePrefix = String.format("broker%d_", node.id()); diff --git a/core/src/test/scala/integration/kafka/coordinator/transaction/ProducerIdsIntegrationTest.scala b/core/src/test/scala/integration/kafka/coordinator/transaction/ProducerIdsIntegrationTest.scala index 1f213fd2a4558..ceb0bd349b533 100644 --- a/core/src/test/scala/integration/kafka/coordinator/transaction/ProducerIdsIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/coordinator/transaction/ProducerIdsIntegrationTest.scala @@ -30,11 +30,8 @@ import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue} import org.junit.jupiter.api.BeforeEach import org.junit.jupiter.api.extension.ExtendWith -import java.io.IOException -import java.util.concurrent.atomic.AtomicBoolean -import java.util.concurrent.{CompletableFuture, Executors} import java.util.stream.{Collectors, IntStream} -import scala.collection.mutable.ArrayBuffer +import scala.jdk.CollectionConverters._ @ExtendWith(value = Array(classOf[ClusterTestExtensions])) class ProducerIdsIntegrationTest { @@ -49,10 +46,24 @@ class ProducerIdsIntegrationTest { new ClusterTest(clusterType = Type.ZK, brokers = 3, ibp = "2.8"), new ClusterTest(clusterType = Type.ZK, brokers = 3, ibp = "3.0-IV0") )) - def testNonOverlapping(clusterInstance: ClusterInstance): Unit = { + def testUniqueProducerIds(clusterInstance: ClusterInstance): Unit = { + verifyNonOverlappingAndUniqueIds(clusterInstance) + } + + @ClusterTest(clusterType = Type.ZK, brokers = 3, autoStart = AutoStart.NO) + def testUniqueProducerIdsBumpIBP(clusterInstance: ClusterInstance): Unit = { + clusterInstance.config().serverProperties().put(KafkaConfig.InterBrokerProtocolVersionProp, "2.8") + clusterInstance.config().brokerServerProperties(0).put(KafkaConfig.InterBrokerProtocolVersionProp, "3.0-IV0") + clusterInstance.start() + verifyNonOverlappingAndUniqueIds(clusterInstance) + clusterInstance.stop() + } + + private def verifyNonOverlappingAndUniqueIds(clusterInstance: ClusterInstance): Unit = { + // Request enough PIDs from each broker to ensure each broker generates two PID blocks val ids = clusterInstance.brokerSocketServers().stream().flatMap( broker => { IntStream.range(0, 1001).parallel().mapToObj( _ => nextProducerId(broker, clusterInstance.clientListener())) - }).collect(Collectors.toSet[Long]) + }).collect(Collectors.toSet[Long]).asScala.toSeq assertEquals(3003, ids.size) @@ -61,42 +72,7 @@ class ProducerIdsIntegrationTest { expectedIds.foreach { id => assertTrue(ids.contains(id), s"Expected to see $id in $idsAsString") } - } - - @ClusterTest(clusterType = Type.ZK, brokers = 3, autoStart = AutoStart.NO) - def testBumpIBP(clusterInstance: ClusterInstance): Unit = { - clusterInstance.config().serverProperties().put(KafkaConfig.InterBrokerProtocolVersionProp, "2.8") - clusterInstance.start() - - // Create a thread that continuously tries to get next producer ID - val running = new AtomicBoolean(true) - val doneLatch = new CompletableFuture[ArrayBuffer[Long]]() - Executors.newSingleThreadExecutor().submit(new Runnable() { - override def run(): Unit = { - val collectedIds = ArrayBuffer[Long]() - while (running.get) { - clusterInstance.brokerSocketServers().stream().forEach( broker => { - try { - collectedIds += nextProducerId(broker, clusterInstance.clientListener()) - } catch { - case _: IOException => // Expected during rolling restart - case t: Throwable => - doneLatch.completeExceptionally(t) - return - } - }) - } - doneLatch.complete(collectedIds) - } - }) - - clusterInstance.config().serverProperties().put(KafkaConfig.InterBrokerProtocolVersionProp, "3.0") - clusterInstance.rollingBrokerRestart() - - running.set(false) - val ids = doneLatch.get() assertEquals(ids.size, ids.distinct.size, "Found duplicate producer IDs") - clusterInstance.stop() } private def nextProducerId(broker: SocketServer, listener: ListenerName): Long = { From 726ab0875efe12eed06ae668e9e315c8574c82df Mon Sep 17 00:00:00 2001 From: David Arthur Date: Wed, 12 May 2021 17:53:12 -0400 Subject: [PATCH 13/20] Fix an integration test --- core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala index 35f6ff8b8c953..f1dcd5f3430ef 100644 --- a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala +++ b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala @@ -641,7 +641,8 @@ class RequestQuotaTest extends BaseRequestTest { case ApiKeys.LIST_TRANSACTIONS => new ListTransactionsRequest.Builder(new ListTransactionsRequestData()) - + case ApiKeys.ALLOCATE_PRODUCER_IDS => + new AllocateProducerIdsRequest.Builder(new AllocateProducerIdsRequestData()) case _ => throw new IllegalArgumentException("Unsupported API key " + apiKey) } From 86d7d8d115d8f7336c196fc31ec258cf05ff6106 Mon Sep 17 00:00:00 2001 From: David Arthur Date: Fri, 14 May 2021 11:29:44 -0400 Subject: [PATCH 14/20] Fix dependencies in KafkaServer --- .../main/scala/kafka/server/KafkaServer.scala | 18 ++++++++++-------- 1 file changed, 10 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 98cadca4c3c22..659f369f67672 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -134,7 +134,7 @@ class KafkaServer( var autoTopicCreationManager: AutoTopicCreationManager = null - var clientToControllerChannelManager: Option[BrokerToControllerChannelManager] = None + var clientToControllerChannelManager: BrokerToControllerChannelManager = null var alterIsrManager: AlterIsrManager = null @@ -257,7 +257,7 @@ class KafkaServer( tokenCache = new DelegationTokenCache(ScramMechanism.mechanismNames) credentialProvider = new CredentialProvider(ScramMechanism.mechanismNames, tokenCache) - val brokerToControllerManager = BrokerToControllerChannelManager( + clientToControllerChannelManager = BrokerToControllerChannelManager( controllerNodeProvider = MetadataCacheControllerNodeProvider(config, metadataCache), time = time, metrics = metrics, @@ -265,12 +265,13 @@ class KafkaServer( channelName = "forwarding", threadNamePrefix = threadNamePrefix, retryTimeoutMs = config.requestTimeoutMs.longValue) - brokerToControllerManager.start() - clientToControllerChannelManager = Some(brokerToControllerManager) + clientToControllerChannelManager.start() /* start forwarding manager */ + var autoTopicCreationChannel = Option.empty[BrokerToControllerChannelManager] if (enableForwarding) { - this.forwardingManager = Some(ForwardingManager(brokerToControllerManager)) + this.forwardingManager = Some(ForwardingManager(clientToControllerChannelManager)) + autoTopicCreationChannel = Some(clientToControllerChannelManager) } val apiVersionManager = ApiVersionManager( @@ -336,7 +337,7 @@ class KafkaServer( ProducerIdGenerator( config.brokerId, brokerEpochSupplier = () => kafkaController.brokerEpoch, - brokerToControllerManager, + clientToControllerChannelManager, config.requestTimeoutMs ) } else { @@ -354,7 +355,7 @@ class KafkaServer( config, metadataCache, threadNamePrefix, - clientToControllerChannelManager, + autoTopicCreationChannel, Some(adminManager), Some(kafkaController), groupCoordinator, @@ -710,7 +711,8 @@ class KafkaServer( if (alterIsrManager != null) CoreUtils.swallow(alterIsrManager.shutdown(), this) - CoreUtils.swallow(clientToControllerChannelManager.foreach(_.shutdown()), this) + if (clientToControllerChannelManager != null) + CoreUtils.swallow(clientToControllerChannelManager.shutdown(), this) if (logManager != null) CoreUtils.swallow(logManager.shutdown(), this) From e88a37a907214bc62af585b010438f44b9288db5 Mon Sep 17 00:00:00 2001 From: David Arthur Date: Fri, 14 May 2021 14:56:21 -0400 Subject: [PATCH 15/20] Fixed throttle integration test --- .../src/test/scala/unit/kafka/server/RequestQuotaTest.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala index f1dcd5f3430ef..5475b406d5ded 100644 --- a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala +++ b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala @@ -131,7 +131,7 @@ class RequestQuotaTest extends BaseRequestTest { @Test def testResponseThrottleTime(): Unit = { - for (apiKey <- RequestQuotaTest.ClientActions) + for (apiKey <- RequestQuotaTest.ClientActions ++ RequestQuotaTest.ClusterActionsWithThrottle) submitTest(apiKey, () => checkRequestThrottleTime(apiKey)) waitAndCheckResults() @@ -160,7 +160,7 @@ class RequestQuotaTest extends BaseRequestTest { @Test def testExemptRequestTime(): Unit = { - for (apiKey <- RequestQuotaTest.ClusterActions) { + for (apiKey <- RequestQuotaTest.ClusterActions -- RequestQuotaTest.ClusterActionsWithThrottle) { submitTest(apiKey, () => checkExemptRequestMetric(apiKey)) } @@ -641,6 +641,7 @@ class RequestQuotaTest extends BaseRequestTest { case ApiKeys.LIST_TRANSACTIONS => new ListTransactionsRequest.Builder(new ListTransactionsRequestData()) + case ApiKeys.ALLOCATE_PRODUCER_IDS => new AllocateProducerIdsRequest.Builder(new AllocateProducerIdsRequestData()) case _ => @@ -763,6 +764,7 @@ class RequestQuotaTest extends BaseRequestTest { object RequestQuotaTest { val ClusterActions = ApiKeys.zkBrokerApis.asScala.filter(_.clusterAction).toSet + val ClusterActionsWithThrottle = Set(ApiKeys.ALLOCATE_PRODUCER_IDS) val SaslActions = Set(ApiKeys.SASL_HANDSHAKE, ApiKeys.SASL_AUTHENTICATE) val ClientActions = ApiKeys.zkBrokerApis.asScala.toSet -- ClusterActions -- SaslActions From 8afade49875eb28418f0f4baad55eaf38cd9afa7 Mon Sep 17 00:00:00 2001 From: David Arthur Date: Fri, 14 May 2021 14:56:41 -0400 Subject: [PATCH 16/20] Don't eaglery fetch first ID block when using the RPC --- .../transaction/ProducerIdManager.scala | 19 +++++++++++-------- .../ProducerIdsIntegrationTest.scala | 8 ++++---- 2 files changed, 15 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/kafka/coordinator/transaction/ProducerIdManager.scala b/core/src/main/scala/kafka/coordinator/transaction/ProducerIdManager.scala index 64d1133f831f2..4c4dc0f237622 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/ProducerIdManager.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/ProducerIdManager.scala @@ -143,18 +143,21 @@ class ProducerIdManager(brokerId: Int, private val nextProducerIdBlock = new ArrayBlockingQueue[Try[ProducerIdsBlock]](1) private val requestInFlight = new AtomicBoolean(false) private var currentProducerIdBlock: ProducerIdsBlock = ProducerIdsBlock.EMPTY - private var nextProducerId: Long = 0L - - // Send an initial request to get the first block - maybeRequestNextBlock() + private var nextProducerId: Long = -1L override def generateProducerId(): Long = { this synchronized { - nextProducerId += 1 - - // Check if we need to fetch the next block - if (nextProducerId >= (currentProducerIdBlock.producerIdStart + currentProducerIdBlock.producerIdLen * ProducerIdGenerator.PidPrefetchThreshold)) { + if (nextProducerId == -1L) { + // Send an initial request to get the first block maybeRequestNextBlock() + nextProducerId = 0L + } else { + nextProducerId += 1 + + // Check if we need to fetch the next block + if (nextProducerId >= (currentProducerIdBlock.producerIdStart + currentProducerIdBlock.producerIdLen * ProducerIdGenerator.PidPrefetchThreshold)) { + maybeRequestNextBlock() + } } // If we've exhausted the current block, grab the next block (waiting if necessary) diff --git a/core/src/test/scala/integration/kafka/coordinator/transaction/ProducerIdsIntegrationTest.scala b/core/src/test/scala/integration/kafka/coordinator/transaction/ProducerIdsIntegrationTest.scala index ceb0bd349b533..09da2feffc501 100644 --- a/core/src/test/scala/integration/kafka/coordinator/transaction/ProducerIdsIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/coordinator/transaction/ProducerIdsIntegrationTest.scala @@ -63,12 +63,12 @@ class ProducerIdsIntegrationTest { // Request enough PIDs from each broker to ensure each broker generates two PID blocks val ids = clusterInstance.brokerSocketServers().stream().flatMap( broker => { IntStream.range(0, 1001).parallel().mapToObj( _ => nextProducerId(broker, clusterInstance.clientListener())) - }).collect(Collectors.toSet[Long]).asScala.toSeq + }).collect(Collectors.toList[Long]).asScala.toSeq - assertEquals(3003, ids.size) + assertEquals(3003, ids.size, "Expected 3003 IDs") - val expectedIds = Set(0L, 999L, 1000L, 1999L, 2000L, 2999L, 3000L, 4000L, 5000L) - val idsAsString = expectedIds.mkString(", ") + val expectedIds = Set(0L, 1000L, 2000L, 3000L, 4000L, 5000L) + val idsAsString = ids.mkString(", ") expectedIds.foreach { id => assertTrue(ids.contains(id), s"Expected to see $id in $idsAsString") } From e37571b1710779dd27608e086e66a623f384cbe2 Mon Sep 17 00:00:00 2001 From: David Arthur Date: Tue, 18 May 2021 12:52:30 -0400 Subject: [PATCH 17/20] More PR feedback --- .../resources/common/message/AllocateProducerIdsResponse.json | 2 +- core/src/main/scala/kafka/controller/KafkaController.scala | 2 +- .../kafka/coordinator/transaction/ProducerIdManager.scala | 3 +++ 3 files changed, 5 insertions(+), 2 deletions(-) diff --git a/clients/src/main/resources/common/message/AllocateProducerIdsResponse.json b/clients/src/main/resources/common/message/AllocateProducerIdsResponse.json index b6b5948f7d00d..0d849c098568b 100644 --- a/clients/src/main/resources/common/message/AllocateProducerIdsResponse.json +++ b/clients/src/main/resources/common/message/AllocateProducerIdsResponse.json @@ -24,7 +24,7 @@ "about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." }, { "name": "ErrorCode", "type": "int16", "versions": "0+", "about": "The top level response error code" }, - { "name": "ProducerIdStart", "type": "int64", "versions": "0+", + { "name": "ProducerIdStart", "type": "int64", "versions": "0+", "entityType": "producerId", "about": "The first producer ID in this range, inclusive"}, { "name": "ProducerIdLen", "type": "int32", "versions": "0+", "about": "The number of producer IDs in this range"} diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index df6d3d6d67648..654649b4f48c2 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -2402,7 +2402,7 @@ class KafkaController(val config: KafkaConfig, val brokerEpochOpt = controllerContext.liveBrokerIdAndEpochs.get(brokerId) if (brokerEpochOpt.isEmpty) { warn(s"Ignoring AllocateProducerIds due to unknown broker $brokerId") - callback.apply(Left(Errors.STALE_BROKER_EPOCH)) + callback.apply(Left(Errors.BROKER_ID_NOT_REGISTERED)) return } diff --git a/core/src/main/scala/kafka/coordinator/transaction/ProducerIdManager.scala b/core/src/main/scala/kafka/coordinator/transaction/ProducerIdManager.scala index 4c4dc0f237622..358a0c1d29a0f 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/ProducerIdManager.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/ProducerIdManager.scala @@ -221,6 +221,9 @@ class ProducerIdManager(brokerId: Int, case Errors.STALE_BROKER_EPOCH => warn("Our broker epoch was stale, trying again.") maybeRequestNextBlock() + case Errors.BROKER_ID_NOT_REGISTERED => + warn("Our broker ID is not yet known by the controller, trying again.") + maybeRequestNextBlock() case e: Errors => warn("Had an unknown error from the controller, giving up.") nextProducerIdBlock.put(Failure(e.exception())) From 437ee4bb01cecf8e20a3db7eabfb2de64f519d7d Mon Sep 17 00:00:00 2001 From: David Arthur Date: Thu, 20 May 2021 09:50:00 -0400 Subject: [PATCH 18/20] Clean up ZK impl --- ...nager.scala => RPCProducerIdManager.scala} | 37 ++++++++++--------- .../main/scala/kafka/server/KafkaServer.scala | 4 +- .../junit/ZkClusterInvocationContext.java | 11 +++--- ...t.scala => RPCProducerIdManagerTest.scala} | 4 +- 4 files changed, 29 insertions(+), 27 deletions(-) rename core/src/main/scala/kafka/coordinator/transaction/{ProducerIdManager.scala => RPCProducerIdManager.scala} (90%) rename core/src/test/scala/unit/kafka/coordinator/transaction/{ProducerIdManagerTest.scala => RPCProducerIdManagerTest.scala} (98%) diff --git a/core/src/main/scala/kafka/coordinator/transaction/ProducerIdManager.scala b/core/src/main/scala/kafka/coordinator/transaction/RPCProducerIdManager.scala similarity index 90% rename from core/src/main/scala/kafka/coordinator/transaction/ProducerIdManager.scala rename to core/src/main/scala/kafka/coordinator/transaction/RPCProducerIdManager.scala index 358a0c1d29a0f..fb32681850274 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/ProducerIdManager.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/RPCProducerIdManager.scala @@ -43,16 +43,16 @@ object ProducerIdGenerator { val PidPrefetchThreshold = 0.90 // Creates a ProducerIdGenerate that directly interfaces with ZooKeeper, IBP < 3.0-IV0 - def apply(brokerId: Int, zkClient: KafkaZkClient): ZkProducerIdManager = { + def zk(brokerId: Int, zkClient: KafkaZkClient): ZkProducerIdManager = { new ZkProducerIdManager(brokerId, zkClient) } // Creates a ProducerIdGenerate that uses AllocateProducerIds RPC, IBP >= 3.0-IV0 - def apply(brokerId: Int, + def rpc(brokerId: Int, brokerEpochSupplier: () => Long, controllerChannel: BrokerToControllerChannelManager, - maxWaitMs: Int): ProducerIdManager = { - new ProducerIdManager(brokerId, brokerEpochSupplier, controllerChannel, maxWaitMs) + maxWaitMs: Int): RPCProducerIdManager = { + new RPCProducerIdManager(brokerId, brokerEpochSupplier, controllerChannel, maxWaitMs) } } @@ -106,39 +106,42 @@ object ZkProducerIdManager { class ZkProducerIdManager(brokerId: Int, zkClient: KafkaZkClient) extends ProducerIdGenerator with Logging { + this.logIdent = "[ZK ProducerId Manager " + brokerId + "]: " + private var currentProducerIdBlock: ProducerIdsBlock = ProducerIdsBlock.EMPTY - private var nextProducerId: Long = -1L + private var nextProducerId: Long = _ // grab the first block of producerIds this synchronized { - getNewProducerIdBlock() + allocateNewProducerIdBlock() nextProducerId = currentProducerIdBlock.producerIdStart } - private def getNewProducerIdBlock(): Unit = { - currentProducerIdBlock = ZkProducerIdManager.getNewProducerIdBlock(brokerId, zkClient, this) + private def allocateNewProducerIdBlock(): Unit = { + this synchronized { + currentProducerIdBlock = ZkProducerIdManager.getNewProducerIdBlock(brokerId, zkClient, this) + } } def generateProducerId(): Long = { this synchronized { // grab a new block of producerIds if this block has been exhausted if (nextProducerId > currentProducerIdBlock.producerIdEnd) { - getNewProducerIdBlock() - nextProducerId = currentProducerIdBlock.producerIdStart + 1 - } else { - nextProducerId += 1 + allocateNewProducerIdBlock() + nextProducerId = currentProducerIdBlock.producerIdStart } + nextProducerId += 1 nextProducerId - 1 } } } -class ProducerIdManager(brokerId: Int, - brokerEpochSupplier: () => Long, - controllerChannel: BrokerToControllerChannelManager, - maxWaitMs: Int) extends ProducerIdGenerator with Logging { +class RPCProducerIdManager(brokerId: Int, + brokerEpochSupplier: () => Long, + controllerChannel: BrokerToControllerChannelManager, + maxWaitMs: Int) extends ProducerIdGenerator with Logging { - this.logIdent = "[ProducerId Manager " + brokerId + "]: " + this.logIdent = "[RPC ProducerId Manager " + brokerId + "]: " private val nextProducerIdBlock = new ArrayBlockingQueue[Try[ProducerIdsBlock]](1) private val requestInFlight = new AtomicBoolean(false) diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 659f369f67672..50427e604d35f 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -334,14 +334,14 @@ class KafkaServer( /* create producer ids manager */ val producerIdManager = if (config.interBrokerProtocolVersion.isAllocateProducerIdsSupported) { - ProducerIdGenerator( + ProducerIdGenerator.rpc( config.brokerId, brokerEpochSupplier = () => kafkaController.brokerEpoch, clientToControllerChannelManager, config.requestTimeoutMs ) } else { - ProducerIdGenerator(config.brokerId, zkClient) + ProducerIdGenerator.zk(config.brokerId, zkClient) } /* start transaction coordinator, with a separate background thread scheduler for transaction expiration and log loading */ // Hardcode Time.SYSTEM for now as some Streams tests fail otherwise, it would be good to fix the underlying issue diff --git a/core/src/test/java/kafka/test/junit/ZkClusterInvocationContext.java b/core/src/test/java/kafka/test/junit/ZkClusterInvocationContext.java index 9c208d1a88cd9..cd8cdc11a800d 100644 --- a/core/src/test/java/kafka/test/junit/ZkClusterInvocationContext.java +++ b/core/src/test/java/kafka/test/junit/ZkClusterInvocationContext.java @@ -264,14 +264,13 @@ public void stop() { @Override public void rollingBrokerRestart() { - if (started.get()) { - for (int i = 0; i < clusterReference.get().brokerCount(); i++) { - clusterReference.get().killBroker(i); - } - clusterReference.get().restartDeadBrokers(true); - } else { + if (!started.get()) { throw new IllegalStateException("Tried to restart brokers but the cluster has not been started!"); } + for (int i = 0; i < clusterReference.get().brokerCount(); i++) { + clusterReference.get().killBroker(i); + } + clusterReference.get().restartDeadBrokers(true); } } } diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/ProducerIdManagerTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/RPCProducerIdManagerTest.scala similarity index 98% rename from core/src/test/scala/unit/kafka/coordinator/transaction/ProducerIdManagerTest.scala rename to core/src/test/scala/unit/kafka/coordinator/transaction/RPCProducerIdManagerTest.scala index a49ce19d1002e..fe39a57186062 100644 --- a/core/src/test/scala/unit/kafka/coordinator/transaction/ProducerIdManagerTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/transaction/RPCProducerIdManagerTest.scala @@ -31,14 +31,14 @@ import org.junit.jupiter.params.provider.{EnumSource, ValueSource} import java.util.stream.IntStream -class ProducerIdManagerTest { +class RPCProducerIdManagerTest { var brokerToController: BrokerToControllerChannelManager = EasyMock.niceMock(classOf[BrokerToControllerChannelManager]) val zkClient: KafkaZkClient = EasyMock.createNiceMock(classOf[KafkaZkClient]) // Mutable test implementation that lets us easily set the idStart and error class MockProducerIdManager(val brokerId: Int, var idStart: Long, val idLen: Int, var error: Errors = Errors.NONE) - extends ProducerIdManager(brokerId, () => 1, brokerToController, 100) { + extends RPCProducerIdManager(brokerId, () => 1, brokerToController, 100) { override private[transaction] def sendRequest(): Unit = { if (error == Errors.NONE) { From 4c83cd70fba7b2c65c10259c6fb012633d767374 Mon Sep 17 00:00:00 2001 From: David Arthur Date: Thu, 20 May 2021 16:34:55 -0400 Subject: [PATCH 19/20] Make the naming consistent --- ...ucerIdManager.scala => ProducerIdManager.scala} | 10 +++++----- .../transaction/TransactionCoordinator.scala | 14 +++++++------- .../src/main/scala/kafka/server/BrokerServer.scala | 6 +++--- core/src/main/scala/kafka/server/KafkaServer.scala | 6 +++--- ...nagerTest.scala => ProducerIdManagerTest.scala} | 2 +- .../TransactionCoordinatorConcurrencyTest.scala | 2 +- .../transaction/TransactionCoordinatorTest.scala | 2 +- 7 files changed, 21 insertions(+), 21 deletions(-) rename core/src/main/scala/kafka/coordinator/transaction/{RPCProducerIdManager.scala => ProducerIdManager.scala} (97%) rename core/src/test/scala/unit/kafka/coordinator/transaction/{RPCProducerIdManagerTest.scala => ProducerIdManagerTest.scala} (99%) diff --git a/core/src/main/scala/kafka/coordinator/transaction/RPCProducerIdManager.scala b/core/src/main/scala/kafka/coordinator/transaction/ProducerIdManager.scala similarity index 97% rename from core/src/main/scala/kafka/coordinator/transaction/RPCProducerIdManager.scala rename to core/src/main/scala/kafka/coordinator/transaction/ProducerIdManager.scala index fb32681850274..b5d419ddf4acb 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/RPCProducerIdManager.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/ProducerIdManager.scala @@ -38,7 +38,7 @@ import scala.util.{Failure, Success, Try} * a unique block. */ -object ProducerIdGenerator { +object ProducerIdManager { // Once we reach this percentage of PIDs consumed from the current block, trigger a fetch of the next block val PidPrefetchThreshold = 0.90 @@ -56,7 +56,7 @@ object ProducerIdGenerator { } } -trait ProducerIdGenerator { +trait ProducerIdManager { def generateProducerId(): Long def shutdown() : Unit = {} } @@ -104,7 +104,7 @@ object ZkProducerIdManager { } class ZkProducerIdManager(brokerId: Int, - zkClient: KafkaZkClient) extends ProducerIdGenerator with Logging { + zkClient: KafkaZkClient) extends ProducerIdManager with Logging { this.logIdent = "[ZK ProducerId Manager " + brokerId + "]: " @@ -139,7 +139,7 @@ class ZkProducerIdManager(brokerId: Int, class RPCProducerIdManager(brokerId: Int, brokerEpochSupplier: () => Long, controllerChannel: BrokerToControllerChannelManager, - maxWaitMs: Int) extends ProducerIdGenerator with Logging { + maxWaitMs: Int) extends ProducerIdManager with Logging { this.logIdent = "[RPC ProducerId Manager " + brokerId + "]: " @@ -158,7 +158,7 @@ class RPCProducerIdManager(brokerId: Int, nextProducerId += 1 // Check if we need to fetch the next block - if (nextProducerId >= (currentProducerIdBlock.producerIdStart + currentProducerIdBlock.producerIdLen * ProducerIdGenerator.PidPrefetchThreshold)) { + if (nextProducerId >= (currentProducerIdBlock.producerIdStart + currentProducerIdBlock.producerIdLen * ProducerIdManager.PidPrefetchThreshold)) { maybeRequestNextBlock() } } diff --git a/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala b/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala index 0e3fa28098118..543e9c85c36d5 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala @@ -35,7 +35,7 @@ object TransactionCoordinator { def apply(config: KafkaConfig, replicaManager: ReplicaManager, scheduler: Scheduler, - createProducerIdGenerator: () => ProducerIdGenerator, + createProducerIdGenerator: () => ProducerIdManager, metrics: Metrics, metadataCache: MetadataCache, time: Time): TransactionCoordinator = { @@ -82,7 +82,7 @@ object TransactionCoordinator { class TransactionCoordinator(brokerId: Int, txnConfig: TransactionConfig, scheduler: Scheduler, - createProducerIdGenerator: () => ProducerIdGenerator, + createProducerIdManager: () => ProducerIdManager, txnManager: TransactionStateManager, txnMarkerChannelManager: TransactionMarkerChannelManager, time: Time, @@ -99,7 +99,7 @@ class TransactionCoordinator(brokerId: Int, /* Active flag of the coordinator */ private val isActive = new AtomicBoolean(false) - val producerIdGenerator = createProducerIdGenerator() + val producerIdManager = createProducerIdManager() def handleInitProducerId(transactionalId: String, transactionTimeoutMs: Int, @@ -109,7 +109,7 @@ class TransactionCoordinator(brokerId: Int, if (transactionalId == null) { // if the transactional id is null, then always blindly accept the request // and return a new producerId from the producerId manager - val producerId = producerIdGenerator.generateProducerId() + val producerId = producerIdManager.generateProducerId() responseCallback(InitProducerIdResult(producerId, producerEpoch = 0, Errors.NONE)) } else if (transactionalId.isEmpty) { // if transactional id is empty then return error as invalid request. This is @@ -121,7 +121,7 @@ class TransactionCoordinator(brokerId: Int, } else { val coordinatorEpochAndMetadata = txnManager.getTransactionState(transactionalId).flatMap { case None => - val producerId = producerIdGenerator.generateProducerId() + val producerId = producerIdManager.generateProducerId() val createdMetadata = new TransactionMetadata(transactionalId = transactionalId, producerId = producerId, lastProducerId = RecordBatch.NO_PRODUCER_ID, @@ -225,7 +225,7 @@ class TransactionCoordinator(brokerId: Int, // If the epoch is exhausted and the expected epoch (if provided) matches it, generate a new producer ID if (txnMetadata.isProducerEpochExhausted && expectedProducerIdAndEpoch.forall(_.epoch == txnMetadata.producerEpoch)) { - val newProducerId = producerIdGenerator.generateProducerId() + val newProducerId = producerIdManager.generateProducerId() Right(txnMetadata.prepareProducerIdRotation(newProducerId, transactionTimeoutMs, time.milliseconds(), expectedProducerIdAndEpoch.isDefined)) } else { @@ -675,7 +675,7 @@ class TransactionCoordinator(brokerId: Int, info("Shutting down.") isActive.set(false) scheduler.shutdown() - producerIdGenerator.shutdown() + producerIdManager.shutdown() txnManager.shutdown() txnMarkerChannelManager.shutdown() info("Shutdown complete.") diff --git a/core/src/main/scala/kafka/server/BrokerServer.scala b/core/src/main/scala/kafka/server/BrokerServer.scala index 49edbfe295068..8751029762b9a 100644 --- a/core/src/main/scala/kafka/server/BrokerServer.scala +++ b/core/src/main/scala/kafka/server/BrokerServer.scala @@ -25,7 +25,7 @@ import java.net.InetAddress import kafka.cluster.Broker.ServerInfo import kafka.coordinator.group.GroupCoordinator -import kafka.coordinator.transaction.{ProducerIdGenerator, TransactionCoordinator} +import kafka.coordinator.transaction.{ProducerIdManager, TransactionCoordinator} import kafka.log.LogManager import kafka.metrics.KafkaYammerMetrics import kafka.network.SocketServer @@ -375,7 +375,7 @@ class BrokerServer( } } - class TemporaryProducerIdManager() extends ProducerIdGenerator { + class TemporaryProducerIdManager() extends ProducerIdManager { val maxProducerIdsPerBrokerEpoch = 1000000 var currentOffset = -1 override def generateProducerId(): Long = { @@ -389,7 +389,7 @@ class BrokerServer( } } - def createTemporaryProducerIdManager(): ProducerIdGenerator = { + def createTemporaryProducerIdManager(): ProducerIdManager = { new TemporaryProducerIdManager() } diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 50427e604d35f..05f34159011a2 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -26,7 +26,7 @@ import kafka.cluster.Broker import kafka.common.{GenerateBrokerIdException, InconsistentBrokerIdException, InconsistentClusterIdException} import kafka.controller.KafkaController import kafka.coordinator.group.GroupCoordinator -import kafka.coordinator.transaction.{ProducerIdGenerator, TransactionCoordinator} +import kafka.coordinator.transaction.{ProducerIdManager, TransactionCoordinator} import kafka.log.LogManager import kafka.metrics.{KafkaMetricsReporter, KafkaYammerMetrics} import kafka.network.SocketServer @@ -334,14 +334,14 @@ class KafkaServer( /* create producer ids manager */ val producerIdManager = if (config.interBrokerProtocolVersion.isAllocateProducerIdsSupported) { - ProducerIdGenerator.rpc( + ProducerIdManager.rpc( config.brokerId, brokerEpochSupplier = () => kafkaController.brokerEpoch, clientToControllerChannelManager, config.requestTimeoutMs ) } else { - ProducerIdGenerator.zk(config.brokerId, zkClient) + ProducerIdManager.zk(config.brokerId, zkClient) } /* start transaction coordinator, with a separate background thread scheduler for transaction expiration and log loading */ // Hardcode Time.SYSTEM for now as some Streams tests fail otherwise, it would be good to fix the underlying issue diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/RPCProducerIdManagerTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/ProducerIdManagerTest.scala similarity index 99% rename from core/src/test/scala/unit/kafka/coordinator/transaction/RPCProducerIdManagerTest.scala rename to core/src/test/scala/unit/kafka/coordinator/transaction/ProducerIdManagerTest.scala index fe39a57186062..9232bf03c0e0e 100644 --- a/core/src/test/scala/unit/kafka/coordinator/transaction/RPCProducerIdManagerTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/transaction/ProducerIdManagerTest.scala @@ -31,7 +31,7 @@ import org.junit.jupiter.params.provider.{EnumSource, ValueSource} import java.util.stream.IntStream -class RPCProducerIdManagerTest { +class ProducerIdManagerTest { var brokerToController: BrokerToControllerChannelManager = EasyMock.niceMock(classOf[BrokerToControllerChannelManager]) val zkClient: KafkaZkClient = EasyMock.createNiceMock(classOf[KafkaZkClient]) diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorConcurrencyTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorConcurrencyTest.scala index ad337038761b4..e1786d0ee21ff 100644 --- a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorConcurrencyTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorConcurrencyTest.scala @@ -77,7 +77,7 @@ class TransactionCoordinatorConcurrencyTest extends AbstractCoordinatorConcurren for (i <- 0 until numPartitions) txnStateManager.addLoadedTransactionsToCache(i, coordinatorEpoch, new Pool[String, TransactionMetadata]()) - val pidGenerator: ProducerIdGenerator = EasyMock.createNiceMock(classOf[ProducerIdGenerator]) + val pidGenerator: ProducerIdManager = EasyMock.createNiceMock(classOf[ProducerIdManager]) EasyMock.expect(pidGenerator.generateProducerId()) .andAnswer(() => if (bumpProducerId) producerId + 1 else producerId) .anyTimes() diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorTest.scala index 72ccc8de6bf4d..f6b5e54dfe19a 100644 --- a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorTest.scala @@ -34,7 +34,7 @@ class TransactionCoordinatorTest { val time = new MockTime() var nextPid: Long = 0L - val pidGenerator: ProducerIdGenerator = EasyMock.createNiceMock(classOf[ProducerIdGenerator]) + val pidGenerator: ProducerIdManager = EasyMock.createNiceMock(classOf[ProducerIdManager]) val transactionManager: TransactionStateManager = EasyMock.createNiceMock(classOf[TransactionStateManager]) val transactionMarkerChannelManager: TransactionMarkerChannelManager = EasyMock.createNiceMock(classOf[TransactionMarkerChannelManager]) val capturedTxn: Capture[TransactionMetadata] = EasyMock.newCapture() From 3262171fa02df1f847ae5afc874133dd2582aafd Mon Sep 17 00:00:00 2001 From: David Arthur Date: Fri, 21 May 2021 10:20:21 -0400 Subject: [PATCH 20/20] Loosen test case --- .../transaction/ProducerIdsIntegrationTest.scala | 16 +++++----------- 1 file changed, 5 insertions(+), 11 deletions(-) diff --git a/core/src/test/scala/integration/kafka/coordinator/transaction/ProducerIdsIntegrationTest.scala b/core/src/test/scala/integration/kafka/coordinator/transaction/ProducerIdsIntegrationTest.scala index 09da2feffc501..6c7c248e63188 100644 --- a/core/src/test/scala/integration/kafka/coordinator/transaction/ProducerIdsIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/coordinator/transaction/ProducerIdsIntegrationTest.scala @@ -26,7 +26,7 @@ import org.apache.kafka.common.message.InitProducerIdRequestData import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.record.RecordBatch import org.apache.kafka.common.requests.{InitProducerIdRequest, InitProducerIdResponse} -import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue} +import org.junit.jupiter.api.Assertions.assertEquals import org.junit.jupiter.api.BeforeEach import org.junit.jupiter.api.extension.ExtendWith @@ -47,7 +47,7 @@ class ProducerIdsIntegrationTest { new ClusterTest(clusterType = Type.ZK, brokers = 3, ibp = "3.0-IV0") )) def testUniqueProducerIds(clusterInstance: ClusterInstance): Unit = { - verifyNonOverlappingAndUniqueIds(clusterInstance) + verifyUniqueIds(clusterInstance) } @ClusterTest(clusterType = Type.ZK, brokers = 3, autoStart = AutoStart.NO) @@ -55,23 +55,17 @@ class ProducerIdsIntegrationTest { clusterInstance.config().serverProperties().put(KafkaConfig.InterBrokerProtocolVersionProp, "2.8") clusterInstance.config().brokerServerProperties(0).put(KafkaConfig.InterBrokerProtocolVersionProp, "3.0-IV0") clusterInstance.start() - verifyNonOverlappingAndUniqueIds(clusterInstance) + verifyUniqueIds(clusterInstance) clusterInstance.stop() } - private def verifyNonOverlappingAndUniqueIds(clusterInstance: ClusterInstance): Unit = { + private def verifyUniqueIds(clusterInstance: ClusterInstance): Unit = { // Request enough PIDs from each broker to ensure each broker generates two PID blocks val ids = clusterInstance.brokerSocketServers().stream().flatMap( broker => { IntStream.range(0, 1001).parallel().mapToObj( _ => nextProducerId(broker, clusterInstance.clientListener())) }).collect(Collectors.toList[Long]).asScala.toSeq - assertEquals(3003, ids.size, "Expected 3003 IDs") - - val expectedIds = Set(0L, 1000L, 2000L, 3000L, 4000L, 5000L) - val idsAsString = ids.mkString(", ") - expectedIds.foreach { id => - assertTrue(ids.contains(id), s"Expected to see $id in $idsAsString") - } + assertEquals(3003, ids.size, "Expected exactly 3003 IDs") assertEquals(ids.size, ids.distinct.size, "Found duplicate producer IDs") }