(kafka) branch trunk updated: KAFKA-16226 Add test for concurrently updatingMetadata and fetching snapshot/cluster (#15385)

2024-02-26 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new ddfcc333f8d KAFKA-16226 Add test for concurrently updatingMetadata and 
fetching snapshot/cluster (#15385)
ddfcc333f8d is described below

commit ddfcc333f8d1c2f4ab0a88abf6d664153d6e82fd
Author: Mayank Shekhar Narula <42991652+msn-t...@users.noreply.github.com>
AuthorDate: Mon Feb 26 23:57:11 2024 +

KAFKA-16226 Add test for concurrently updatingMetadata and fetching 
snapshot/cluster (#15385)

Add test for concurrently updatingMetadata and fetching snapshot/cluster

Reviewers: Jason Gustafson 

Co-authored-by: Zhifeng Chen 
---
 .../org/apache/kafka/clients/MetadataTest.java | 106 +
 1 file changed, 106 insertions(+)

diff --git a/clients/src/test/java/org/apache/kafka/clients/MetadataTest.java 
b/clients/src/test/java/org/apache/kafka/clients/MetadataTest.java
index 600fc23ecb9..e6db9685eb5 100644
--- a/clients/src/test/java/org/apache/kafka/clients/MetadataTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/MetadataTest.java
@@ -16,6 +16,11 @@
  */
 package org.apache.kafka.clients;
 
+import java.util.OptionalInt;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
 import org.apache.kafka.common.Cluster;
 import org.apache.kafka.common.ClusterResourceListener;
 import org.apache.kafka.common.Node;
@@ -1260,6 +1265,107 @@ public class MetadataTest {
 Mockito.reset(mockListener);
 }
 
+/**
+ * Test that concurrently updating Metadata, and fetching the 
corresponding MetadataSnapshot & Cluster work as expected, i.e.
+ * snapshot & cluster contain the relevant updates.
+ */
+@Test
+public void testConcurrentUpdateAndFetchForSnapshotAndCluster() throws 
InterruptedException {
+Time time = new MockTime();
+metadata = new Metadata(refreshBackoffMs, refreshBackoffMaxMs, 
metadataExpireMs, new LogContext(), new ClusterResourceListeners());
+
+// Setup metadata with 10 nodes, 2 topics, topic1 & 2, both to be 
retained in the update. Both will have leader-epoch 100.
+int oldNodeCount = 10;
+String topic1 = "test_topic1";
+String topic2 = "test_topic2";
+TopicPartition topic1Part0 = new TopicPartition(topic1, 0);
+Map topicPartitionCounts = new HashMap<>();
+int oldPartitionCount = 1;
+topicPartitionCounts.put(topic1, oldPartitionCount);
+topicPartitionCounts.put(topic2, oldPartitionCount);
+Map topicIds = new HashMap<>();
+topicIds.put(topic1, Uuid.randomUuid());
+topicIds.put(topic2, Uuid.randomUuid());
+int oldLeaderEpoch = 100;
+MetadataResponse metadataResponse =
+RequestTestUtils.metadataUpdateWithIds("cluster", oldNodeCount, 
Collections.emptyMap(), topicPartitionCounts, _tp -> oldLeaderEpoch, topicIds);
+metadata.updateWithCurrentRequestVersion(metadataResponse, true, 
time.milliseconds());
+MetadataSnapshot snapshot = metadata.fetchMetadataSnapshot();
+Cluster cluster = metadata.fetch();
+// Validate metadata snapshot & cluster are setup as expected.
+assertEquals(cluster, snapshot.cluster());
+assertEquals(oldNodeCount, snapshot.cluster().nodes().size());
+assertEquals(oldPartitionCount, 
snapshot.cluster().partitionCountForTopic(topic1));
+assertEquals(oldPartitionCount, 
snapshot.cluster().partitionCountForTopic(topic2));
+assertEquals(OptionalInt.of(oldLeaderEpoch), 
snapshot.leaderEpochFor(topic1Part0));
+
+// Setup 6 threads, where 3 are updating metadata & 3 are reading 
snapshot/cluster.
+// Metadata will be updated with higher # of nodes, partition-counts, 
leader-epoch.
+int numThreads = 6;
+ExecutorService service = Executors.newFixedThreadPool(numThreads);
+CountDownLatch allThreadsDoneLatch = new CountDownLatch(numThreads);
+CountDownLatch atleastMetadataUpdatedOnceLatch = new CountDownLatch(1);
+AtomicReference newSnapshot = new 
AtomicReference<>();
+AtomicReference newCluster = new AtomicReference<>();
+for (int i = 0; i < numThreads; i++) {
+final int id = i + 1;
+service.execute(() -> {
+if (id % 2 == 0) { // Thread to update metadata.
+String oldClusterId = "clusterId";
+int nNodes = oldNodeCount + id;
+Map newTopicPartitionCounts = new 
HashMap<>();
+newTopicPartitionCounts.put(topic1, oldPartitionCount + 
id);
+  

(kafka) branch trunk updated: KAFKA-16226; Reduce synchronization between producer threads (#15323)

2024-02-15 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new ff90f78c700 KAFKA-16226; Reduce synchronization between producer 
threads (#15323)
ff90f78c700 is described below

commit ff90f78c700c582f9800013faad827c36b45ceb7
Author: Mayank Shekhar Narula <42991652+msn-t...@users.noreply.github.com>
AuthorDate: Thu Feb 15 17:26:47 2024 +

KAFKA-16226; Reduce synchronization between producer threads (#15323)

As this [JIRA](https://issues.apache.org/jira/browse/KAFKA-16226) explains, 
there is increased synchronization between application-thread, and the 
background thread as the background thread started to synchronized methods 
Metadata.currentLeader() in [original 
PR](https://github.com/apache/kafka/pull/14384). So this PR does the following 
changes
1. Changes background thread, i.e. RecordAccumulator's partitionReady(), 
and drainBatchesForOneNode(), to not use `Metadata.currentLeader()`. Instead 
rely on `MetadataCache` that is immutable. So access to it is unsynchronized.
2.  This PR repurposes `MetadataCache` as an immutable snapshot of 
Metadata. This is a wrapper around public `Cluster`. `MetadataCache`'s 
API/functionality should be extended for internal client usage Vs public 
`Cluster`. For example, this PR adds `MetadataCache.leaderEpochFor()`
3. Rename `MetadataCache` to `MetadataSnapshot` to make it explicit its 
immutable.

**Note both `Cluster` and `MetadataCache` are not syncronized, hence reduce 
synchronization from the hot path for high partition counts.**

Reviewers: Jason Gustafson 
---
 .../java/org/apache/kafka/clients/Metadata.java|  65 +--
 .../{MetadataCache.java => MetadataSnapshot.java}  |  77 ++--
 .../clients/producer/internals/ProducerBatch.java  |  13 +-
 .../producer/internals/RecordAccumulator.java  |  78 ++--
 .../kafka/clients/producer/internals/Sender.java   |   6 +-
 .../kafka/common/requests/MetadataResponse.java|   6 +-
 ...ataCacheTest.java => MetadataSnapshotTest.java} |  56 ++-
 .../org/apache/kafka/clients/MetadataTest.java |   4 +-
 .../producer/internals/ProducerBatchTest.java  |  36 +-
 .../producer/internals/RecordAccumulatorTest.java  | 468 +
 .../clients/producer/internals/SenderTest.java |   9 +-
 .../producer/internals/TransactionManagerTest.java |  33 +-
 .../test/java/org/apache/kafka/test/TestUtils.java |  39 ++
 13 files changed, 476 insertions(+), 414 deletions(-)

diff --git a/clients/src/main/java/org/apache/kafka/clients/Metadata.java 
b/clients/src/main/java/org/apache/kafka/clients/Metadata.java
index 607c74eeddb..30cad44a4bc 100644
--- a/clients/src/main/java/org/apache/kafka/clients/Metadata.java
+++ b/clients/src/main/java/org/apache/kafka/clients/Metadata.java
@@ -75,7 +75,7 @@ public class Metadata implements Closeable {
 private KafkaException fatalException;
 private Set invalidTopics;
 private Set unauthorizedTopics;
-private MetadataCache cache = MetadataCache.empty();
+private volatile MetadataSnapshot metadataSnapshot = 
MetadataSnapshot.empty();
 private boolean needFullUpdate;
 private boolean needPartialUpdate;
 private long equivalentResponseCount;
@@ -123,8 +123,15 @@ public class Metadata implements Closeable {
 /**
  * Get the current cluster info without blocking
  */
-public synchronized Cluster fetch() {
-return cache.cluster();
+public Cluster fetch() {
+return metadataSnapshot.cluster();
+}
+
+/**
+ * Get the current metadata cache.
+ */
+public MetadataSnapshot fetchMetadataSnapshot() {
+return metadataSnapshot;
 }
 
 /**
@@ -265,7 +272,7 @@ public class Metadata implements Closeable {
  */
 synchronized Optional 
partitionMetadataIfCurrent(TopicPartition topicPartition) {
 Integer epoch = lastSeenLeaderEpochs.get(topicPartition);
-Optional partitionMetadata = 
cache.partitionMetadata(topicPartition);
+Optional partitionMetadata = 
metadataSnapshot.partitionMetadata(topicPartition);
 if (epoch == null) {
 // old cluster format (no epochs)
 return partitionMetadata;
@@ -278,8 +285,8 @@ public class Metadata implements Closeable {
 /**
  * @return a mapping from topic names to topic IDs for all topics with 
valid IDs in the cache
  */
-public synchronized Map topicIds() {
-return cache.topicIds();
+public Map topicIds() {
+return metadataSnapshot.topicIds();
 }
 
 public synchronized LeaderAndEpoch currentLeader(TopicPartition 
topicPartition) {
@@ -289,14 +296,14 @@ public class Metadata implements Closeable {
 
 MetadataResponse.PartitionMetadata partitionMetadata = 
maybeMetadata.get();
 Optional le

(kafka) branch trunk updated: MINOR: Move Raft io thread implementation to Java (#15119)

2024-01-05 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 599e22b8421 MINOR: Move Raft io thread implementation to Java (#15119)
599e22b8421 is described below

commit 599e22b84214d7ee7e1749fd3059584f32f80132
Author: Jason Gustafson 
AuthorDate: Fri Jan 5 09:27:36 2024 -0800

MINOR: Move Raft io thread implementation to Java (#15119)

This patch moves the `RaftIOThread` implementation into Java. I changed the 
name to `KafkaRaftClientDriver` since the main thing it does is drive the calls 
to `poll()`. There shouldn't be any changes to the logic.

Reviewers: José Armando García Sancio 
---
 build.gradle   |   1 +
 checkstyle/import-control.xml  |   1 +
 core/src/main/scala/kafka/raft/RaftManager.scala   |  61 +--
 .../scala/unit/kafka/raft/RaftManagerTest.scala|  49 +
 .../apache/kafka/raft/KafkaRaftClientDriver.java   | 122 +
 .../kafka/raft/KafkaRaftClientDriverTest.java  |  96 
 6 files changed, 227 insertions(+), 103 deletions(-)

diff --git a/build.gradle b/build.gradle
index f7abbf4f0b2..ca98890bd5a 100644
--- a/build.gradle
+++ b/build.gradle
@@ -1582,6 +1582,7 @@ project(':raft') {
 implementation libs.jacksonDatabind
 
 testImplementation project(':server-common')
+testImplementation project(':server-common').sourceSets.test.output
 testImplementation project(':clients')
 testImplementation project(':clients').sourceSets.test.output
 testImplementation libs.junitJupiter
diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml
index 9b66f143b77..39a77326bde 100644
--- a/checkstyle/import-control.xml
+++ b/checkstyle/import-control.xml
@@ -441,6 +441,7 @@
 
 
 
+
 
 
 
diff --git a/core/src/main/scala/kafka/raft/RaftManager.scala 
b/core/src/main/scala/kafka/raft/RaftManager.scala
index cd8a9739654..d80a0d50137 100644
--- a/core/src/main/scala/kafka/raft/RaftManager.scala
+++ b/core/src/main/scala/kafka/raft/RaftManager.scala
@@ -24,7 +24,6 @@ import java.util.OptionalInt
 import java.util.concurrent.CompletableFuture
 import kafka.log.LogManager
 import kafka.log.UnifiedLog
-import kafka.raft.KafkaRaftManager.RaftIoThread
 import kafka.server.KafkaRaftServer.ControllerRole
 import kafka.server.KafkaConfig
 import kafka.utils.CoreUtils
@@ -42,52 +41,15 @@ import org.apache.kafka.common.security.JaasContext
 import org.apache.kafka.common.security.auth.SecurityProtocol
 import org.apache.kafka.common.utils.{LogContext, Time}
 import org.apache.kafka.raft.RaftConfig.{AddressSpec, InetAddressSpec, 
NON_ROUTABLE_ADDRESS, UnknownAddressSpec}
-import org.apache.kafka.raft.{FileBasedStateStore, KafkaNetworkChannel, 
KafkaRaftClient, LeaderAndEpoch, RaftClient, RaftConfig, RaftRequest, 
ReplicatedLog}
+import org.apache.kafka.raft.{FileBasedStateStore, KafkaNetworkChannel, 
KafkaRaftClient, KafkaRaftClientDriver, LeaderAndEpoch, RaftClient, RaftConfig, 
ReplicatedLog}
 import org.apache.kafka.server.common.serialization.RecordSerde
-import org.apache.kafka.server.util.{KafkaScheduler, ShutdownableThread}
+import org.apache.kafka.server.util.KafkaScheduler
 import org.apache.kafka.server.fault.FaultHandler
 import org.apache.kafka.server.util.timer.SystemTimer
 
 import scala.jdk.CollectionConverters._
 
 object KafkaRaftManager {
-  class RaftIoThread(
-client: KafkaRaftClient[_],
-threadNamePrefix: String,
-fatalFaultHandler: FaultHandler
-  ) extends ShutdownableThread(threadNamePrefix + "-io-thread", false) with 
Logging {
-
-this.logIdent = logPrefix
-
-override def doWork(): Unit = {
-  try {
-client.poll()
-  } catch {
-case t: Throwable =>
-  throw fatalFaultHandler.handleFault("Unexpected error in raft IO 
thread", t)
-  }
-}
-
-override def initiateShutdown(): Boolean = {
-  if (super.initiateShutdown()) {
-client.shutdown(5000).whenComplete { (_, exception) =>
-  if (exception != null) {
-error("Graceful shutdown of RaftClient failed", exception)
-  } else {
-info("Completed graceful shutdown of RaftClient")
-  }
-}
-true
-  } else {
-false
-  }
-}
-
-override def isRunning: Boolean = {
-  client.isRunning && !isThreadFailed
-}
-  }
-
   private def createLogDirectory(logDir: File, logDirName: String): File = {
 val logDirPath = logDir.getAbsolutePath
 val dir = new File(logDirPath, logDirName)
@@ -172,7 +134,7 @@ class KafkaRaftManager[T](
   private val expirationTimer = new SystemTimer("raft-expiration-executor")
   private val expirationService = new 
Timi

(kafka) branch trunk updated: MINOR: Fix flaky `DescribeClusterRequestTest.testDescribeClusterRequestIncludingClusterAuthorizedOperations` (#14890)

2023-12-01 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new a701c0e04fb MINOR: Fix flaky 
`DescribeClusterRequestTest.testDescribeClusterRequestIncludingClusterAuthorizedOperations`
 (#14890)
a701c0e04fb is described below

commit a701c0e04fbe9d570f5a8c3a46d2e6bbe42619a5
Author: Jason Gustafson 
AuthorDate: Fri Dec 1 09:33:17 2023 -0800

MINOR: Fix flaky 
`DescribeClusterRequestTest.testDescribeClusterRequestIncludingClusterAuthorizedOperations`
 (#14890)

Test startup does not assure that all brokers are registered. In flaky 
failures,
the `DescribeCluster` API does not return a complete list of brokers. To fix
the issue, we add a call to `ensureConsistentKRaftMetadata()` to ensure 
that all
brokers are registered and have caught up to current metadata.

Reviewers: David Jacot 
---
 core/src/test/scala/unit/kafka/server/DescribeClusterRequestTest.scala | 2 ++
 1 file changed, 2 insertions(+)

diff --git 
a/core/src/test/scala/unit/kafka/server/DescribeClusterRequestTest.scala 
b/core/src/test/scala/unit/kafka/server/DescribeClusterRequestTest.scala
index 7c260dae853..acb0a215b19 100644
--- a/core/src/test/scala/unit/kafka/server/DescribeClusterRequestTest.scala
+++ b/core/src/test/scala/unit/kafka/server/DescribeClusterRequestTest.scala
@@ -83,6 +83,8 @@ class DescribeClusterRequestTest extends BaseRequestTest {
   Int.MinValue
 }
 
+ensureConsistentKRaftMetadata()
+
 for (version <- ApiKeys.DESCRIBE_CLUSTER.oldestVersion to 
ApiKeys.DESCRIBE_CLUSTER.latestVersion) {
   val describeClusterRequest = new DescribeClusterRequest.Builder(new 
DescribeClusterRequestData()
 
.setIncludeClusterAuthorizedOperations(includeClusterAuthorizedOperations))



(kafka) branch trunk updated: MINOR: Fix flaky `MetadataLoaderTest.testNoPublishEmptyImage` (#14875)

2023-11-30 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new a35e021925d MINOR: Fix flaky 
`MetadataLoaderTest.testNoPublishEmptyImage` (#14875)
a35e021925d is described below

commit a35e021925d2ac371497a823a5ae81271193a3fd
Author: Jason Gustafson 
AuthorDate: Thu Nov 30 09:50:19 2023 -0800

MINOR: Fix flaky `MetadataLoaderTest.testNoPublishEmptyImage` (#14875)

There is a race in the assertion on `capturedImages`. Since the future is 
signaled first, it is still possible to see an empty list. By adding to the 
collection first, we can ensure the assertion will succeed.

Reviewers: Reviewers: David Jacot 
---
 .../src/test/java/org/apache/kafka/image/loader/MetadataLoaderTest.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git 
a/metadata/src/test/java/org/apache/kafka/image/loader/MetadataLoaderTest.java 
b/metadata/src/test/java/org/apache/kafka/image/loader/MetadataLoaderTest.java
index 62d974b8b5c..921c241a09a 100644
--- 
a/metadata/src/test/java/org/apache/kafka/image/loader/MetadataLoaderTest.java
+++ 
b/metadata/src/test/java/org/apache/kafka/image/loader/MetadataLoaderTest.java
@@ -793,10 +793,10 @@ public class MetadataLoaderTest {
 
 @Override
 public void onMetadataUpdate(MetadataDelta delta, MetadataImage 
newImage, LoaderManifest manifest) {
+capturedImages.add(newImage);
 if (!firstPublish.isDone()) {
 firstPublish.complete(null);
 }
-capturedImages.add(newImage);
 }
 };
 



(kafka) branch trunk updated: MINOR: No need for response callback when applying controller mutation throttle (#14861)

2023-11-29 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 085f1d340b4 MINOR: No need for response callback when applying 
controller mutation throttle (#14861)
085f1d340b4 is described below

commit 085f1d340b4a4ffca21c94966413c24c11dc3b3a
Author: Jason Gustafson 
AuthorDate: Wed Nov 29 16:33:05 2023 -0800

MINOR: No need for response callback when applying controller mutation 
throttle (#14861)

With `AbstractResponse.maybeSetThrottleTimeMs`, we don't need to use a 
callback to build the response with the respective throttle.

Reviewers: David Jacot 
---
 .../main/scala/kafka/server/ControllerApis.scala   | 45 -
 core/src/main/scala/kafka/server/KafkaApis.scala   | 56 +-
 .../scala/kafka/server/RequestHandlerHelper.scala  | 11 +++--
 3 files changed, 48 insertions(+), 64 deletions(-)

diff --git a/core/src/main/scala/kafka/server/ControllerApis.scala 
b/core/src/main/scala/kafka/server/ControllerApis.scala
index 7278bf417ad..aef1924b89a 100644
--- a/core/src/main/scala/kafka/server/ControllerApis.scala
+++ b/core/src/main/scala/kafka/server/ControllerApis.scala
@@ -205,16 +205,14 @@ class ControllerApis(
   names => authHelper.filterByAuthorized(request.context, DESCRIBE, TOPIC, 
names)(n => n),
   names => authHelper.filterByAuthorized(request.context, DELETE, TOPIC, 
names)(n => n))
 future.handle[Unit] { (results, exception) =>
-  
requestHelper.sendResponseMaybeThrottleWithControllerQuota(controllerMutationQuota,
 request, throttleTimeMs => {
-if (exception != null) {
-  deleteTopicsRequest.getErrorResponse(throttleTimeMs, exception)
-} else {
-  val responseData = new DeleteTopicsResponseData().
-setResponses(new DeletableTopicResultCollection(results.iterator)).
-setThrottleTimeMs(throttleTimeMs)
-  new DeleteTopicsResponse(responseData)
-}
-  })
+  val response = if (exception != null) {
+deleteTopicsRequest.getErrorResponse(exception)
+  } else {
+val responseData = new DeleteTopicsResponseData()
+  .setResponses(new DeletableTopicResultCollection(results.iterator))
+new DeleteTopicsResponse(responseData)
+  }
+  
requestHelper.sendResponseMaybeThrottleWithControllerQuota(controllerMutationQuota,
 request, response)
 }
   }
 
@@ -371,14 +369,12 @@ class ControllerApis(
 names => authHelper.filterByAuthorized(request.context, 
DESCRIBE_CONFIGS, TOPIC,
 names, logIfDenied = false)(identity))
 future.handle[Unit] { (result, exception) =>
-  
requestHelper.sendResponseMaybeThrottleWithControllerQuota(controllerMutationQuota,
 request, throttleTimeMs => {
-if (exception != null) {
-  createTopicsRequest.getErrorResponse(throttleTimeMs, exception)
-} else {
-  result.setThrottleTimeMs(throttleTimeMs)
-  new CreateTopicsResponse(result)
-}
-  })
+  val response = if (exception != null) {
+createTopicsRequest.getErrorResponse(exception)
+  } else {
+new CreateTopicsResponse(result)
+  }
+  
requestHelper.sendResponseMaybeThrottleWithControllerQuota(controllerMutationQuota,
 request, response)
 }
   }
 
@@ -802,16 +798,13 @@ class ControllerApis(
   createPartitionsRequest.data(),
   filterAlterAuthorizedTopics)
 future.handle[Unit] { (responses, exception) =>
-  if (exception != null) {
-requestHelper.handleError(request, exception)
+  val response = if (exception != null) {
+createPartitionsRequest.getErrorResponse(exception)
   } else {
-
requestHelper.sendResponseMaybeThrottleWithControllerQuota(controllerMutationQuota,
 request, requestThrottleMs => {
-  val responseData = new CreatePartitionsResponseData().
-setResults(responses).
-setThrottleTimeMs(requestThrottleMs)
-  new CreatePartitionsResponse(responseData)
-})
+val responseData = new 
CreatePartitionsResponseData().setResults(responses)
+new CreatePartitionsResponse(responseData)
   }
+  
requestHelper.sendResponseMaybeThrottleWithControllerQuota(controllerMutationQuota,
 request, response)
 }
   }
 
diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala 
b/core/src/main/scala/kafka/server/KafkaApis.scala
index 2b27d4647ac..cb912769f17 100644
--- a/core/src/main/scala/kafka/server/KafkaApis.scala
+++ b/core/src/main/scala/kafka/server/KafkaApis.scala
@@ -1941,16 +1941,12 @@ class KafkaApis(val requestChannel: RequestChannel,
 val controllerMutationQuota = 
quotas.controllerMutation.newQuotaFor(request, strictSinceVersion = 6)
 
 def sendResponseCallback(results: Crea

(kafka) branch trunk updated (4eb8ae68ca1 -> e905ef1edfb)

2023-11-20 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a change to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


from 4eb8ae68ca1 KAFKA-15836: KafkaConsumer subscribes to multiple topics 
does not respect max.poll.records (#14789)
 add e905ef1edfb MINOR: Small LogValidator clean ups (#14697)

No new revisions were added by this update.

Summary of changes:
 .../kafka/common/record/ConvertedRecords.java  | 10 +--
 .../apache/kafka/common/record/FileRecords.java|  2 +-
 .../record/LazyDownConversionRecordsSend.java  | 10 +--
 .../kafka/common/record/MultiRecordsSend.java  |  4 +-
 ...ersionStats.java => RecordValidationStats.java} | 17 +++--
 .../apache/kafka/common/record/RecordsUtil.java|  2 +-
 .../common/record/MemoryRecordsBuilderTest.java|  2 +-
 core/src/main/scala/kafka/log/UnifiedLog.scala |  4 +-
 core/src/main/scala/kafka/server/KafkaApis.scala   |  6 +-
 .../main/scala/kafka/server/ReplicaManager.scala   | 12 ++--
 .../AbstractCoordinatorConcurrencyTest.scala   |  4 +-
 .../scala/unit/kafka/log/LogValidatorTest.scala| 64 +++---
 .../unit/kafka/server/MockFetcherThread.scala  |  2 +-
 .../kafka/server/ReplicaFetcherThreadTest.scala|  6 +-
 .../kafka/storage/internals/log/LogAppendInfo.java | 34 +-
 .../kafka/storage/internals/log/LogValidator.java  | 77 +++---
 16 files changed, 140 insertions(+), 116 deletions(-)
 rename 
clients/src/main/java/org/apache/kafka/common/record/{RecordConversionStats.java
 => RecordValidationStats.java} (78%)



[kafka] branch trunk updated (cc66d1feee6 -> d46781d4db3)

2023-10-11 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a change to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


from cc66d1feee6 MINOR: Add javadoc to all ConfigDef.Types values (#14515)
 add d46781d4db3 KAFKA-15221; Fix the race between fetch requests from a 
rebooted follower. (#14053)

No new revisions were added by this update.

Summary of changes:
 core/src/main/scala/kafka/cluster/Partition.scala  | 25 +++---
 core/src/main/scala/kafka/cluster/Replica.scala| 22 +-
 .../scala/unit/kafka/cluster/PartitionTest.scala   | 88 +++---
 .../scala/unit/kafka/cluster/ReplicaTest.scala | 59 ++-
 .../unit/kafka/server/IsrExpirationTest.scala  | 12 +--
 5 files changed, 175 insertions(+), 31 deletions(-)



[kafka] branch trunk updated: KAFKA-14694: RPCProducerIdManager should not wait on new block (#13267)

2023-06-22 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 1dbcb7da9e3 KAFKA-14694: RPCProducerIdManager should not wait on new 
block (#13267)
1dbcb7da9e3 is described below

commit 1dbcb7da9e3625ec2078a82f84542a3127730fef
Author: Jeff Kim 
AuthorDate: Thu Jun 22 13:19:39 2023 -0400

KAFKA-14694: RPCProducerIdManager should not wait on new block (#13267)

RPCProducerIdManager initiates an async request to the controller to grab a 
block of producer IDs and then blocks waiting for a response from the 
controller.

This is done in the request handler threads while holding a global lock. 
This means that if many producers are requesting producer IDs and the 
controller is slow to respond, many threads can get stuck waiting for the lock.

This patch aims to:
* resolve the deadlock scenario mentioned above by not waiting for a new 
block and returning an error immediately
* remove synchronization usages in RpcProducerIdManager.generateProducerId()
* handle errors returned from generateProducerId() so that KafkaApis does 
not log unexpected errors
* confirm producers backoff before retrying
* introduce backoff if manager fails to process AllocateProducerIdsResponse

Reviewers: Artem Livshits , Jason Gustafson 

---
 .../clients/producer/internals/SenderTest.java |  24 ++-
 .../transaction/ProducerIdManager.scala| 167 +++
 .../transaction/TransactionCoordinator.scala   |  47 --
 .../src/main/scala/kafka/server/BrokerServer.scala |   4 +-
 core/src/main/scala/kafka/server/KafkaServer.scala |   4 +-
 .../transaction/ProducerIdsIntegrationTest.scala   |  66 ++--
 .../transaction/ProducerIdManagerTest.scala| 180 -
 .../TransactionCoordinatorConcurrencyTest.scala|   7 +-
 .../transaction/TransactionCoordinatorTest.scala   |  11 +-
 .../AddPartitionsToTxnRequestServerTest.scala  |   9 +-
 .../kafka/server/common/ProducerIdsBlock.java  |  16 ++
 .../kafka/server/common/ProducerIdsBlockTest.java  |  25 +++
 12 files changed, 414 insertions(+), 146 deletions(-)

diff --git 
a/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java
 
b/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java
index f6c91659356..b80817465a5 100644
--- 
a/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java
+++ 
b/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java
@@ -3035,6 +3035,28 @@ public class SenderTest {
 verifyErrorMessage(produceResponse(tp0, 0L, Errors.INVALID_REQUEST, 0, 
-1, errorMessage), errorMessage);
 }
 
+@Test
+public void testSenderShouldRetryWithBackoffOnRetriableError() {
+final long producerId = 343434L;
+TransactionManager transactionManager = createTransactionManager();
+setupWithTransactionState(transactionManager);
+long start = time.milliseconds();
+
+// first request is sent immediately
+prepareAndReceiveInitProducerId(producerId, (short) -1, 
Errors.COORDINATOR_LOAD_IN_PROGRESS);
+long request1 = time.milliseconds();
+assertEquals(start, request1);
+
+// backoff before sending second request
+prepareAndReceiveInitProducerId(producerId, (short) -1, 
Errors.COORDINATOR_LOAD_IN_PROGRESS);
+long request2 = time.milliseconds();
+assertEquals(RETRY_BACKOFF_MS, request2 - request1);
+
+// third request should also backoff
+prepareAndReceiveInitProducerId(producerId, Errors.NONE);
+assertEquals(RETRY_BACKOFF_MS, time.milliseconds() - request2);
+}
+
 private void verifyErrorMessage(ProduceResponse response, String 
expectedMessage) throws Exception {
 Future future = appendToAccumulator(tp0, 0L, "key", 
"value");
 sender.runOnce(); // connect
@@ -3191,7 +3213,7 @@ public class SenderTest {
 }
 
 private TransactionManager createTransactionManager() {
-return new TransactionManager(new LogContext(), null, 0, 100L, new 
ApiVersions());
+return new TransactionManager(new LogContext(), null, 0, 
RETRY_BACKOFF_MS, new ApiVersions());
 }
 
 private void setupWithTransactionState(TransactionManager 
transactionManager) {
diff --git 
a/core/src/main/scala/kafka/coordinator/transaction/ProducerIdManager.scala 
b/core/src/main/scala/kafka/coordinator/transaction/ProducerIdManager.scala
index f16785a7b6c..1e2b6ffac5a 100644
--- a/core/src/main/scala/kafka/coordinator/transaction/ProducerIdManager.scala
+++ b/core/src/main/scala/kafka/coordinator/transaction/ProducerIdManager.scala
@@ -16,6 +16,7 @@
  */
 package kafka.coordinator.transaction
 
+import kafka.coordinator.transacti

[kafka] branch trunk updated (c39123d83d9 -> c9a42c85e2c)

2023-02-21 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a change to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


from c39123d83d9 KAKFA-14733: Added a few missing checks for Kraft 
Authorizer and updated AclAuthorizerTest to run tests for both zk and kraft 
(#13282)
 add c9a42c85e2c KAFKA-14675: Extract metadata-related tasks from Fetcher 
into MetadataFetcher 1/4 (#13192)

No new revisions were added by this update.

Summary of changes:
 .../kafka/clients/consumer/KafkaConsumer.java  |   50 +-
 .../kafka/clients/consumer/internals/Fetcher.java  |  752 +-
 .../clients/consumer/internals/OffsetFetcher.java  |  717 ++
 .../consumer/internals/SubscriptionState.java  |2 +-
 .../consumer/internals/TopicMetadataFetcher.java   |  160 ++
 .../kafka/clients/consumer/KafkaConsumerTest.java  |   14 +-
 .../clients/consumer/internals/FetcherTest.java| 2544 
 .../consumer/internals/OffsetFetcherTest.java  | 1736 +
 .../internals/TopicMetadataFetcherTest.java|  259 ++
 9 files changed, 3363 insertions(+), 2871 deletions(-)
 create mode 100644 
clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetFetcher.java
 create mode 100644 
clients/src/main/java/org/apache/kafka/clients/consumer/internals/TopicMetadataFetcher.java
 create mode 100644 
clients/src/test/java/org/apache/kafka/clients/consumer/internals/OffsetFetcherTest.java
 create mode 100644 
clients/src/test/java/org/apache/kafka/clients/consumer/internals/TopicMetadataFetcherTest.java



[kafka] branch trunk updated: KAFKA-14664; Fix inaccurate raft idle ratio metric (#13207)

2023-02-15 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 35142d43e6a KAFKA-14664; Fix inaccurate raft idle ratio metric (#13207)
35142d43e6a is described below

commit 35142d43e6aa55bd11484224b8b09a83800d3a22
Author: Jason Gustafson 
AuthorDate: Wed Feb 15 14:40:00 2023 -0800

KAFKA-14664; Fix inaccurate raft idle ratio metric (#13207)

The raft idle ratio is currently computed as the average of all recorded 
poll durations. This tends to underestimate the actual idle ratio since it 
treats all measurements equally regardless how much time was spent. For 
example, say we poll twice with the following durations:

Poll 1: 2s
Poll 2: 0s

Assume that the busy time is negligible, so 2s passes overall.

In the first measurement, 2s is spent waiting, so we compute and record a 
ratio of 1.0. In the second measurement, no time passes, and we record 0.0. The 
idle ratio is then computed as the average of these two values (1.0 + 0.0 / 2 = 
0.5), which suggests that the process was busy for 1s, which overestimates the 
true busy time.

In this patch, we create a new `TimeRatio` class which tracks the total 
duration of a periodic event over a full interval of time measurement.

Reviewers: José Armando García Sancio 
---
 .../org/apache/kafka/raft/KafkaRaftClient.java | 21 +++---
 .../kafka/raft/internals/KafkaRaftMetrics.java | 32 -
 .../org/apache/kafka/raft/internals/TimeRatio.java | 82 ++
 .../kafka/raft/internals/KafkaRaftMetricsTest.java | 60 ++--
 .../apache/kafka/raft/internals/TimeRatioTest.java | 67 ++
 5 files changed, 231 insertions(+), 31 deletions(-)

diff --git a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java 
b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java
index f21c31eeab9..737b6326171 100644
--- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java
+++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java
@@ -2224,27 +2224,28 @@ public class KafkaRaftClient implements 
RaftClient {
  * requests and send any needed outbound requests.
  */
 public void poll() {
-pollListeners();
-
-long currentTimeMs = time.milliseconds();
-if (maybeCompleteShutdown(currentTimeMs)) {
+long startPollTimeMs = time.milliseconds();
+if (maybeCompleteShutdown(startPollTimeMs)) {
 return;
 }
 
-long pollStateTimeoutMs = pollCurrentState(currentTimeMs);
-long cleaningTimeoutMs = snapshotCleaner.maybeClean(currentTimeMs);
+long pollStateTimeoutMs = pollCurrentState(startPollTimeMs);
+long cleaningTimeoutMs = snapshotCleaner.maybeClean(startPollTimeMs);
 long pollTimeoutMs = Math.min(pollStateTimeoutMs, cleaningTimeoutMs);
 
-kafkaRaftMetrics.updatePollStart(currentTimeMs);
+long startWaitTimeMs = time.milliseconds();
+kafkaRaftMetrics.updatePollStart(startWaitTimeMs);
 
 RaftMessage message = messageQueue.poll(pollTimeoutMs);
 
-currentTimeMs = time.milliseconds();
-kafkaRaftMetrics.updatePollEnd(currentTimeMs);
+long endWaitTimeMs = time.milliseconds();
+kafkaRaftMetrics.updatePollEnd(endWaitTimeMs);
 
 if (message != null) {
-handleInboundMessage(message, currentTimeMs);
+handleInboundMessage(message, endWaitTimeMs);
 }
+
+pollListeners();
 }
 
 @Override
diff --git 
a/raft/src/main/java/org/apache/kafka/raft/internals/KafkaRaftMetrics.java 
b/raft/src/main/java/org/apache/kafka/raft/internals/KafkaRaftMetrics.java
index b6e03381106..6fb2ce1c8b7 100644
--- a/raft/src/main/java/org/apache/kafka/raft/internals/KafkaRaftMetrics.java
+++ b/raft/src/main/java/org/apache/kafka/raft/internals/KafkaRaftMetrics.java
@@ -39,7 +39,6 @@ public class KafkaRaftMetrics implements AutoCloseable {
 private volatile int numUnknownVoterConnections;
 private volatile OptionalLong electionStartMs;
 private volatile OptionalLong pollStartMs;
-private volatile OptionalLong pollEndMs;
 
 private final MetricName currentLeaderIdMetricName;
 private final MetricName currentVotedIdMetricName;
@@ -53,14 +52,13 @@ public class KafkaRaftMetrics implements AutoCloseable {
 private final Sensor electionTimeSensor;
 private final Sensor fetchRecordsSensor;
 private final Sensor appendRecordsSensor;
-private final Sensor pollIdleSensor;
+private final Sensor pollDurationSensor;
 
 public KafkaRaftMetrics(Metrics metrics, String metricGrpPrefix, 
QuorumState state) {
 this.metrics = metrics;
 String metricGroupName = metricGrpPrefix + "-metrics";
 
 this.pollStartMs = Optional

[kafka] branch trunk updated: MINOR: Better logging to distinguish clean vs unclean loading times (#13242)

2023-02-14 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 9584b48a2a8 MINOR: Better logging to distinguish clean vs unclean 
loading times (#13242)
9584b48a2a8 is described below

commit 9584b48a2a873c692d5054dd06b66232dc25e080
Author: Jason Gustafson 
AuthorDate: Tue Feb 14 09:15:05 2023 -0800

MINOR: Better logging to distinguish clean vs unclean loading times (#13242)

Current log loading logging makes it difficult to analyze the behavior in 
the case of clean and unclean shutdown. The log message looks the same either 
way. Additionally, the logging about unclean recovery also catches the case 
when a new broker is initializing from an empty log dir. This patch adds some 
additional information to existing log messages to make it easier to 
distinguish these cases.

Reviewers: David Jacot 
---
 core/src/main/scala/kafka/log/LogManager.scala | 20 +++-
 1 file changed, 15 insertions(+), 5 deletions(-)

diff --git a/core/src/main/scala/kafka/log/LogManager.scala 
b/core/src/main/scala/kafka/log/LogManager.scala
index 6b2704d0fc1..d2b9ec56f08 100755
--- a/core/src/main/scala/kafka/log/LogManager.scala
+++ b/core/src/main/scala/kafka/log/LogManager.scala
@@ -354,6 +354,7 @@ class LogManager(logDirs: Seq[File],
   error(s"Error while loading log dir $logDirAbsolutePath", e)
 }
 
+val uncleanLogDirs = mutable.Buffer.empty[String]
 for (dir <- liveLogDirs) {
   val logDirAbsolutePath = dir.getAbsolutePath
   var hadCleanShutdown: Boolean = false
@@ -364,14 +365,10 @@ class LogManager(logDirs: Seq[File],
 
 val cleanShutdownFile = new File(dir, LogLoader.CleanShutdownFile)
 if (cleanShutdownFile.exists) {
-  info(s"Skipping recovery for all logs in $logDirAbsolutePath since 
clean shutdown file was found")
   // Cache the clean shutdown status and use that for rest of log 
loading workflow. Delete the CleanShutdownFile
   // so that if broker crashes while loading the log, it is considered 
hard shutdown during the next boot up. KAFKA-10471
   Files.deleteIfExists(cleanShutdownFile.toPath)
   hadCleanShutdown = true
-} else {
-  // log recovery itself is being performed by `Log` class during 
initialization
-  info(s"Attempting recovery for all logs in $logDirAbsolutePath since 
no clean shutdown file was found")
 }
 
 var recoveryPoints = Map[TopicPartition, Long]()
@@ -401,6 +398,17 @@ class LogManager(logDirs: Seq[File],
 numTotalLogs += logsToLoad.length
 numRemainingLogs.put(dir.getAbsolutePath, logsToLoad.length)
 
+if (logsToLoad.isEmpty) {
+  info(s"No logs found to be loaded in $logDirAbsolutePath")
+} else if (hadCleanShutdown) {
+  info(s"Skipping recovery of ${logsToLoad.length} logs from 
$logDirAbsolutePath since " +
+"clean shutdown file was found")
+} else {
+  info(s"Recovering ${logsToLoad.length} logs from $logDirAbsolutePath 
since no " +
+"clean shutdown file was found")
+  uncleanLogDirs.append(logDirAbsolutePath)
+}
+
 val jobsForDir = logsToLoad.map { logDir =>
   val runnable: Runnable = () => {
 debug(s"Loading log $logDir")
@@ -454,7 +462,9 @@ class LogManager(logDirs: Seq[File],
   threadPools.foreach(_.shutdown())
 }
 
-info(s"Loaded $numTotalLogs logs in ${time.hiResClockMs() - startMs}ms.")
+val elapsedMs = time.hiResClockMs() - startMs
+val printedUncleanLogDirs = if (uncleanLogDirs.isEmpty) "" else s" 
(unclean log dirs = $uncleanLogDirs)"
+info(s"Loaded $numTotalLogs logs in ${elapsedMs}ms$printedUncleanLogDirs")
   }
 
   private[log] def addLogRecoveryMetrics(numRemainingLogs: 
ConcurrentMap[String, Int],



[kafka] branch trunk updated: KAFKA-6793: Unused configuration logging appears to be noisy and unnecessary (#13225)

2023-02-13 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new eee2bf9686d KAFKA-6793: Unused configuration logging appears to be 
noisy and unnecessary  (#13225)
eee2bf9686d is described below

commit eee2bf9686db85514c474732874d14456ae96ebc
Author: Philip Nee 
AuthorDate: Mon Feb 13 09:27:55 2023 -0800

KAFKA-6793: Unused configuration logging appears to be noisy and 
unnecessary  (#13225)

Warnings about unused configs are most often spurious. This patch changes 
the current warning to an info message.

Reviewers: Chris Egerton , Jason Gustafson 

---
 .../main/java/org/apache/kafka/common/config/AbstractConfig.java| 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)

diff --git 
a/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java 
b/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java
index e620f18f7d8..faf174034a8 100644
--- a/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java
+++ b/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java
@@ -18,11 +18,11 @@ package org.apache.kafka.common.config;
 
 import org.apache.kafka.common.Configurable;
 import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.config.provider.ConfigProvider;
 import org.apache.kafka.common.config.types.Password;
 import org.apache.kafka.common.utils.Utils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.apache.kafka.common.config.provider.ConfigProvider;
 
 import java.util.ArrayList;
 import java.util.Collections;
@@ -370,12 +370,12 @@ public class AbstractConfig {
 }
 
 /**
- * Log warnings for any unused configurations
+ * Info level log for any unused configurations
  */
 public void logUnused() {
 Set unusedkeys = unused();
 if (!unusedkeys.isEmpty()) {
-log.warn("These configurations '{}' were supplied but are not used 
yet.", unusedkeys);
+log.info("These configurations '{}' were supplied but are not used 
yet.", unusedkeys);
 }
 }
 



[kafka] branch 3.4 updated: KAFKA-13972; Ensure replica state deleted after reassignment cancellation (#13107)

2023-02-06 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch 3.4
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.4 by this push:
 new 4064f4a7e43 KAFKA-13972; Ensure replica state deleted after 
reassignment cancellation (#13107)
4064f4a7e43 is described below

commit 4064f4a7e434b7096ee506d17413fc37258384d0
Author: Jason Gustafson 
AuthorDate: Wed Jan 18 10:26:48 2023 -0800

KAFKA-13972; Ensure replica state deleted after reassignment cancellation 
(#13107)

When a reassignment is cancelled, we need to delete the partition state of 
adding replicas. Failing to do so causes "stray" replicas which take up disk 
space and can cause topicId conflicts if the topic is later recreated. 
Currently, this logic does not work because the leader epoch does not always 
get bumped after cancellation. Without a leader epoch bump, the replica will 
ignore StopReplica requests sent by the controller and the replica may remain 
online.

In this patch, we fix the problem by loosening the epoch check on the 
broker side when a StopReplica request is received. Instead of ignoring the 
request when the request epoch matches the current epoch, the request will be 
accepted.

Note, this problem only affects the ZK controller. The integration tests 
added here nevertheless cover both metadata modes.

Reviewers:  David Jacot , Justine Olshan 

---
 .../main/scala/kafka/server/ReplicaManager.scala   |  2 +-
 .../admin/ReassignPartitionsIntegrationTest.scala  | 95 ++
 .../unit/kafka/server/ReplicaManagerTest.scala |  4 +-
 .../test/scala/unit/kafka/utils/TestUtils.scala| 21 +++--
 4 files changed, 94 insertions(+), 28 deletions(-)

diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala 
b/core/src/main/scala/kafka/server/ReplicaManager.scala
index b2a37479bae..069ef27d67b 100644
--- a/core/src/main/scala/kafka/server/ReplicaManager.scala
+++ b/core/src/main/scala/kafka/server/ReplicaManager.scala
@@ -387,7 +387,7 @@ class ReplicaManager(val config: KafkaConfig,
   // epoch, a sentinel value (NoEpoch) is used and bypass the 
epoch validation.
   if (requestLeaderEpoch == LeaderAndIsr.EpochDuringDelete ||
   requestLeaderEpoch == LeaderAndIsr.NoEpoch ||
-  requestLeaderEpoch > currentLeaderEpoch) {
+  requestLeaderEpoch >= currentLeaderEpoch) {
 stoppedPartitions += topicPartition -> deletePartition
 // Assume that everything will go right. It is overwritten in 
case of an error.
 responseMap.put(topicPartition, Errors.NONE)
diff --git 
a/core/src/test/scala/integration/kafka/admin/ReassignPartitionsIntegrationTest.scala
 
b/core/src/test/scala/integration/kafka/admin/ReassignPartitionsIntegrationTest.scala
index 9b3b935f23e..d3a04da4d13 100644
--- 
a/core/src/test/scala/integration/kafka/admin/ReassignPartitionsIntegrationTest.scala
+++ 
b/core/src/test/scala/integration/kafka/admin/ReassignPartitionsIntegrationTest.scala
@@ -108,12 +108,13 @@ class ReassignPartitionsIntegrationTest extends 
QuorumTestHarness {
   
"""{"topic":"bar","partition":0,"replicas":[3,2,0],"log_dirs":["any","any","any"]}"""
 +
   """]}"""
 
+val foo0 = new TopicPartition("foo", 0)
+val bar0 = new TopicPartition("bar", 0)
+
 // Check that the assignment has not yet been started yet.
 val initialAssignment = Map(
-  new TopicPartition("foo", 0) ->
-PartitionReassignmentState(Seq(0, 1, 2), Seq(0, 1, 3), true),
-  new TopicPartition("bar", 0) ->
-PartitionReassignmentState(Seq(3, 2, 1), Seq(3, 2, 0), true)
+  foo0 -> PartitionReassignmentState(Seq(0, 1, 2), Seq(0, 1, 3), true),
+  bar0 -> PartitionReassignmentState(Seq(3, 2, 1), Seq(3, 2, 0), true)
 )
 waitForVerifyAssignment(cluster.adminClient, assignment, false,
   VerifyAssignmentResult(initialAssignment))
@@ -122,10 +123,8 @@ class ReassignPartitionsIntegrationTest extends 
QuorumTestHarness {
 runExecuteAssignment(cluster.adminClient, false, assignment, -1L, -1L)
 assertEquals(unthrottledBrokerConfigs, 
describeBrokerLevelThrottles(unthrottledBrokerConfigs.keySet.toSeq))
 val finalAssignment = Map(
-  new TopicPartition("foo", 0) ->
-PartitionReassignmentState(Seq(0, 1, 3), Seq(0, 1, 3), true),
-  new TopicPartition("bar", 0) ->
-PartitionReassignmentState(Seq(3, 2, 0), Seq(3, 2, 0), true)
+  foo0 -> PartitionReassignmentState(Seq(0, 1, 3), Seq(0, 1, 3), true),
+  bar0 -> PartitionReassignmentState(Seq(3, 2, 0), Seq(3, 2, 0), t

[kafka] branch trunk updated (0711375b1ef -> 598ba5f2fd5)

2023-01-25 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a change to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


from 0711375b1ef KRaft brokers and controllesr should fail on Authorizer 
failure (#13151)
 add 598ba5f2fd5 KAFKA-14644: Process should crash after failure in Raft IO 
thread (#13140)

No new revisions were added by this update.

Summary of changes:
 core/src/main/scala/kafka/raft/RaftManager.scala| 16 
 core/src/main/scala/kafka/server/KafkaServer.scala  |  4 +++-
 core/src/main/scala/kafka/server/SharedServer.scala | 10 +-
 core/src/main/scala/kafka/tools/TestRaftServer.scala|  3 ++-
 .../test/scala/unit/kafka/raft/RaftManagerTest.scala| 17 +
 5 files changed, 39 insertions(+), 11 deletions(-)



[kafka] branch trunk updated (cfbd8979bc7 -> b2cb546fba0)

2023-01-23 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a change to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


from cfbd8979bc7 KAFKA-14311: Cleanly cleanly stop connectors/tasks on 
Connect worker shutdown (#12802)
 add b2cb546fba0 MINOR: Small cleanups in refactored consumer 
implementation (#13138)

No new revisions were added by this update.

Summary of changes:
 .../internals/CoordinatorRequestManager.java   | 103 +
 .../internals/DefaultBackgroundThread.java |  10 +-
 .../consumer/internals/ErrorEventHandler.java  |   2 +-
 .../consumer/internals/NetworkClientDelegate.java  | 109 +
 .../clients/consumer/internals/RequestState.java   |  64 --
 .../internals/events/ErrorBackgroundEvent.java |   4 +-
 .../java/org/apache/kafka/clients/MockClient.java  |  17 ++
 .../internals/CoordinatorRequestManagerTest.java   | 244 ++---
 .../internals/DefaultBackgroundThreadTest.java |   9 +-
 .../internals/NetworkClientDelegateTest.java   | 173 ++-
 .../consumer/internals/RequestStateTest.java   |   6 +-
 11 files changed, 365 insertions(+), 376 deletions(-)



[kafka] branch 3.4 updated: KAFKA-14612: Make sure to write a new topics ConfigRecords to metadata log iff the topic is created (#13104)

2023-01-12 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch 3.4
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.4 by this push:
 new 4d2a60584e4 KAFKA-14612: Make sure to write a new topics ConfigRecords 
to metadata log iff the topic is created (#13104)
4d2a60584e4 is described below

commit 4d2a60584e4523c6834596503302cb88d0737d02
Author: andymg3 <97484553+andy...@users.noreply.github.com>
AuthorDate: Thu Jan 12 19:23:57 2023 -0500

KAFKA-14612: Make sure to write a new topics ConfigRecords to metadata log 
iff the topic is created (#13104)

### JIRA
https://issues.apache.org/jira/browse/KAFKA-14612

### Details
Makes sure we emit `ConfigRecord`s for a topic iff it actually gets 
created. Currently, we might emit `ConfigRecord`s even if the topic creation 
fails later in the `createTopics` method.

I created a new method `incrementalAlterConfig` in 
`ConfigurationControlManager` that is similar to `incrementalAlterConfig` but 
it just handles one config at a time. This is used in 
`ReplicationControlManager` for each topic. By handling one topic's config at a 
time, it's easier to isolate each topic's config records. This enables us to 
make sure we only write config records for topics that get created.

I refactored `incrementalAlterConfigResource` to return an `ApiError`. This 
made it easier to implement the new method `incrementalAlterConfig` in 
`ConfigurationControlManager` because it then doesnt have to search in the 
`Map` for the result.

### Testing
Enhanced pre-existing test 
`ReplicationControlManagerTest.testCreateTopicsWithConfigs`. I ran the tests 
without the changes to `ReplicationControlManager` and made sure each assertion 
ends up failing. Also ran `./gradlew metadata:test --tests 
org.apache.kafka.controller.ReplicationControlManagerTest`.

Reviewers: Jason Gustafson 
---
 .../controller/ConfigurationControlManager.java| 45 +++---
 .../controller/ReplicationControlManager.java  | 31 
 .../ConfigurationControlManagerTest.java   | 25 ++
 .../controller/ReplicationControlManagerTest.java  | 55 ++
 4 files changed, 130 insertions(+), 26 deletions(-)

diff --git 
a/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java
 
b/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java
index b5d71230cf4..2cbf51c9baf 100644
--- 
a/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java
+++ 
b/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java
@@ -166,26 +166,41 @@ public class ConfigurationControlManager {
  * @return  The result.
  */
 ControllerResult> incrementalAlterConfigs(
-Map>> 
configChanges,
-boolean newlyCreatedResource) {
+Map>> configChanges,
+boolean newlyCreatedResource
+) {
 List outputRecords = new ArrayList<>();
 Map outputResults = new HashMap<>();
 for (Entry>> 
resourceEntry :
 configChanges.entrySet()) {
-incrementalAlterConfigResource(resourceEntry.getKey(),
+ApiError apiError = 
incrementalAlterConfigResource(resourceEntry.getKey(),
 resourceEntry.getValue(),
 newlyCreatedResource,
-outputRecords,
-outputResults);
+outputRecords);
+outputResults.put(resourceEntry.getKey(), apiError);
 }
 return ControllerResult.atomicOf(outputRecords, outputResults);
 }
 
-private void incrementalAlterConfigResource(ConfigResource configResource,
-Map> keysToOps,
-boolean newlyCreatedResource,
-List 
outputRecords,
-Map 
outputResults) {
+ControllerResult incrementalAlterConfig(
+ConfigResource configResource,
+Map> keyToOps,
+boolean newlyCreatedResource
+) {
+List outputRecords = new ArrayList<>();
+ApiError apiError = incrementalAlterConfigResource(configResource,
+keyToOps,
+newlyCreatedResource,
+outputRecords);
+return ControllerResult.atomicOf(outputRecords, apiError);
+}
+
+private ApiError incrementalAlterConfigResource(
+ConfigResource configResource,
+Map> keysToOps,
+boolean newlyCreatedResource,
+List outputRecords
+) {
 List newRecords = new ArrayList<>();
 for (Entry> keysToOpsEntry : 
keysToOps.entrySet()) {
 String key = keysToOpsEntry.getKey()

[kafka] branch trunk updated: KAFKA-14612: Make sure to write a new topics ConfigRecords to metadata log iff the topic is created (#13104)

2023-01-12 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 0d9a7022a44 KAFKA-14612: Make sure to write a new topics ConfigRecords 
to metadata log iff the topic is created (#13104)
0d9a7022a44 is described below

commit 0d9a7022a4486e08dc4bd71e2fd7632ecdd3b76f
Author: andymg3 <97484553+andy...@users.noreply.github.com>
AuthorDate: Thu Jan 12 19:23:57 2023 -0500

KAFKA-14612: Make sure to write a new topics ConfigRecords to metadata log 
iff the topic is created (#13104)

### JIRA
https://issues.apache.org/jira/browse/KAFKA-14612

### Details
Makes sure we emit `ConfigRecord`s for a topic iff it actually gets 
created. Currently, we might emit `ConfigRecord`s even if the topic creation 
fails later in the `createTopics` method.

I created a new method `incrementalAlterConfig` in 
`ConfigurationControlManager` that is similar to `incrementalAlterConfig` but 
it just handles one config at a time. This is used in 
`ReplicationControlManager` for each topic. By handling one topic's config at a 
time, it's easier to isolate each topic's config records. This enables us to 
make sure we only write config records for topics that get created.

I refactored `incrementalAlterConfigResource` to return an `ApiError`. This 
made it easier to implement the new method `incrementalAlterConfig` in 
`ConfigurationControlManager` because it then doesnt have to search in the 
`Map` for the result.

### Testing
Enhanced pre-existing test 
`ReplicationControlManagerTest.testCreateTopicsWithConfigs`. I ran the tests 
without the changes to `ReplicationControlManager` and made sure each assertion 
ends up failing. Also ran `./gradlew metadata:test --tests 
org.apache.kafka.controller.ReplicationControlManagerTest`.

Reviewers: Jason Gustafson 
---
 .../controller/ConfigurationControlManager.java| 45 +++---
 .../controller/ReplicationControlManager.java  | 31 
 .../ConfigurationControlManagerTest.java   | 25 ++
 .../controller/ReplicationControlManagerTest.java  | 55 ++
 4 files changed, 130 insertions(+), 26 deletions(-)

diff --git 
a/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java
 
b/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java
index b5d71230cf4..2cbf51c9baf 100644
--- 
a/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java
+++ 
b/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java
@@ -166,26 +166,41 @@ public class ConfigurationControlManager {
  * @return  The result.
  */
 ControllerResult> incrementalAlterConfigs(
-Map>> 
configChanges,
-boolean newlyCreatedResource) {
+Map>> configChanges,
+boolean newlyCreatedResource
+) {
 List outputRecords = new ArrayList<>();
 Map outputResults = new HashMap<>();
 for (Entry>> 
resourceEntry :
 configChanges.entrySet()) {
-incrementalAlterConfigResource(resourceEntry.getKey(),
+ApiError apiError = 
incrementalAlterConfigResource(resourceEntry.getKey(),
 resourceEntry.getValue(),
 newlyCreatedResource,
-outputRecords,
-outputResults);
+outputRecords);
+outputResults.put(resourceEntry.getKey(), apiError);
 }
 return ControllerResult.atomicOf(outputRecords, outputResults);
 }
 
-private void incrementalAlterConfigResource(ConfigResource configResource,
-Map> keysToOps,
-boolean newlyCreatedResource,
-List 
outputRecords,
-Map 
outputResults) {
+ControllerResult incrementalAlterConfig(
+ConfigResource configResource,
+Map> keyToOps,
+boolean newlyCreatedResource
+) {
+List outputRecords = new ArrayList<>();
+ApiError apiError = incrementalAlterConfigResource(configResource,
+keyToOps,
+newlyCreatedResource,
+outputRecords);
+return ControllerResult.atomicOf(outputRecords, apiError);
+}
+
+private ApiError incrementalAlterConfigResource(
+ConfigResource configResource,
+Map> keysToOps,
+boolean newlyCreatedResource,
+List outputRecords
+) {
 List newRecords = new ArrayList<>();
 for (Entry> keysToOpsEntry : 
keysToOps.entrySet()) {
 String key = keysToOpsEntry.getKey()

[kafka] branch 3.3 updated: KAFKA-14417: Address incompatible error code returned by broker from `InitProducerId` (#12968)

2022-12-19 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch 3.3
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.3 by this push:
 new 0b75a38dabf KAFKA-14417: Address incompatible error code returned by 
broker from `InitProducerId` (#12968)
0b75a38dabf is described below

commit 0b75a38dabf77b915b6bf2df5113cf71a9dfba34
Author: Justine Olshan 
AuthorDate: Mon Dec 19 09:33:11 2022 -0800

KAFKA-14417: Address incompatible error code returned by broker from 
`InitProducerId` (#12968)

Older clients can not handle the `REQUEST_TIMED_OUT` error that is returned 
from `InitProducerId` when the next producerId block cannot be fetched from the 
controller. In this patch, we return `COORDINATOR_LOAD_IN_PROGRESS` instead 
which is retriable.

Reviewers: Jason Gustafson 
---
 .../kafka/coordinator/transaction/ProducerIdManager.scala   |  5 +++--
 .../coordinator/transaction/ProducerIdManagerTest.scala | 13 +++--
 2 files changed, 14 insertions(+), 4 deletions(-)

diff --git 
a/core/src/main/scala/kafka/coordinator/transaction/ProducerIdManager.scala 
b/core/src/main/scala/kafka/coordinator/transaction/ProducerIdManager.scala
index e1f46eb3712..f16785a7b6c 100644
--- a/core/src/main/scala/kafka/coordinator/transaction/ProducerIdManager.scala
+++ b/core/src/main/scala/kafka/coordinator/transaction/ProducerIdManager.scala
@@ -167,7 +167,9 @@ class RPCProducerIdManager(brokerId: Int,
   if (nextProducerId > currentProducerIdBlock.lastProducerId) {
 val block = nextProducerIdBlock.poll(maxWaitMs, TimeUnit.MILLISECONDS)
 if (block == null) {
-  throw Errors.REQUEST_TIMED_OUT.exception("Timed out waiting for next 
producer ID block")
+  // Return COORDINATOR_LOAD_IN_PROGRESS rather than REQUEST_TIMED_OUT 
since older clients treat the error as fatal
+  // when it should be retriable like COORDINATOR_LOAD_IN_PROGRESS.
+  throw Errors.COORDINATOR_LOAD_IN_PROGRESS.exception("Timed out 
waiting for next producer ID block")
 } else {
   block match {
 case Success(nextBlock) =>
@@ -236,7 +238,6 @@ class RPCProducerIdManager(brokerId: Int,
   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/test/scala/unit/kafka/coordinator/transaction/ProducerIdManagerTest.scala
 
b/core/src/test/scala/unit/kafka/coordinator/transaction/ProducerIdManagerTest.scala
index eefe61d17d6..666a3c363ff 100644
--- 
a/core/src/test/scala/unit/kafka/coordinator/transaction/ProducerIdManagerTest.scala
+++ 
b/core/src/test/scala/unit/kafka/coordinator/transaction/ProducerIdManagerTest.scala
@@ -19,6 +19,7 @@ package kafka.coordinator.transaction
 import kafka.server.BrokerToControllerChannelManager
 import kafka.zk.{KafkaZkClient, ProducerIdBlockZNode}
 import org.apache.kafka.common.KafkaException
+import org.apache.kafka.common.errors.CoordinatorLoadInProgressException
 import org.apache.kafka.common.message.AllocateProducerIdsResponseData
 import org.apache.kafka.common.protocol.Errors
 import org.apache.kafka.common.requests.AllocateProducerIdsResponse
@@ -30,7 +31,6 @@ import org.junit.jupiter.params.provider.{EnumSource, 
ValueSource}
 import org.mockito.ArgumentCaptor
 import org.mockito.ArgumentMatchers.{any, anyString}
 import org.mockito.Mockito.{mock, when}
-
 import java.util.stream.IntStream
 
 class ProducerIdManagerTest {
@@ -39,10 +39,13 @@ class ProducerIdManagerTest {
   val zkClient: KafkaZkClient = mock(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)
+  class MockProducerIdManager(val brokerId: Int, var idStart: Long, val idLen: 
Int, var error: Errors = Errors.NONE, timeout: Boolean = false)
 extends RPCProducerIdManager(brokerId, () => 1, brokerToController, 100) {
 
 override private[transaction] def sendRequest(): Unit = {
+  if (timeout)
+return
+
   if (error == Errors.NONE) {
 handleAllocateProducerIdsResponse(new AllocateProducerIdsResponse(
   new 
AllocateProducerIdsResponseData().setProducerIdStart(idStart).setProducerIdLen(idLen)))
@@ -93,6 +96,12 @@ class ProducerIdManagerTest {
 assertEquals(pid2 + ProducerIdsBlock.PRODUCER_ID_BLOCK_SIZE * 2, 
manager2.generateProducerId())
   }
 
+  @Test
+  def testRPCProducerIdManagerThrowsConcurrentTransactions(): Unit = {
+val manager1 = new MockProducerIdManager(0, 0, 0, timeout = true)
+assertThrows(classOf[CoordinatorLoadInProgressException], () => 
mana

[kafka] branch trunk updated: KAFKA-14264; New logic to discover group coordinator (#12862)

2022-12-19 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 4548c272ae3 KAFKA-14264; New logic to discover group coordinator 
(#12862)
4548c272ae3 is described below

commit 4548c272ae39a6630174ab428bc82fce7c98f7fd
Author: Philip Nee 
AuthorDate: Mon Dec 19 09:48:52 2022 -0800

KAFKA-14264; New logic to discover group coordinator (#12862)

[KAFKA-14264](https://issues.apache.org/jira/browse/KAFKA-14264)
In this patch, we refactored the existing FindCoordinator mechanism. In 
particular, we first centralize all of the network operation (send, poll) in 
`NetworkClientDelegate`, then we introduced a RequestManager interface that is 
responsible to handle the timing of different kind of requests, based on the 
implementation.  In this path, we implemented a `CoordinatorRequestManager` 
which determines when to create an `UnsentRequest` upon polling the request 
manager.

Reviewers: Jason Gustafson 
---
 .../internals/CoordinatorRequestManager.java   | 222 +
 .../internals/DefaultBackgroundThread.java | 189 ---
 .../consumer/internals/DefaultEventHandler.java|  31 +--
 ...BackgroundEvent.java => ErrorEventHandler.java} |  23 +-
 .../consumer/internals/NetworkClientDelegate.java  | 267 +
 .../consumer/internals/PrototypeAsyncConsumer.java |   7 +-
 .../BackgroundEvent.java => RequestManager.java}   |  18 +-
 .../clients/consumer/internals/RequestState.java   |  91 +++
 .../internals/events/ApplicationEvent.java |  14 +-
 ...onEvent.java => ApplicationEventProcessor.java} |  35 +--
 .../consumer/internals/events/BackgroundEvent.java |   2 +-
 ...kgroundEvent.java => ErrorBackgroundEvent.java} |  15 +-
 .../internals/events/NoopApplicationEvent.java |  16 +-
 .../common/requests/FindCoordinatorResponse.java   |  18 ++
 .../internals/CoordinatorRequestManagerTest.java   | 202 
 .../internals/DefaultBackgroundThreadTest.java | 177 +++---
 .../internals/DefaultEventHandlerTest.java |  92 +++
 .../internals/NetworkClientDelegateTest.java   | 175 ++
 .../consumer/internals/RequestStateTest.java   |  48 
 19 files changed, 1306 insertions(+), 336 deletions(-)

diff --git 
a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManager.java
 
b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManager.java
new file mode 100644
index 000..bc40b67901f
--- /dev/null
+++ 
b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CoordinatorRequestManager.java
@@ -0,0 +1,222 @@
+/*
+ * 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.clients.consumer.internals;
+
+import org.apache.kafka.clients.ClientResponse;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.errors.GroupAuthorizationException;
+import org.apache.kafka.common.errors.RetriableException;
+import org.apache.kafka.common.message.FindCoordinatorRequestData;
+import org.apache.kafka.common.message.FindCoordinatorResponseData;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.FindCoordinatorRequest;
+import org.apache.kafka.common.requests.FindCoordinatorResponse;
+import org.apache.kafka.common.utils.LogContext;
+import org.slf4j.Logger;
+
+import java.util.Collections;
+import java.util.Objects;
+import java.util.Optional;
+
+/**
+ * This is responsible for timing to send the next {@link 
FindCoordinatorRequest} based on the following criteria:
+ *
+ * Whether there is an existing coordinator.
+ * Whether there is an inflight request.
+ * Whether the backoff timer has expired.
+ * The {@link 
org.apache.kafka.clients.consumer.internals.NetworkClientDelegate.PollResult} 
contains either a wait timer
+ * or a singleton list of {@link 
org.apache.kafka.clients.consumer.internals.NetworkClientDelegate.Un

[kafka] branch 3.4 updated: KAFKA-14417: Address incompatible error code returned by broker from `InitProducerId` (#12968)

2022-12-19 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch 3.4
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.4 by this push:
 new 3ff56ebfc7a KAFKA-14417: Address incompatible error code returned by 
broker from `InitProducerId` (#12968)
3ff56ebfc7a is described below

commit 3ff56ebfc7ac06bb8b0b604ff9780ddcf6da6f2c
Author: Justine Olshan 
AuthorDate: Mon Dec 19 09:33:11 2022 -0800

KAFKA-14417: Address incompatible error code returned by broker from 
`InitProducerId` (#12968)

Older clients can not handle the `REQUEST_TIMED_OUT` error that is returned 
from `InitProducerId` when the next producerId block cannot be fetched from the 
controller. In this patch, we return `COORDINATOR_LOAD_IN_PROGRESS` instead 
which is retriable.

Reviewers: Jason Gustafson 
---
 .../kafka/coordinator/transaction/ProducerIdManager.scala   |  5 +++--
 .../coordinator/transaction/ProducerIdManagerTest.scala | 13 +++--
 2 files changed, 14 insertions(+), 4 deletions(-)

diff --git 
a/core/src/main/scala/kafka/coordinator/transaction/ProducerIdManager.scala 
b/core/src/main/scala/kafka/coordinator/transaction/ProducerIdManager.scala
index e1f46eb3712..f16785a7b6c 100644
--- a/core/src/main/scala/kafka/coordinator/transaction/ProducerIdManager.scala
+++ b/core/src/main/scala/kafka/coordinator/transaction/ProducerIdManager.scala
@@ -167,7 +167,9 @@ class RPCProducerIdManager(brokerId: Int,
   if (nextProducerId > currentProducerIdBlock.lastProducerId) {
 val block = nextProducerIdBlock.poll(maxWaitMs, TimeUnit.MILLISECONDS)
 if (block == null) {
-  throw Errors.REQUEST_TIMED_OUT.exception("Timed out waiting for next 
producer ID block")
+  // Return COORDINATOR_LOAD_IN_PROGRESS rather than REQUEST_TIMED_OUT 
since older clients treat the error as fatal
+  // when it should be retriable like COORDINATOR_LOAD_IN_PROGRESS.
+  throw Errors.COORDINATOR_LOAD_IN_PROGRESS.exception("Timed out 
waiting for next producer ID block")
 } else {
   block match {
 case Success(nextBlock) =>
@@ -236,7 +238,6 @@ class RPCProducerIdManager(brokerId: Int,
   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/test/scala/unit/kafka/coordinator/transaction/ProducerIdManagerTest.scala
 
b/core/src/test/scala/unit/kafka/coordinator/transaction/ProducerIdManagerTest.scala
index eefe61d17d6..666a3c363ff 100644
--- 
a/core/src/test/scala/unit/kafka/coordinator/transaction/ProducerIdManagerTest.scala
+++ 
b/core/src/test/scala/unit/kafka/coordinator/transaction/ProducerIdManagerTest.scala
@@ -19,6 +19,7 @@ package kafka.coordinator.transaction
 import kafka.server.BrokerToControllerChannelManager
 import kafka.zk.{KafkaZkClient, ProducerIdBlockZNode}
 import org.apache.kafka.common.KafkaException
+import org.apache.kafka.common.errors.CoordinatorLoadInProgressException
 import org.apache.kafka.common.message.AllocateProducerIdsResponseData
 import org.apache.kafka.common.protocol.Errors
 import org.apache.kafka.common.requests.AllocateProducerIdsResponse
@@ -30,7 +31,6 @@ import org.junit.jupiter.params.provider.{EnumSource, 
ValueSource}
 import org.mockito.ArgumentCaptor
 import org.mockito.ArgumentMatchers.{any, anyString}
 import org.mockito.Mockito.{mock, when}
-
 import java.util.stream.IntStream
 
 class ProducerIdManagerTest {
@@ -39,10 +39,13 @@ class ProducerIdManagerTest {
   val zkClient: KafkaZkClient = mock(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)
+  class MockProducerIdManager(val brokerId: Int, var idStart: Long, val idLen: 
Int, var error: Errors = Errors.NONE, timeout: Boolean = false)
 extends RPCProducerIdManager(brokerId, () => 1, brokerToController, 100) {
 
 override private[transaction] def sendRequest(): Unit = {
+  if (timeout)
+return
+
   if (error == Errors.NONE) {
 handleAllocateProducerIdsResponse(new AllocateProducerIdsResponse(
   new 
AllocateProducerIdsResponseData().setProducerIdStart(idStart).setProducerIdLen(idLen)))
@@ -93,6 +96,12 @@ class ProducerIdManagerTest {
 assertEquals(pid2 + ProducerIdsBlock.PRODUCER_ID_BLOCK_SIZE * 2, 
manager2.generateProducerId())
   }
 
+  @Test
+  def testRPCProducerIdManagerThrowsConcurrentTransactions(): Unit = {
+val manager1 = new MockProducerIdManager(0, 0, 0, timeout = true)
+assertThrows(classOf[CoordinatorLoadInProgressException], () => 
mana

[kafka] branch trunk updated: KAFKA-14417: Address incompatible error code returned by broker from `InitProducerId` (#12968)

2022-12-19 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 22bb3e7a585 KAFKA-14417: Address incompatible error code returned by 
broker from `InitProducerId` (#12968)
22bb3e7a585 is described below

commit 22bb3e7a5853c8f54fc030273f83f1db8c99648f
Author: Justine Olshan 
AuthorDate: Mon Dec 19 09:33:11 2022 -0800

KAFKA-14417: Address incompatible error code returned by broker from 
`InitProducerId` (#12968)

Older clients can not handle the `REQUEST_TIMED_OUT` error that is returned 
from `InitProducerId` when the next producerId block cannot be fetched from the 
controller. In this patch, we return `COORDINATOR_LOAD_IN_PROGRESS` instead 
which is retriable.

Reviewers: Jason Gustafson 
---
 .../kafka/coordinator/transaction/ProducerIdManager.scala   |  5 +++--
 .../coordinator/transaction/ProducerIdManagerTest.scala | 13 +++--
 2 files changed, 14 insertions(+), 4 deletions(-)

diff --git 
a/core/src/main/scala/kafka/coordinator/transaction/ProducerIdManager.scala 
b/core/src/main/scala/kafka/coordinator/transaction/ProducerIdManager.scala
index e1f46eb3712..f16785a7b6c 100644
--- a/core/src/main/scala/kafka/coordinator/transaction/ProducerIdManager.scala
+++ b/core/src/main/scala/kafka/coordinator/transaction/ProducerIdManager.scala
@@ -167,7 +167,9 @@ class RPCProducerIdManager(brokerId: Int,
   if (nextProducerId > currentProducerIdBlock.lastProducerId) {
 val block = nextProducerIdBlock.poll(maxWaitMs, TimeUnit.MILLISECONDS)
 if (block == null) {
-  throw Errors.REQUEST_TIMED_OUT.exception("Timed out waiting for next 
producer ID block")
+  // Return COORDINATOR_LOAD_IN_PROGRESS rather than REQUEST_TIMED_OUT 
since older clients treat the error as fatal
+  // when it should be retriable like COORDINATOR_LOAD_IN_PROGRESS.
+  throw Errors.COORDINATOR_LOAD_IN_PROGRESS.exception("Timed out 
waiting for next producer ID block")
 } else {
   block match {
 case Success(nextBlock) =>
@@ -236,7 +238,6 @@ class RPCProducerIdManager(brokerId: Int,
   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/test/scala/unit/kafka/coordinator/transaction/ProducerIdManagerTest.scala
 
b/core/src/test/scala/unit/kafka/coordinator/transaction/ProducerIdManagerTest.scala
index eefe61d17d6..666a3c363ff 100644
--- 
a/core/src/test/scala/unit/kafka/coordinator/transaction/ProducerIdManagerTest.scala
+++ 
b/core/src/test/scala/unit/kafka/coordinator/transaction/ProducerIdManagerTest.scala
@@ -19,6 +19,7 @@ package kafka.coordinator.transaction
 import kafka.server.BrokerToControllerChannelManager
 import kafka.zk.{KafkaZkClient, ProducerIdBlockZNode}
 import org.apache.kafka.common.KafkaException
+import org.apache.kafka.common.errors.CoordinatorLoadInProgressException
 import org.apache.kafka.common.message.AllocateProducerIdsResponseData
 import org.apache.kafka.common.protocol.Errors
 import org.apache.kafka.common.requests.AllocateProducerIdsResponse
@@ -30,7 +31,6 @@ import org.junit.jupiter.params.provider.{EnumSource, 
ValueSource}
 import org.mockito.ArgumentCaptor
 import org.mockito.ArgumentMatchers.{any, anyString}
 import org.mockito.Mockito.{mock, when}
-
 import java.util.stream.IntStream
 
 class ProducerIdManagerTest {
@@ -39,10 +39,13 @@ class ProducerIdManagerTest {
   val zkClient: KafkaZkClient = mock(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)
+  class MockProducerIdManager(val brokerId: Int, var idStart: Long, val idLen: 
Int, var error: Errors = Errors.NONE, timeout: Boolean = false)
 extends RPCProducerIdManager(brokerId, () => 1, brokerToController, 100) {
 
 override private[transaction] def sendRequest(): Unit = {
+  if (timeout)
+return
+
   if (error == Errors.NONE) {
 handleAllocateProducerIdsResponse(new AllocateProducerIdsResponse(
   new 
AllocateProducerIdsResponseData().setProducerIdStart(idStart).setProducerIdLen(idLen)))
@@ -93,6 +96,12 @@ class ProducerIdManagerTest {
 assertEquals(pid2 + ProducerIdsBlock.PRODUCER_ID_BLOCK_SIZE * 2, 
manager2.generateProducerId())
   }
 
+  @Test
+  def testRPCProducerIdManagerThrowsConcurrentTransactions(): Unit = {
+val manager1 = new MockProducerIdManager(0, 0, 0, timeout = true)
+assertThrows(classOf[CoordinatorLoadInProgressException], () => 
mana

[kafka] branch trunk updated: MINOR: Pass snapshot ID directly in `RaftClient.createSnapshot` (#12981)

2022-12-13 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 26a4d420726 MINOR: Pass snapshot ID directly in 
`RaftClient.createSnapshot` (#12981)
26a4d420726 is described below

commit 26a4d420726f4c63da6ea5a045694a6c6ac87207
Author: Jason Gustafson 
AuthorDate: Tue Dec 13 10:44:56 2022 -0800

MINOR: Pass snapshot ID directly in `RaftClient.createSnapshot` (#12981)

Let `RaftClient.createSnapshot` take the snapshotId directly instead of the 
committed offset/epoch (which may not exist).

Reviewers: José Armando García Sancio 
---
 .../src/main/scala/kafka/server/BrokerServer.scala |  4 +-
 .../apache/kafka/controller/QuorumController.java  |  9 +++-
 .../org/apache/kafka/metalog/LocalLogManager.java  |  8 ++--
 .../org/apache/kafka/raft/KafkaRaftClient.java |  5 +--
 .../java/org/apache/kafka/raft/RaftClient.java | 23 ++-
 .../org/apache/kafka/raft/ReplicatedCounter.java   |  3 +-
 .../kafka/snapshot/RecordsSnapshotWriter.java  | 46 +++--
 .../org/apache/kafka/snapshot/SnapshotWriter.java  |  2 +-
 .../kafka/raft/KafkaRaftClientSnapshotTest.java| 48 +++---
 .../kafka/snapshot/SnapshotWriterReaderTest.java   |  8 ++--
 10 files changed, 90 insertions(+), 66 deletions(-)

diff --git a/core/src/main/scala/kafka/server/BrokerServer.scala 
b/core/src/main/scala/kafka/server/BrokerServer.scala
index a83da6e0258..623338fd2f1 100644
--- a/core/src/main/scala/kafka/server/BrokerServer.scala
+++ b/core/src/main/scala/kafka/server/BrokerServer.scala
@@ -43,6 +43,7 @@ import org.apache.kafka.common.utils.{LogContext, Time, Utils}
 import org.apache.kafka.common.{ClusterResource, Endpoint}
 import org.apache.kafka.metadata.authorizer.ClusterMetadataAuthorizer
 import org.apache.kafka.metadata.{BrokerState, VersionRange}
+import org.apache.kafka.raft
 import org.apache.kafka.raft.{RaftClient, RaftConfig}
 import org.apache.kafka.server.authorizer.Authorizer
 import org.apache.kafka.server.common.ApiMessageAndVersion
@@ -59,7 +60,8 @@ class BrokerSnapshotWriterBuilder(raftClient: 
RaftClient[ApiMessageAndVersion])
   override def build(committedOffset: Long,
  committedEpoch: Int,
  lastContainedLogTime: Long): 
Option[SnapshotWriter[ApiMessageAndVersion]] = {
-raftClient.createSnapshot(committedOffset, committedEpoch, 
lastContainedLogTime).asScala
+val snapshotId = new raft.OffsetAndEpoch(committedOffset + 1, 
committedEpoch)
+raftClient.createSnapshot(snapshotId, lastContainedLogTime).asScala
   }
 }
 
diff --git 
a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java 
b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java
index 37e96b294d7..3dba5b401d6 100644
--- a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java
+++ b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java
@@ -533,9 +533,14 @@ public final class QuorumController implements Controller {
 )
 );
 }
+
+OffsetAndEpoch snapshotId = new OffsetAndEpoch(
+committedOffset + 1,
+committedEpoch
+);
+
 Optional> writer = 
raftClient.createSnapshot(
-committedOffset,
-committedEpoch,
+snapshotId,
 committedTimestamp
 );
 if (writer.isPresent()) {
diff --git 
a/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java 
b/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java
index 20c5c324dd2..3bc07c06af0 100644
--- a/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java
+++ b/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java
@@ -40,8 +40,8 @@ import org.apache.kafka.snapshot.RawSnapshotReader;
 import org.apache.kafka.snapshot.RawSnapshotWriter;
 import org.apache.kafka.snapshot.RecordsSnapshotReader;
 import org.apache.kafka.snapshot.RecordsSnapshotWriter;
-import org.apache.kafka.snapshot.SnapshotWriter;
 import org.apache.kafka.snapshot.SnapshotReader;
+import org.apache.kafka.snapshot.SnapshotWriter;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -51,8 +51,8 @@ import java.util.HashMap;
 import java.util.IdentityHashMap;
 import java.util.Iterator;
 import java.util.List;
-import java.util.Map.Entry;
 import java.util.Map;
+import java.util.Map.Entry;
 import java.util.NavigableMap;
 import java.util.Objects;
 import java.util.Optional;
@@ -774,11 +774,9 @@ public final class LocalLogManager implements 
RaftClient,
 
 @Override
 public Optional> createSnapshot(
-long committedOffset,
-int committedEpoch,
+OffsetAndEpoch snapshotId,
   

[kafka] branch 3.3 updated: KAFKA-14417: Producer doesn't handle REQUEST_TIMED_OUT for InitProducerIdRequest, treats as fatal error (#12915)

2022-12-08 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch 3.3
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.3 by this push:
 new d424aea8c3d KAFKA-14417: Producer doesn't handle REQUEST_TIMED_OUT for 
InitProducerIdRequest, treats as fatal error (#12915)
d424aea8c3d is described below

commit d424aea8c3d847f7d5d742693e47a3f8495cee93
Author: Justine Olshan 
AuthorDate: Tue Dec 6 11:41:31 2022 -0800

KAFKA-14417: Producer doesn't handle REQUEST_TIMED_OUT for 
InitProducerIdRequest, treats as fatal error (#12915)

The broker may return the `REQUEST_TIMED_OUT` error in `InitProducerId` 
responses when allocating the ID using the `AllocateProducerIds` request. The 
client currently does not handle this. Instead of retrying as we would expect, 
the client raises a fatal exception to the application.

In this patch, we address this problem by modifying the producer to handle 
`REQUEST_TIMED_OUT` and any other retriable errors by re-enqueuing the request.

Reviewers: Jason Gustafson 
---
 .../producer/internals/TransactionManager.java | 10 ++--
 .../errors/ConcurrentTransactionsException.java|  2 +-
 .../producer/internals/TransactionManagerTest.java | 57 ++
 3 files changed, 63 insertions(+), 6 deletions(-)

diff --git 
a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java
 
b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java
index 5aab62eaf22..de5a6ced41c 100644
--- 
a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java
+++ 
b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java
@@ -1290,7 +1290,7 @@ public class TransactionManager {
 } else if (error == Errors.NOT_COORDINATOR || error == 
Errors.COORDINATOR_NOT_AVAILABLE) {
 
lookupCoordinator(FindCoordinatorRequest.CoordinatorType.TRANSACTION, 
transactionalId);
 reenqueue();
-} else if (error == Errors.COORDINATOR_LOAD_IN_PROGRESS || error 
== Errors.CONCURRENT_TRANSACTIONS) {
+} else if (error.exception() instanceof RetriableException) {
 reenqueue();
 } else if (error == Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED ||
 error == Errors.CLUSTER_AUTHORIZATION_FAILED) {
@@ -1347,7 +1347,7 @@ public class TransactionManager {
 maybeOverrideRetryBackoffMs();
 reenqueue();
 return;
-} else if (error == Errors.COORDINATOR_LOAD_IN_PROGRESS || 
error == Errors.UNKNOWN_TOPIC_OR_PARTITION) {
+} else if (error.exception() instanceof RetriableException) {
 reenqueue();
 return;
 } else if (error == Errors.INVALID_PRODUCER_EPOCH || error == 
Errors.PRODUCER_FENCED) {
@@ -1467,7 +1467,7 @@ public class TransactionManager {
 }
 result.done();
 log.info("Discovered {} coordinator {}", 
coordinatorType.toString().toLowerCase(Locale.ROOT), node);
-} else if (error == Errors.COORDINATOR_NOT_AVAILABLE) {
+} else if (error.exception() instanceof RetriableException) {
 reenqueue();
 } else if (error == Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED) {
 fatalError(error.exception());
@@ -1515,7 +1515,7 @@ public class TransactionManager {
 } else if (error == Errors.COORDINATOR_NOT_AVAILABLE || error == 
Errors.NOT_COORDINATOR) {
 
lookupCoordinator(FindCoordinatorRequest.CoordinatorType.TRANSACTION, 
transactionalId);
 reenqueue();
-} else if (error == Errors.COORDINATOR_LOAD_IN_PROGRESS || error 
== Errors.CONCURRENT_TRANSACTIONS) {
+} else if (error.exception() instanceof RetriableException) {
 reenqueue();
 } else if (error == Errors.INVALID_PRODUCER_EPOCH || error == 
Errors.PRODUCER_FENCED) {
 // We could still receive INVALID_PRODUCER_EPOCH from old 
versioned transaction coordinator,
@@ -1572,7 +1572,7 @@ public class TransactionManager {
 } else if (error == Errors.COORDINATOR_NOT_AVAILABLE || error == 
Errors.NOT_COORDINATOR) {
 
lookupCoordinator(FindCoordinatorRequest.CoordinatorType.TRANSACTION, 
transactionalId);
 reenqueue();
-} else if (error == Errors.COORDINATOR_LOAD_IN_PROGRESS || error 
== Errors.CONCURRENT_TRANSACTIONS) {
+} else if (error.exception() instanceof RetriableException) {
 reenqueue();
 } else if (error == Errors.UNKNOWN_PRODUCER_ID || error == 
Errors.INVALID_PRODUCER_ID_MAPPING) {
 abortableErrorIfPossible(error.exception());

[kafka] branch trunk updated: KAFKA-14417: Producer doesn't handle REQUEST_TIMED_OUT for InitProducerIdRequest, treats as fatal error (#12915)

2022-12-06 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 5aad085a8e7 KAFKA-14417: Producer doesn't handle REQUEST_TIMED_OUT for 
InitProducerIdRequest, treats as fatal error (#12915)
5aad085a8e7 is described below

commit 5aad085a8e7514c14a17121d316a2e2b2add8bcc
Author: Justine Olshan 
AuthorDate: Tue Dec 6 11:41:31 2022 -0800

KAFKA-14417: Producer doesn't handle REQUEST_TIMED_OUT for 
InitProducerIdRequest, treats as fatal error (#12915)

The broker may return the `REQUEST_TIMED_OUT` error in `InitProducerId` 
responses when allocating the ID using the `AllocateProducerIds` request. The 
client currently does not handle this. Instead of retrying as we would expect, 
the client raises a fatal exception to the application.

In this patch, we address this problem by modifying the producer to handle 
`REQUEST_TIMED_OUT` and any other retriable errors by re-enqueuing the request.

Reviewers: Jason Gustafson 
---
 .../producer/internals/TransactionManager.java | 10 ++--
 .../errors/ConcurrentTransactionsException.java|  2 +-
 .../producer/internals/TransactionManagerTest.java | 57 ++
 3 files changed, 63 insertions(+), 6 deletions(-)

diff --git 
a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java
 
b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java
index 5aab62eaf22..de5a6ced41c 100644
--- 
a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java
+++ 
b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java
@@ -1290,7 +1290,7 @@ public class TransactionManager {
 } else if (error == Errors.NOT_COORDINATOR || error == 
Errors.COORDINATOR_NOT_AVAILABLE) {
 
lookupCoordinator(FindCoordinatorRequest.CoordinatorType.TRANSACTION, 
transactionalId);
 reenqueue();
-} else if (error == Errors.COORDINATOR_LOAD_IN_PROGRESS || error 
== Errors.CONCURRENT_TRANSACTIONS) {
+} else if (error.exception() instanceof RetriableException) {
 reenqueue();
 } else if (error == Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED ||
 error == Errors.CLUSTER_AUTHORIZATION_FAILED) {
@@ -1347,7 +1347,7 @@ public class TransactionManager {
 maybeOverrideRetryBackoffMs();
 reenqueue();
 return;
-} else if (error == Errors.COORDINATOR_LOAD_IN_PROGRESS || 
error == Errors.UNKNOWN_TOPIC_OR_PARTITION) {
+} else if (error.exception() instanceof RetriableException) {
 reenqueue();
 return;
 } else if (error == Errors.INVALID_PRODUCER_EPOCH || error == 
Errors.PRODUCER_FENCED) {
@@ -1467,7 +1467,7 @@ public class TransactionManager {
 }
 result.done();
 log.info("Discovered {} coordinator {}", 
coordinatorType.toString().toLowerCase(Locale.ROOT), node);
-} else if (error == Errors.COORDINATOR_NOT_AVAILABLE) {
+} else if (error.exception() instanceof RetriableException) {
 reenqueue();
 } else if (error == Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED) {
 fatalError(error.exception());
@@ -1515,7 +1515,7 @@ public class TransactionManager {
 } else if (error == Errors.COORDINATOR_NOT_AVAILABLE || error == 
Errors.NOT_COORDINATOR) {
 
lookupCoordinator(FindCoordinatorRequest.CoordinatorType.TRANSACTION, 
transactionalId);
 reenqueue();
-} else if (error == Errors.COORDINATOR_LOAD_IN_PROGRESS || error 
== Errors.CONCURRENT_TRANSACTIONS) {
+} else if (error.exception() instanceof RetriableException) {
 reenqueue();
 } else if (error == Errors.INVALID_PRODUCER_EPOCH || error == 
Errors.PRODUCER_FENCED) {
 // We could still receive INVALID_PRODUCER_EPOCH from old 
versioned transaction coordinator,
@@ -1572,7 +1572,7 @@ public class TransactionManager {
 } else if (error == Errors.COORDINATOR_NOT_AVAILABLE || error == 
Errors.NOT_COORDINATOR) {
 
lookupCoordinator(FindCoordinatorRequest.CoordinatorType.TRANSACTION, 
transactionalId);
 reenqueue();
-} else if (error == Errors.COORDINATOR_LOAD_IN_PROGRESS || error 
== Errors.CONCURRENT_TRANSACTIONS) {
+} else if (error.exception() instanceof RetriableException) {
 reenqueue();
 } else if (error == Errors.UNKNOWN_PRODUCER_ID || error == 
Errors.INVALID_PRODUCER_ID_MAPPING) {
 abortableErrorIfPossible(error.exception());

[kafka] branch trunk updated: MINOR: Fix compilation issue in FileStreamSourceConnector (#12938)

2022-12-01 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 8bb89c4beb6 MINOR: Fix compilation issue in FileStreamSourceConnector 
(#12938)
8bb89c4beb6 is described below

commit 8bb89c4beb6c21558bff6207dfa5f68729ac51bc
Author: Mickael Maison 
AuthorDate: Thu Dec 1 22:01:12 2022 +0100

MINOR: Fix compilation issue in FileStreamSourceConnector (#12938)

Fix compilation failure introduced in 
https://github.com/apache/kafka/pull/12355.

Reviewers: Mickael Maison 
---
 .../java/org/apache/kafka/connect/file/FileStreamSourceConnector.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git 
a/connect/file/src/main/java/org/apache/kafka/connect/file/FileStreamSourceConnector.java
 
b/connect/file/src/main/java/org/apache/kafka/connect/file/FileStreamSourceConnector.java
index e732d55f81d..55090d889c9 100644
--- 
a/connect/file/src/main/java/org/apache/kafka/connect/file/FileStreamSourceConnector.java
+++ 
b/connect/file/src/main/java/org/apache/kafka/connect/file/FileStreamSourceConnector.java
@@ -91,7 +91,7 @@ public class FileStreamSourceConnector extends 
SourceConnector {
 @Override
 public ExactlyOnceSupport exactlyOnceSupport(Map props) {
 AbstractConfig parsedConfig = new AbstractConfig(CONFIG_DEF, props);
-filename = parsedConfig.getString(FILE_CONFIG);
+String filename = parsedConfig.getString(FILE_CONFIG);
 // We can provide exactly-once guarantees if reading from a "real" file
 // (as long as the file is only appended to over the lifetime of the 
connector)
 // If we're reading from stdin, we can't provide exactly-once 
guarantees



[kafka] branch trunk updated (76214bfb859 -> c2fc36f3319)

2022-11-16 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a change to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


from 76214bfb859 KAFKA-13785: Add JavaDocs for emit final (#12867)
 add c2fc36f3319 MINOR: Handle JoinGroupResponseData.protocolName backward 
compatibility in JoinGroupResponse (#12864)

No new revisions were added by this update.

Summary of changes:
 .../kafka/common/requests/JoinGroupRequest.java|  2 +-
 .../kafka/common/requests/JoinGroupResponse.java   | 10 +--
 .../kafka/clients/consumer/KafkaConsumerTest.java  |  6 ++--
 .../internals/AbstractCoordinatorTest.java |  4 ++-
 .../internals/ConsumerCoordinatorTest.java |  6 ++--
 .../common/requests/JoinGroupResponseTest.java | 35 +++---
 .../kafka/common/requests/RequestResponseTest.java |  2 +-
 .../runtime/distributed/WorkerCoordinatorTest.java | 15 ++
 core/src/main/scala/kafka/server/KafkaApis.scala   | 10 ++-
 9 files changed, 50 insertions(+), 40 deletions(-)
 copy metadata/src/test/java/org/apache/kafka/metadata/BrokerStateTest.java => 
clients/src/test/java/org/apache/kafka/common/requests/JoinGroupResponseTest.java
 (50%)



[kafka] branch trunk updated (e7c1e4a0a15 -> dc18dd921c1)

2022-11-01 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a change to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


from e7c1e4a0a15 KAFKA-14299: Handle TaskCorruptedException during 
initialization (#12771)
 add dc18dd921c1 MINOR: Remove outdated comment about metadata request 
(#12810)

No new revisions were added by this update.

Summary of changes:
 .../apache/kafka/clients/consumer/internals/AbstractCoordinator.java | 5 ++---
 1 file changed, 2 insertions(+), 3 deletions(-)



[kafka] branch trunk updated: MINOR: Reduce tries in RecordsIteratorTest to improve build time (#12798)

2022-10-31 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new c710ecd071f MINOR: Reduce tries in RecordsIteratorTest to improve 
build time (#12798)
c710ecd071f is described below

commit c710ecd071fef2796a3a626a0464933605358474
Author: Jason Gustafson 
AuthorDate: Mon Oct 31 09:29:19 2022 -0700

MINOR: Reduce tries in RecordsIteratorTest to improve build time (#12798)

`RecordsIteratorTest` takes the longest times in recent builds (even 
including integration tests). The default of 1000 tries from jqwik is probably 
overkill and causes the test to take 10 minutes locally. Decreasing to 50 tries 
reduces that to less than 30s.

Reviewers: David Jacot 
---
 .../org/apache/kafka/raft/internals/RecordsIteratorTest.java   | 10 +-
 1 file changed, 5 insertions(+), 5 deletions(-)

diff --git 
a/raft/src/test/java/org/apache/kafka/raft/internals/RecordsIteratorTest.java 
b/raft/src/test/java/org/apache/kafka/raft/internals/RecordsIteratorTest.java
index 9dfbfd62fbf..9676831b30e 100644
--- 
a/raft/src/test/java/org/apache/kafka/raft/internals/RecordsIteratorTest.java
+++ 
b/raft/src/test/java/org/apache/kafka/raft/internals/RecordsIteratorTest.java
@@ -66,7 +66,7 @@ public final class RecordsIteratorTest {
 testIterator(Collections.emptyList(), records, true);
 }
 
-@Property
+@Property(tries = 50)
 public void testMemoryRecords(
 @ForAll CompressionType compressionType,
 @ForAll long seed
@@ -77,7 +77,7 @@ public final class RecordsIteratorTest {
 testIterator(batches, memRecords, true);
 }
 
-@Property
+@Property(tries = 50)
 public void testFileRecords(
 @ForAll CompressionType compressionType,
 @ForAll long seed
@@ -92,10 +92,10 @@ public final class RecordsIteratorTest {
 fileRecords.close();
 }
 
-@Property
+@Property(tries = 50)
 public void testCrcValidation(
-@ForAll CompressionType compressionType,
-@ForAll long seed
+@ForAll CompressionType compressionType,
+@ForAll long seed
 ) throws IOException {
 List> batches = createBatches(seed);
 MemoryRecords memRecords = buildRecords(compressionType, batches);



[kafka] branch trunk updated: MINOR: Add test case for topic recreation with collision chars (#12796)

2022-10-28 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new cbe50d95a98 MINOR: Add test case for topic recreation with collision 
chars (#12796)
cbe50d95a98 is described below

commit cbe50d95a980529c84698d0d2be476ff178c2c29
Author: Jason Gustafson 
AuthorDate: Fri Oct 28 18:50:08 2022 -0700

MINOR: Add test case for topic recreation with collision chars (#12796)

This patch adds a unit test for topic recreation with colliding characters 
(such as `.`). This was broken up until 
https://github.com/apache/kafka/pull/12790.

Reviewers: José Armando García Sancio 
---
 .../controller/ReplicationControlManager.java  |  4 +++
 .../controller/ReplicationControlManagerTest.java  | 31 ++
 2 files changed, 35 insertions(+)

diff --git 
a/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java
 
b/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java
index 8c3e2880334..6683d5e1476 100644
--- 
a/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java
+++ 
b/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java
@@ -263,6 +263,10 @@ public class ReplicationControlManager {
 public Uuid topicId() {
 return id;
 }
+
+public int numPartitions(long epoch) {
+return parts.size(epoch);
+}
 }
 
 /**
diff --git 
a/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java
 
b/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java
index ff67e0e6d66..c81f25e4c00 100644
--- 
a/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java
+++ 
b/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java
@@ -60,6 +60,7 @@ import org.apache.kafka.common.metadata.ConfigRecord;
 import org.apache.kafka.common.metadata.PartitionChangeRecord;
 import org.apache.kafka.common.metadata.PartitionRecord;
 import org.apache.kafka.common.metadata.RegisterBrokerRecord;
+import org.apache.kafka.common.metadata.RemoveTopicRecord;
 import org.apache.kafka.common.metadata.TopicRecord;
 import org.apache.kafka.common.protocol.ApiKeys;
 import org.apache.kafka.common.protocol.Errors;
@@ -269,6 +270,21 @@ public class ReplicationControlManagerTest {
 return topicResult;
 }
 
+void deleteTopic(Uuid topicId) throws Exception {
+ControllerResult> result = 
replicationControl.deleteTopics(Collections.singleton(topicId));
+assertEquals(Collections.singleton(topicId), 
result.response().keySet());
+assertEquals(NONE, result.response().get(topicId).error());
+assertEquals(1, result.records().size());
+
+ApiMessageAndVersion removeRecordAndVersion = 
result.records().get(0);
+assertTrue(removeRecordAndVersion.message() instanceof 
RemoveTopicRecord);
+
+RemoveTopicRecord removeRecord = (RemoveTopicRecord) 
removeRecordAndVersion.message();
+assertEquals(topicId, removeRecord.topicId());
+
+replay(result.records());
+}
+
 void createPartitions(int count, String name,
 int[][] replicas, short expectedErrorCode) throws Exception {
 assertFalse(replicas.length == 0);
@@ -698,6 +714,21 @@ public class ReplicationControlManagerTest {
 ctx.createTestTopic("quux", new int[][] {new int[] {1, 2, 0}}, 
POLICY_VIOLATION.code());
 }
 
+@Test
+public void testCreateTopicWithCollisionChars() throws Exception {
+ReplicationControlTestContext ctx = new 
ReplicationControlTestContext(Optional.empty());
+ctx.registerBrokers(0, 1, 2);
+ctx.unfenceBrokers(0, 1, 2);
+
+CreatableTopicResult initialTopic = ctx.createTestTopic("foo.bar", 2, 
(short) 2, NONE.code());
+assertEquals(2, 
ctx.replicationControl.getTopic(initialTopic.topicId()).numPartitions(Long.MAX_VALUE));
+ctx.deleteTopic(initialTopic.topicId());
+
+CreatableTopicResult recreatedTopic = ctx.createTestTopic("foo.bar", 
4, (short) 2, NONE.code());
+assertNotEquals(initialTopic.topicId(), recreatedTopic.topicId());
+assertEquals(4, 
ctx.replicationControl.getTopic(recreatedTopic.topicId()).numPartitions(Long.MAX_VALUE));
+}
+
 @Test
 public void testGlobalTopicAndPartitionMetrics() throws Exception {
 ReplicationControlTestContext ctx = new 
ReplicationControlTestContext();



[kafka] branch 3.3 updated: KAFKA-14337; Correctly remove topicsWithCollisionChars after topic deletion (#12790)

2022-10-28 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch 3.3
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.3 by this push:
 new 026f98f8c47 KAFKA-14337; Correctly remove topicsWithCollisionChars 
after topic deletion (#12790)
026f98f8c47 is described below

commit 026f98f8c478fa306eaff99b0eba7107fb40be74
Author: Luke Chen 
AuthorDate: Sat Oct 29 01:08:53 2022 +0800

KAFKA-14337; Correctly remove topicsWithCollisionChars after topic deletion 
(#12790)

In https://github.com/apache/kafka/pull/11910 , we added a feature to 
prevent topics with conflicting metrics names from being created. We added a 
map to store the normalized topic name to the topic names, but we didn't remove 
it correctly while deleting topics. This PR fixes this bug and add a test.

Reviewers: Igor Soarez , dengziming 
, Jason Gustafson 
---
 .../kafka/admin/TopicCommandIntegrationTest.scala  | 25 ++
 .../controller/ReplicationControlManager.java  |  4 ++--
 2 files changed, 27 insertions(+), 2 deletions(-)

diff --git 
a/core/src/test/scala/integration/kafka/admin/TopicCommandIntegrationTest.scala 
b/core/src/test/scala/integration/kafka/admin/TopicCommandIntegrationTest.scala
index 3082babd06f..ea4c748da92 100644
--- 
a/core/src/test/scala/integration/kafka/admin/TopicCommandIntegrationTest.scala
+++ 
b/core/src/test/scala/integration/kafka/admin/TopicCommandIntegrationTest.scala
@@ -33,6 +33,7 @@ import org.apache.kafka.common.protocol.Errors
 import org.apache.kafka.common.requests.MetadataResponse
 import org.apache.kafka.common.security.auth.SecurityProtocol
 import org.junit.jupiter.api.Assertions._
+import org.junit.jupiter.api.function.Executable
 import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo}
 import org.junit.jupiter.params.ParameterizedTest
 import org.junit.jupiter.params.provider.ValueSource
@@ -470,6 +471,30 @@ class TopicCommandIntegrationTest extends 
KafkaServerTestHarness with Logging wi
 TestUtils.verifyTopicDeletion(zkClientOrNull, testTopicName, 1, brokers)
   }
 
+  @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName)
+  @ValueSource(strings = Array("zk", "kraft"))
+  def testTopicWithCollidingCharDeletionAndCreateAgain(quorum: String): Unit = 
{
+// create the topic with colliding chars
+val topicWithCollidingChar = "test.a"
+val createOpts = new TopicCommandOptions(Array("--partitions", "1",
+  "--replication-factor", "1",
+  "--topic", topicWithCollidingChar))
+createAndWaitTopic(createOpts)
+
+// delete the topic
+val deleteOpts = new TopicCommandOptions(Array("--topic", 
topicWithCollidingChar))
+
+if (!isKRaftTest()) {
+  val deletePath = DeleteTopicsTopicZNode.path(topicWithCollidingChar)
+  assertFalse(zkClient.pathExists(deletePath), "Delete path for topic 
shouldn't exist before deletion.")
+}
+topicService.deleteTopic(deleteOpts)
+TestUtils.verifyTopicDeletion(zkClientOrNull, topicWithCollidingChar, 1, 
brokers)
+
+val createTopic: Executable = () => createAndWaitTopic(createOpts)
+assertDoesNotThrow(createTopic)
+  }
+
   @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName)
   @ValueSource(strings = Array("zk", "kraft"))
   def testDeleteInternalTopic(quorum: String): Unit = {
diff --git 
a/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java
 
b/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java
index df7097df83d..1fc713c207f 100644
--- 
a/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java
+++ 
b/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java
@@ -342,7 +342,7 @@ public class ReplicationControlManager {
  * Since we reject topic creations that would collide, under normal 
conditions the
  * sets in this map should only have a size of 1. However, if the cluster 
was
  * upgraded from a version prior to KAFKA-13743, it may be possible to 
have more
- * values here, since collidiing topic names will be "grandfathered in."
+ * values here, since colliding topic names will be "grandfathered in."
  */
 private final TimelineHashMap> 
topicsWithCollisionChars;
 
@@ -527,7 +527,7 @@ public class ReplicationControlManager {
 if (colliding != null) {
 colliding.remove(topic.name);
 if (colliding.isEmpty()) {
-topicsWithCollisionChars.remove(topic.name);
+topicsWithCollisionChars.remove(normalizedName);
 }
 }
 }



[kafka] branch trunk updated: KAFKA-14337; Correctly remove topicsWithCollisionChars after topic deletion (#12790)

2022-10-28 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new c1bb307a361 KAFKA-14337; Correctly remove topicsWithCollisionChars 
after topic deletion (#12790)
c1bb307a361 is described below

commit c1bb307a361b7b6e17261cc84ea2b108bacca84d
Author: Luke Chen 
AuthorDate: Sat Oct 29 01:08:53 2022 +0800

KAFKA-14337; Correctly remove topicsWithCollisionChars after topic deletion 
(#12790)

In https://github.com/apache/kafka/pull/11910 , we added a feature to 
prevent topics with conflicting metrics names from being created. We added a 
map to store the normalized topic name to the topic names, but we didn't remove 
it correctly while deleting topics. This PR fixes this bug and add a test.

Reviewers: Igor Soarez , dengziming 
, Jason Gustafson 
---
 .../kafka/admin/TopicCommandIntegrationTest.scala  | 25 ++
 .../controller/ReplicationControlManager.java  |  4 ++--
 2 files changed, 27 insertions(+), 2 deletions(-)

diff --git 
a/core/src/test/scala/integration/kafka/admin/TopicCommandIntegrationTest.scala 
b/core/src/test/scala/integration/kafka/admin/TopicCommandIntegrationTest.scala
index 3082babd06f..ea4c748da92 100644
--- 
a/core/src/test/scala/integration/kafka/admin/TopicCommandIntegrationTest.scala
+++ 
b/core/src/test/scala/integration/kafka/admin/TopicCommandIntegrationTest.scala
@@ -33,6 +33,7 @@ import org.apache.kafka.common.protocol.Errors
 import org.apache.kafka.common.requests.MetadataResponse
 import org.apache.kafka.common.security.auth.SecurityProtocol
 import org.junit.jupiter.api.Assertions._
+import org.junit.jupiter.api.function.Executable
 import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo}
 import org.junit.jupiter.params.ParameterizedTest
 import org.junit.jupiter.params.provider.ValueSource
@@ -470,6 +471,30 @@ class TopicCommandIntegrationTest extends 
KafkaServerTestHarness with Logging wi
 TestUtils.verifyTopicDeletion(zkClientOrNull, testTopicName, 1, brokers)
   }
 
+  @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName)
+  @ValueSource(strings = Array("zk", "kraft"))
+  def testTopicWithCollidingCharDeletionAndCreateAgain(quorum: String): Unit = 
{
+// create the topic with colliding chars
+val topicWithCollidingChar = "test.a"
+val createOpts = new TopicCommandOptions(Array("--partitions", "1",
+  "--replication-factor", "1",
+  "--topic", topicWithCollidingChar))
+createAndWaitTopic(createOpts)
+
+// delete the topic
+val deleteOpts = new TopicCommandOptions(Array("--topic", 
topicWithCollidingChar))
+
+if (!isKRaftTest()) {
+  val deletePath = DeleteTopicsTopicZNode.path(topicWithCollidingChar)
+  assertFalse(zkClient.pathExists(deletePath), "Delete path for topic 
shouldn't exist before deletion.")
+}
+topicService.deleteTopic(deleteOpts)
+TestUtils.verifyTopicDeletion(zkClientOrNull, topicWithCollidingChar, 1, 
brokers)
+
+val createTopic: Executable = () => createAndWaitTopic(createOpts)
+assertDoesNotThrow(createTopic)
+  }
+
   @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName)
   @ValueSource(strings = Array("zk", "kraft"))
   def testDeleteInternalTopic(quorum: String): Unit = {
diff --git 
a/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java
 
b/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java
index c48e1aed819..8c3e2880334 100644
--- 
a/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java
+++ 
b/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java
@@ -342,7 +342,7 @@ public class ReplicationControlManager {
  * Since we reject topic creations that would collide, under normal 
conditions the
  * sets in this map should only have a size of 1. However, if the cluster 
was
  * upgraded from a version prior to KAFKA-13743, it may be possible to 
have more
- * values here, since collidiing topic names will be "grandfathered in."
+ * values here, since colliding topic names will be "grandfathered in."
  */
 private final TimelineHashMap> 
topicsWithCollisionChars;
 
@@ -527,7 +527,7 @@ public class ReplicationControlManager {
 if (colliding != null) {
 colliding.remove(topic.name);
 if (colliding.isEmpty()) {
-topicsWithCollisionChars.remove(topic.name);
+topicsWithCollisionChars.remove(normalizedName);
 }
 }
 }



[kafka] branch trunk updated (0a045d4ef7d -> b525ddc9f1f)

2022-10-20 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a change to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


from 0a045d4ef7d KAFKA-14247: Consumer background thread base 
implementation (#12672)
 add b525ddc9f1f MINOR: Fix PartitionRegistration.hashCode (#12774)

No new revisions were added by this update.

Summary of changes:
 .../kafka/metadata/PartitionRegistration.java  |  4 +-
 .../kafka/metadata/PartitionRegistrationTest.java  | 45 ++
 2 files changed, 47 insertions(+), 2 deletions(-)



[kafka] branch 3.3 updated: KAFKA-14316; Fix feature control iterator metadata version handling (#12765)

2022-10-18 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch 3.3
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.3 by this push:
 new 5cd9c9bf1df KAFKA-14316; Fix feature control iterator metadata version 
handling (#12765)
5cd9c9bf1df is described below

commit 5cd9c9bf1df34d60d4d147bfb77e220fb76467f7
Author: Jason Gustafson 
AuthorDate: Tue Oct 18 15:30:45 2022 -0700

KAFKA-14316; Fix feature control iterator metadata version handling (#12765)

The iterator `FeatureControlIterator.hasNext()` checks two conditions: 1) 
whether we have already written the metadata version, and 2) whether the 
underlying iterator has additional records. However, in `next()`, we also check 
that the metadata version is at least high enough to include it in the log. 
When this fails, then we can see an unexpected `NoSuchElementException` if the 
underlying iterator is empty.

Reviewers: Colin Patrick McCabe 
---
 .../kafka/controller/FeatureControlManager.java | 21 -
 .../kafka/controller/FeatureControlManagerTest.java | 18 ++
 2 files changed, 30 insertions(+), 9 deletions(-)

diff --git 
a/metadata/src/main/java/org/apache/kafka/controller/FeatureControlManager.java 
b/metadata/src/main/java/org/apache/kafka/controller/FeatureControlManager.java
index 274e28906bf..e596b705f45 100644
--- 
a/metadata/src/main/java/org/apache/kafka/controller/FeatureControlManager.java
+++ 
b/metadata/src/main/java/org/apache/kafka/controller/FeatureControlManager.java
@@ -320,22 +320,25 @@ public class FeatureControlManager {
 
 @Override
 public boolean hasNext() {
-return !wroteVersion || iterator.hasNext();
+return needsWriteMetadataVersion() || iterator.hasNext();
+}
+
+private boolean needsWriteMetadataVersion() {
+return !wroteVersion && 
metadataVersion.isAtLeast(minimumBootstrapVersion);
 }
 
 @Override
 public List next() {
 // Write the metadata.version first
-if (!wroteVersion) {
-if (metadataVersion.isAtLeast(minimumBootstrapVersion)) {
-wroteVersion = true;
-return Collections.singletonList(new 
ApiMessageAndVersion(new FeatureLevelRecord()
-.setName(MetadataVersion.FEATURE_NAME)
-.setFeatureLevel(metadataVersion.featureLevel()), 
FEATURE_LEVEL_RECORD.lowestSupportedVersion()));
-}
+if (needsWriteMetadataVersion()) {
+wroteVersion = true;
+return Collections.singletonList(new ApiMessageAndVersion(new 
FeatureLevelRecord()
+.setName(MetadataVersion.FEATURE_NAME)
+.setFeatureLevel(metadataVersion.featureLevel()), 
FEATURE_LEVEL_RECORD.lowestSupportedVersion()));
 }
+
 // Then write the rest of the features
-if (!hasNext()) throw new NoSuchElementException();
+if (!iterator.hasNext()) throw new NoSuchElementException();
 Entry entry = iterator.next();
 return Collections.singletonList(new ApiMessageAndVersion(new 
FeatureLevelRecord()
 .setName(entry.getKey())
diff --git 
a/metadata/src/test/java/org/apache/kafka/controller/FeatureControlManagerTest.java
 
b/metadata/src/test/java/org/apache/kafka/controller/FeatureControlManagerTest.java
index 3845f646668..2478f4ce164 100644
--- 
a/metadata/src/test/java/org/apache/kafka/controller/FeatureControlManagerTest.java
+++ 
b/metadata/src/test/java/org/apache/kafka/controller/FeatureControlManagerTest.java
@@ -201,6 +201,24 @@ public class FeatureControlManagerTest {
 );
 }
 
+@Test
+public void testFeatureControlIteratorWithOldMetadataVersion() throws 
Exception {
+// We require minimum of IBP_3_3_IV0 to write metadata version in the 
snapshot.
+
+LogContext logContext = new LogContext();
+SnapshotRegistry snapshotRegistry = new SnapshotRegistry(logContext);
+FeatureControlManager manager = new FeatureControlManager.Builder()
+.setLogContext(logContext)
+.setSnapshotRegistry(snapshotRegistry)
+.setMetadataVersion(MetadataVersion.IBP_3_2_IV0)
+.build();
+
+RecordTestUtils.assertBatchIteratorContains(
+Collections.emptyList(),
+manager.iterator(Long.MAX_VALUE)
+);
+}
+
 @Test
 public void testFeatureControlIterator() throws Exception {
 LogContext logContext = new LogContext();



[kafka] branch trunk updated: KAFKA-14316; Fix feature control iterator metadata version handling (#12765)

2022-10-18 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new a692223a44c KAFKA-14316; Fix feature control iterator metadata version 
handling (#12765)
a692223a44c is described below

commit a692223a44cd92c6379705865426cd68858cc830
Author: Jason Gustafson 
AuthorDate: Tue Oct 18 15:30:45 2022 -0700

KAFKA-14316; Fix feature control iterator metadata version handling (#12765)

The iterator `FeatureControlIterator.hasNext()` checks two conditions: 1) 
whether we have already written the metadata version, and 2) whether the 
underlying iterator has additional records. However, in `next()`, we also check 
that the metadata version is at least high enough to include it in the log. 
When this fails, then we can see an unexpected `NoSuchElementException` if the 
underlying iterator is empty.

Reviewers: Colin Patrick McCabe 
---
 .../kafka/controller/FeatureControlManager.java | 21 -
 .../kafka/controller/FeatureControlManagerTest.java | 18 ++
 2 files changed, 30 insertions(+), 9 deletions(-)

diff --git 
a/metadata/src/main/java/org/apache/kafka/controller/FeatureControlManager.java 
b/metadata/src/main/java/org/apache/kafka/controller/FeatureControlManager.java
index 274e28906bf..e596b705f45 100644
--- 
a/metadata/src/main/java/org/apache/kafka/controller/FeatureControlManager.java
+++ 
b/metadata/src/main/java/org/apache/kafka/controller/FeatureControlManager.java
@@ -320,22 +320,25 @@ public class FeatureControlManager {
 
 @Override
 public boolean hasNext() {
-return !wroteVersion || iterator.hasNext();
+return needsWriteMetadataVersion() || iterator.hasNext();
+}
+
+private boolean needsWriteMetadataVersion() {
+return !wroteVersion && 
metadataVersion.isAtLeast(minimumBootstrapVersion);
 }
 
 @Override
 public List next() {
 // Write the metadata.version first
-if (!wroteVersion) {
-if (metadataVersion.isAtLeast(minimumBootstrapVersion)) {
-wroteVersion = true;
-return Collections.singletonList(new 
ApiMessageAndVersion(new FeatureLevelRecord()
-.setName(MetadataVersion.FEATURE_NAME)
-.setFeatureLevel(metadataVersion.featureLevel()), 
FEATURE_LEVEL_RECORD.lowestSupportedVersion()));
-}
+if (needsWriteMetadataVersion()) {
+wroteVersion = true;
+return Collections.singletonList(new ApiMessageAndVersion(new 
FeatureLevelRecord()
+.setName(MetadataVersion.FEATURE_NAME)
+.setFeatureLevel(metadataVersion.featureLevel()), 
FEATURE_LEVEL_RECORD.lowestSupportedVersion()));
 }
+
 // Then write the rest of the features
-if (!hasNext()) throw new NoSuchElementException();
+if (!iterator.hasNext()) throw new NoSuchElementException();
 Entry entry = iterator.next();
 return Collections.singletonList(new ApiMessageAndVersion(new 
FeatureLevelRecord()
 .setName(entry.getKey())
diff --git 
a/metadata/src/test/java/org/apache/kafka/controller/FeatureControlManagerTest.java
 
b/metadata/src/test/java/org/apache/kafka/controller/FeatureControlManagerTest.java
index 3845f646668..2478f4ce164 100644
--- 
a/metadata/src/test/java/org/apache/kafka/controller/FeatureControlManagerTest.java
+++ 
b/metadata/src/test/java/org/apache/kafka/controller/FeatureControlManagerTest.java
@@ -201,6 +201,24 @@ public class FeatureControlManagerTest {
 );
 }
 
+@Test
+public void testFeatureControlIteratorWithOldMetadataVersion() throws 
Exception {
+// We require minimum of IBP_3_3_IV0 to write metadata version in the 
snapshot.
+
+LogContext logContext = new LogContext();
+SnapshotRegistry snapshotRegistry = new SnapshotRegistry(logContext);
+FeatureControlManager manager = new FeatureControlManager.Builder()
+.setLogContext(logContext)
+.setSnapshotRegistry(snapshotRegistry)
+.setMetadataVersion(MetadataVersion.IBP_3_2_IV0)
+.build();
+
+RecordTestUtils.assertBatchIteratorContains(
+Collections.emptyList(),
+manager.iterator(Long.MAX_VALUE)
+);
+}
+
 @Test
 public void testFeatureControlIterator() throws Exception {
 LogContext logContext = new LogContext();



[kafka] branch trunk updated: MINOR: Address flakiness in `KRaftClusterTest::testDescribeQuorumRequestToBrokers` (#12738)

2022-10-17 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 3e7eddecd6a MINOR: Address flakiness in 
`KRaftClusterTest::testDescribeQuorumRequestToBrokers` (#12738)
3e7eddecd6a is described below

commit 3e7eddecd6a63ea6a9793d3270bef6d0be5c9021
Author: Niket 
AuthorDate: Mon Oct 17 16:16:26 2022 -0700

MINOR: Address flakiness in 
`KRaftClusterTest::testDescribeQuorumRequestToBrokers` (#12738)

We have seen some errors such as the following:
```
org.opentest4j.AssertionFailedError: expected: not equal but was: 

Stacktrace
org.opentest4j.AssertionFailedError: expected: not equal but was: 

at org.junit.jupiter.api.AssertionUtils.fail(AssertionUtils.java:39)
at 
org.junit.jupiter.api.AssertNotEquals.failEqual(AssertNotEquals.java:276)
at 
org.junit.jupiter.api.AssertNotEquals.assertNotEquals(AssertNotEquals.java:265)
at 
org.junit.jupiter.api.AssertNotEquals.assertNotEquals(AssertNotEquals.java:260)
at 
org.junit.jupiter.api.Assertions.assertNotEquals(Assertions.java:2815)
at 
kafka.server.KRaftClusterTest.$anonfun$testDescribeQuorumRequestToBrokers$5(KRaftClusterTest.scala:818)
at java.util.ArrayList.forEach(ArrayList.java:1259)
at 
kafka.server.KRaftClusterTest.testDescribeQuorumRequestToBrokers(KRaftClusterTest.scala:814)
```
The patch changes some of the assertions to wait longer for the condition 
to be satisfied.

Reviewers: Jason Gustafson 
---
 .../kafka/server/KRaftClusterTest.scala| 39 ++
 1 file changed, 26 insertions(+), 13 deletions(-)

diff --git 
a/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala 
b/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala
index cc726653f75..f52c2a72da2 100644
--- a/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala
+++ b/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala
@@ -811,20 +811,33 @@ class KRaftClusterTest {
 
assertTrue(cluster.controllers.asScala.keySet.contains(quorumInfo.leaderId),
   s"Leader ID ${quorumInfo.leaderId} was not a controller ID.")
 
-quorumInfo.voters.forEach { voter =>
-  assertTrue(0 < voter.logEndOffset,
-s"logEndOffset for voter with ID ${voter.replicaId} was 
${voter.logEndOffset}")
-  assertNotEquals(OptionalLong.empty(), voter.lastFetchTimestamp)
-  assertNotEquals(OptionalLong.empty(), voter.lastCaughtUpTimestamp)
-}
+val (voters, voterResponseValid) =
+  TestUtils.computeUntilTrue(
+admin.describeMetadataQuorum(new DescribeMetadataQuorumOptions)
+  .quorumInfo().get().voters()
+  ) { voters => voters.stream
+.allMatch(voter => (voter.logEndOffset > 0
+  && voter.lastFetchTimestamp() != OptionalLong.empty()
+  && voter.lastCaughtUpTimestamp() != OptionalLong.empty()))
+  }
 
-assertEquals(cluster.brokers.asScala.keySet, 
quorumInfo.observers.asScala.map(_.replicaId).toSet)
-quorumInfo.observers.forEach { observer =>
-  assertTrue(0 < observer.logEndOffset,
-s"logEndOffset for observer with ID ${observer.replicaId} was 
${observer.logEndOffset}")
-  assertNotEquals(OptionalLong.empty(), observer.lastFetchTimestamp)
-  assertNotEquals(OptionalLong.empty(), observer.lastCaughtUpTimestamp)
-}
+assertTrue(voterResponseValid, s"At least one voter did not return the 
expected state within timeout." +
+  s"The responses gathered for all the voters: ${voters.toString}")
+
+val (observers, observerResponseValid) =
+  TestUtils.computeUntilTrue(
+admin.describeMetadataQuorum(new DescribeMetadataQuorumOptions)
+  .quorumInfo().get().observers()
+  ) { observers =>
+(
+  cluster.brokers.asScala.keySet == 
observers.asScala.map(_.replicaId).toSet
+&& observers.stream.allMatch(observer => 
(observer.logEndOffset > 0
+&& observer.lastFetchTimestamp() != OptionalLong.empty()
+&& observer.lastCaughtUpTimestamp() != OptionalLong.empty(
+  }
+
+assertTrue(observerResponseValid, s"At least one observer did not 
return the expected state within timeout." +
+s"The responses gathered for all the observers: 
${observers.toString}")
   } finally {
 admin.close()
   }



[kafka] branch 3.3 updated: KAFKA-14296; Partition leaders are not demoted during kraft controlled shutdown (#12741)

2022-10-13 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch 3.3
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.3 by this push:
 new fd64b53e6a3 KAFKA-14296; Partition leaders are not demoted during 
kraft controlled shutdown (#12741)
fd64b53e6a3 is described below

commit fd64b53e6a3f0c3e088ca457caeecad1210536b7
Author: David Jacot 
AuthorDate: Fri Oct 14 01:59:19 2022 +0200

KAFKA-14296; Partition leaders are not demoted during kraft controlled 
shutdown (#12741)

When the `BrokerServer` starts its shutting down process, it transitions to 
`SHUTTING_DOWN` and sets `isShuttingDown` to `true`. With this state change, 
the follower state changes are short-cutted. This means that a broker which was 
serving as leader would remain acting as a leader until controlled shutdown 
completes. Instead, we want the leader and ISR state to be updated so that 
requests will return NOT_LEADER and the client can find the new leader.

We missed this case while implementing 
https://github.com/apache/kafka/pull/12187.

This patch fixes the issue and updates an existing test to ensure that 
`isShuttingDown` has not effect. We should consider adding integration tests 
for this as well. We can do this separately.

Reviewers: Ismael Juma , José Armando García Sancio 
, Jason Gustafson 
---
 .../main/scala/kafka/server/ReplicaManager.scala   | 39 ++
 .../unit/kafka/server/ReplicaManagerTest.scala | 14 ++--
 2 files changed, 28 insertions(+), 25 deletions(-)

diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala 
b/core/src/main/scala/kafka/server/ReplicaManager.scala
index f60bd53a085..00ab7d563fa 100644
--- a/core/src/main/scala/kafka/server/ReplicaManager.scala
+++ b/core/src/main/scala/kafka/server/ReplicaManager.scala
@@ -2142,7 +2142,6 @@ class ReplicaManager(val config: KafkaConfig,
   ): Unit = {
 stateChangeLogger.info(s"Transitioning ${localFollowers.size} partition(s) 
to " +
   "local followers.")
-val shuttingDown = isShuttingDown.get()
 val partitionsToStartFetching = new mutable.HashMap[TopicPartition, 
Partition]
 val partitionsToStopFetching = new mutable.HashMap[TopicPartition, Boolean]
 val followerTopicSet = new mutable.HashSet[String]
@@ -2151,28 +2150,24 @@ class ReplicaManager(val config: KafkaConfig,
 try {
   followerTopicSet.add(tp.topic)
 
-  if (shuttingDown) {
-stateChangeLogger.trace(s"Unable to start fetching $tp with topic 
" +
-  s"ID ${info.topicId} because the replica manager is shutting 
down.")
-  } else {
-// We always update the follower state.
-// - This ensure that a replica with no leader can step down;
-// - This also ensures that the local replica is created even if 
the leader
-//   is unavailable. This is required to ensure that we include 
the partition's
-//   high watermark in the checkpoint file (see KAFKA-1647).
-val state = info.partition.toLeaderAndIsrPartitionState(tp, isNew)
-val isNewLeaderEpoch = partition.makeFollower(state, 
offsetCheckpoints, Some(info.topicId))
-
-if (isInControlledShutdown && (info.partition.leader == NO_LEADER 
||
-!info.partition.isr.contains(config.brokerId))) {
-  // During controlled shutdown, replica with no leaders and 
replica
-  // where this broker is not in the ISR are stopped.
-  partitionsToStopFetching.put(tp, false)
-} else if (isNewLeaderEpoch) {
-  // Otherwise, fetcher is restarted if the leader epoch has 
changed.
-  partitionsToStartFetching.put(tp, partition)
-}
+  // We always update the follower state.
+  // - This ensure that a replica with no leader can step down;
+  // - This also ensures that the local replica is created even if the 
leader
+  //   is unavailable. This is required to ensure that we include the 
partition's
+  //   high watermark in the checkpoint file (see KAFKA-1647).
+  val state = info.partition.toLeaderAndIsrPartitionState(tp, isNew)
+  val isNewLeaderEpoch = partition.makeFollower(state, 
offsetCheckpoints, Some(info.topicId))
+
+  if (isInControlledShutdown && (info.partition.leader == NO_LEADER ||
+  !info.partition.isr.contains(config.brokerId))) {
+// During controlled shutdown, replica with no leaders and replica
+// where this broker is not in the ISR are stopped.
+partitionsToStopFetching.put(tp, false)
+  } else if (isNewLeaderEpoch) {
+// Otherwise, fetcher is restarted if the leader epoch has changed.
+partitionsToStartFetching.put(tp, pa

[kafka] branch trunk updated (0cb1d614136 -> 5cff8f67b06)

2022-10-13 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a change to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


from 0cb1d614136 KAFKA-14292; Fix KRaft controlled shutdown delay (#12736)
 add 5cff8f67b06 KAFKA-14296; Partition leaders are not demoted during 
kraft controlled shutdown (#12741)

No new revisions were added by this update.

Summary of changes:
 .../main/scala/kafka/server/ReplicaManager.scala   | 39 ++
 .../unit/kafka/server/ReplicaManagerTest.scala | 14 ++--
 2 files changed, 28 insertions(+), 25 deletions(-)



[kafka] branch 3.3 updated: KAFKA-14292; Fix KRaft controlled shutdown delay (#12736)

2022-10-13 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch 3.3
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.3 by this push:
 new eda3d76ead2 KAFKA-14292; Fix KRaft controlled shutdown delay (#12736)
eda3d76ead2 is described below

commit eda3d76ead2aec87dfef347bc922451ee56e687b
Author: Alyssa Huang 
AuthorDate: Thu Oct 13 13:29:45 2022 -0700

KAFKA-14292; Fix KRaft controlled shutdown delay (#12736)

The `controlledShutDownOffset` is defined as the "offset at which the 
broker should complete its controlled shutdown, or -1 if the broker is not 
performing a controlled shutdown". The controller sets this offset to a 
non-negative integer on receiving a heartbeat from a broker that's in 
controlled shutdown state. Currently, this offset is being updated and bumped 
every single time a broker in controlled shutdown mode send a heartbeat, 
delaying when controlled shutdown can actually comp [...]

Reviewers: Luke Chen , Jason Gustafson 

---
 .../kafka/controller/BrokerHeartbeatManager.java   | 43 +++---
 .../apache/kafka/controller/QuorumController.java  |  2 +-
 .../controller/BrokerHeartbeatManagerTest.java | 30 +--
 3 files changed, 56 insertions(+), 19 deletions(-)

diff --git 
a/metadata/src/main/java/org/apache/kafka/controller/BrokerHeartbeatManager.java
 
b/metadata/src/main/java/org/apache/kafka/controller/BrokerHeartbeatManager.java
index 428f1c5833e..f061cd95034 100644
--- 
a/metadata/src/main/java/org/apache/kafka/controller/BrokerHeartbeatManager.java
+++ 
b/metadata/src/main/java/org/apache/kafka/controller/BrokerHeartbeatManager.java
@@ -17,6 +17,7 @@
 
 package org.apache.kafka.controller;
 
+import java.util.OptionalLong;
 import org.apache.kafka.common.message.BrokerHeartbeatRequestData;
 import org.apache.kafka.common.utils.LogContext;
 import org.apache.kafka.common.utils.Time;
@@ -42,8 +43,9 @@ import static 
org.apache.kafka.controller.BrokerControlState.UNFENCED;
 /**
  * The BrokerHeartbeatManager manages all the soft state associated with 
broker heartbeats.
  * Soft state is state which does not appear in the metadata log.  This state 
includes
- * things like the last time each broker sent us a heartbeat, and whether the 
broker is
- * trying to perform a controlled shutdown.
+ * things like the last time each broker sent us a heartbeat.  As of KIP-841, 
the controlled
+ * shutdown state is no longer treated as soft state and is persisted to the 
metadata log on broker
+ * controlled shutdown requests.
  *
  * Only the active controller has a BrokerHeartbeatManager, since only the 
active
  * controller handles broker heartbeats.  Standby controllers will create a 
heartbeat
@@ -77,7 +79,7 @@ public class BrokerHeartbeatManager {
  * if the broker is not performing a controlled shutdown.  When this 
field is
  * updated, we also have to update the broker's position in the 
shuttingDown set.
  */
-private long controlledShutDownOffset;
+private long controlledShutdownOffset;
 
 /**
  * The previous entry in the unfenced list, or null if the broker is 
not in that list.
@@ -95,7 +97,7 @@ public class BrokerHeartbeatManager {
 this.prev = null;
 this.next = null;
 this.metadataOffset = -1;
-this.controlledShutDownOffset = -1;
+this.controlledShutdownOffset = -1;
 }
 
 /**
@@ -116,7 +118,7 @@ public class BrokerHeartbeatManager {
  * Returns true only if the broker is in controlled shutdown state.
  */
 boolean shuttingDown() {
-return controlledShutDownOffset >= 0;
+return controlledShutdownOffset >= 0;
 }
 }
 
@@ -275,6 +277,16 @@ public class BrokerHeartbeatManager {
 return brokers.values();
 }
 
+// VisibleForTesting
+OptionalLong controlledShutdownOffset(int brokerId) {
+BrokerHeartbeatState broker = brokers.get(brokerId);
+if (broker == null || broker.controlledShutdownOffset == -1) {
+return OptionalLong.empty();
+}
+return OptionalLong.of(broker.controlledShutdownOffset);
+}
+
+
 /**
  * Mark a broker as fenced.
  *
@@ -381,7 +393,7 @@ public class BrokerHeartbeatManager {
 if (fenced) {
 // If a broker is fenced, it leaves controlled shutdown.  On its 
next heartbeat,
 // it will shut down immediately.
-broker.controlledShutDownOffset = -1;
+broker.controlledShutdownOffset = -1;
 } else {
 unfenced.add(broker);
 if (!broker.shuttingDown()) {
@@ -400,12 +412,13 @@ public class BrokerHeartbeatManager {
 }
 
 /**
- * Mark a broker as being in the controlled shutdown state.
+ * Mark a broker as being in the controlled shutdown state

[kafka] branch trunk updated (484f85ff53b -> 0cb1d614136)

2022-10-13 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a change to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


from 484f85ff53b HOTFIX: Revert "KAFKA-12960: Enforcing strict retention 
time for WindowStore and Sess… (#11211)" (#12745)
 add 0cb1d614136 KAFKA-14292; Fix KRaft controlled shutdown delay (#12736)

No new revisions were added by this update.

Summary of changes:
 .../kafka/controller/BrokerHeartbeatManager.java   | 43 +++---
 .../apache/kafka/controller/QuorumController.java  |  2 +-
 .../controller/BrokerHeartbeatManagerTest.java | 30 +--
 3 files changed, 56 insertions(+), 19 deletions(-)



[kafka] branch trunk updated (dac81161dbf -> 484f85ff53b)

2022-10-13 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a change to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


from dac81161dbf MINOR; Introduce ImageWriter and ImageWriterOptions 
(#12715)
 add 484f85ff53b HOTFIX: Revert "KAFKA-12960: Enforcing strict retention 
time for WindowStore and Sess… (#11211)" (#12745)

No new revisions were added by this update.

Summary of changes:
 ...stractDualSchemaRocksDBSegmentedBytesStore.java |  47 +
 .../AbstractRocksDBSegmentedBytesStore.java|  75 ++--
 ...tractRocksDBTimeOrderedSegmentedBytesStore.java |  70 +++-
 .../internals/RocksDBSegmentedBytesStore.java  |   2 +-
 .../RocksDBTimestampedSegmentedBytesStore.java |   2 +-
 .../TimeWindowedKStreamIntegrationTest.java|  66 +++
 .../KStreamSlidingWindowAggregateTest.java |  18 +-
 .../internals/KStreamWindowAggregateTest.java  |  61 +++
 .../internals/SessionWindowedKStreamImplTest.java  |  63 ++-
 .../internals/TimeWindowedKStreamImplTest.java | 121 -
 ...ctDualSchemaRocksDBSegmentedBytesStoreTest.java | 190 -
 .../AbstractRocksDBSegmentedBytesStoreTest.java| 163 +-
 .../internals/AbstractWindowBytesStoreTest.java|   4 +-
 .../CachingPersistentWindowStoreTest.java  |   3 +-
 .../state/internals/MeteredSessionStoreTest.java   |  49 --
 .../MeteredTimestampedWindowStoreTest.java |   1 -
 .../state/internals/MeteredWindowStoreTest.java|  14 --
 .../state/internals/RocksDBSessionStoreTest.java   |  93 --
 .../state/internals/RocksDBWindowStoreTest.java| 143 
 19 files changed, 350 insertions(+), 835 deletions(-)



[kafka] branch trunk updated (62914129c7d -> 4b30ed8e47a)

2022-10-12 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a change to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


from 62914129c7d KAFKA-14099 - Fix request logging in connect (#12434)
 add 4b30ed8e47a MINOR: Fix incorrect example in feature command help 
(#12732)

No new revisions were added by this update.

Summary of changes:
 core/src/main/scala/kafka/admin/FeatureCommand.scala | 5 ++---
 1 file changed, 2 insertions(+), 3 deletions(-)



[kafka] branch 3.3 updated: KAFKA-14225; Fix deadlock caused by lazy val exemptSensor (#12634)

2022-10-11 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch 3.3
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.3 by this push:
 new e2136746159 KAFKA-14225; Fix deadlock caused by lazy val exemptSensor 
(#12634)
e2136746159 is described below

commit e2136746159915c3cf66df0705bac65ebc6a2aa6
Author: Huilin Shi 
AuthorDate: Tue Oct 11 10:30:07 2022 -0700

KAFKA-14225; Fix deadlock caused by lazy val exemptSensor (#12634)

There is a chance to cause deadlock when multiple threads access 
ClientRequestQuotaManager. In the version Scala 2.12, the lazy val 
initialization is under the object lock. The deadlock could happen in the 
following condition:

In thread a, when ClientRequestQuotaManager.exemptSensor is being 
initialized, it has acquired the object lock and enters the the actual 
initialization block. The initialization of 'exemptSensor' requires another 
lock private val lock = new ReentrantReadWriteLock() and it is waiting for this 
lock.

In thread b, at the same time, ClientQuotaManager.updateQuota() is called 
and it has already acquired ReentrantReadWriteLock lock by calling 
lock.writeLock().lock(). And then it executes info(). If this is the first time 
accessing Logging.logger, which is also a lazy val, it need to wait for the 
object lock.

The deadlock happens.

Since the lazy val initialization is under the object lock, we should avoid 
using lazy val if the initialization function holds another lock to prevent 
holding two locks at the same time which is prone for deadlock. Change to 
create exemptSensor during ClientRequestQuotaManager initialization with an 
expiration time of Long.MaxValue to prevent expiration if request quota is not 
enabled at that time.

Reviewers: Jason Gustafson 
---
 core/src/main/scala/kafka/server/ClientQuotaManager.scala| 7 +++
 core/src/main/scala/kafka/server/ClientRequestQuotaManager.scala | 9 +++--
 core/src/test/scala/integration/kafka/api/BaseQuotaTest.scala| 3 ++-
 .../test/scala/integration/kafka/api/PlaintextConsumerTest.scala | 7 ---
 4 files changed, 12 insertions(+), 14 deletions(-)

diff --git a/core/src/main/scala/kafka/server/ClientQuotaManager.scala 
b/core/src/main/scala/kafka/server/ClientQuotaManager.scala
index ee7c70bec93..45c7c31570f 100644
--- a/core/src/main/scala/kafka/server/ClientQuotaManager.scala
+++ b/core/src/main/scala/kafka/server/ClientQuotaManager.scala
@@ -433,12 +433,11 @@ class ClientQuotaManager(private val config: 
ClientQuotaManagerConfig,
   .quota(new Quota(quotaLimit, true))
   }
 
-  protected def getOrCreateSensor(sensorName: String, metricName: MetricName): 
Sensor = {
+  protected def getOrCreateSensor(sensorName: String, expirationTimeSeconds: 
Long, registerMetrics: Sensor => Unit): Sensor = {
 sensorAccessor.getOrCreate(
   sensorName,
-  ClientQuotaManager.InactiveSensorExpirationTimeSeconds,
-  sensor => sensor.add(metricName, new Rate)
-)
+  expirationTimeSeconds,
+  registerMetrics)
   }
 
   /**
diff --git a/core/src/main/scala/kafka/server/ClientRequestQuotaManager.scala 
b/core/src/main/scala/kafka/server/ClientRequestQuotaManager.scala
index 2ceaab9c9af..6e57d97bc3e 100644
--- a/core/src/main/scala/kafka/server/ClientRequestQuotaManager.scala
+++ b/core/src/main/scala/kafka/server/ClientRequestQuotaManager.scala
@@ -17,11 +17,11 @@
 package kafka.server
 
 import java.util.concurrent.TimeUnit
-
 import kafka.network.RequestChannel
 import kafka.utils.QuotaUtils
 import org.apache.kafka.common.MetricName
 import org.apache.kafka.common.metrics._
+import org.apache.kafka.common.metrics.stats.Rate
 import org.apache.kafka.common.utils.Time
 import org.apache.kafka.server.quota.ClientQuotaCallback
 
@@ -30,6 +30,9 @@ import scala.jdk.CollectionConverters._
 object ClientRequestQuotaManager {
   val QuotaRequestPercentDefault = Int.MaxValue.toDouble
   val NanosToPercentagePerSecond = 100.0 / TimeUnit.SECONDS.toNanos(1)
+  // Since exemptSensor is for all clients and has a constant name, we do not 
expire exemptSensor and only
+  // create once.
+  val DefaultInactiveExemptSensorExpirationTimeSeconds = Long.MaxValue
 
   private val ExemptSensorName = "exempt-" + QuotaType.Request
 }
@@ -45,7 +48,9 @@ class ClientRequestQuotaManager(private val config: 
ClientQuotaManagerConfig,
   private val exemptMetricName = metrics.metricName("exempt-request-time",
 QuotaType.Request.toString, "Tracking exempt-request-time utilization 
percentage")
 
-  lazy val exemptSensor: Sensor = 
getOrCreateSensor(ClientRequestQuotaManager.ExemptSensorName, exemptMetricName)
+  val exemptSensor: Sensor = 
getOrCreateSensor(ClientRequestQuotaManager.ExemptSensorName,
+ClientRequestQuotaManager.DefaultInactiveExemptSensorExpirationTimeSeconds,
+sensor => sensor.ad

[kafka] branch trunk updated (98a3dcb4779 -> d834947ae7a)

2022-10-11 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a change to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


from 98a3dcb4779 KAFKA-14275; KRaft Controllers should crash after failing 
to apply any metadata record (#12709)
 add d834947ae7a KAFKA-14225; Fix deadlock caused by lazy val exemptSensor 
(#12634)

No new revisions were added by this update.

Summary of changes:
 core/src/main/scala/kafka/server/ClientQuotaManager.scala| 7 +++
 core/src/main/scala/kafka/server/ClientRequestQuotaManager.scala | 9 +++--
 core/src/test/scala/integration/kafka/api/BaseQuotaTest.scala| 3 ++-
 .../test/scala/integration/kafka/api/PlaintextConsumerTest.scala | 7 ---
 4 files changed, 12 insertions(+), 14 deletions(-)



[kafka] branch trunk updated (24cb2d2f95a -> 98a3dcb4779)

2022-10-11 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a change to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


from 24cb2d2f95a MINOR; Some sections are missing in the TOC (#12726)
 add 98a3dcb4779 KAFKA-14275; KRaft Controllers should crash after failing 
to apply any metadata record (#12709)

No new revisions were added by this update.

Summary of changes:
 .../main/scala/kafka/server/ControllerServer.scala |  1 -
 .../kafka/server/QuorumTestHarness.scala   |  2 +-
 .../apache/kafka/controller/QuorumController.java  | 20 +
 .../kafka/controller/QuorumControllerTest.java | 49 --
 .../kafka/controller/QuorumControllerTestEnv.java  | 31 --
 5 files changed, 66 insertions(+), 37 deletions(-)



[kafka] branch trunk updated (5c5dcb7a96 -> c5745d2845)

2022-10-04 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a change to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


from 5c5dcb7a96 MINOR; Use 3.3.1 release for system test (#12714)
 add c5745d2845 MINOR: Add initial property tests for StandardAuthorizer 
(#12703)

No new revisions were added by this update.

Summary of changes:
 .gitignore |   2 +-
 build.gradle   |   1 +
 checkstyle/import-control.xml  |   2 +
 .../org/apache/kafka/common/internals/Topic.java   |  30 ++-
 .../ControllerConfigurationValidatorTest.scala |   4 +-
 .../controller/ReplicationControlManagerTest.java  |   4 +-
 .../authorizer/StandardAuthorizerPropertyTest.java | 299 +
 7 files changed, 329 insertions(+), 13 deletions(-)
 create mode 100644 
metadata/src/test/java/org/apache/kafka/metadata/authorizer/StandardAuthorizerPropertyTest.java



[kafka] branch trunk updated (c1f23b6c9a -> c3690a3b4a)

2022-10-03 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a change to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


from c1f23b6c9a MINOR: Fix delegation token system test (#12693)
 add c3690a3b4a KAFKA-14247; Define event handler interface and events 
(#12663)

No new revisions were added by this update.

Summary of changes:
 .../consumer/internals/DefaultEventHandler.java|  60 ++
 .../consumer/internals/PrototypeAsyncConsumer.java | 126 +
 .../internals/events/ApplicationEvent.java |   8 +-
 .../consumer/internals/events/BackgroundEvent.java |   8 +-
 .../consumer/internals/events/EventHandler.java|  46 
 5 files changed, 238 insertions(+), 10 deletions(-)
 create mode 100644 
clients/src/main/java/org/apache/kafka/clients/consumer/internals/DefaultEventHandler.java
 create mode 100644 
clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java
 copy generator/src/main/java/org/apache/kafka/message/ClauseGenerator.java => 
clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEvent.java
 (81%)
 copy generator/src/main/java/org/apache/kafka/message/ClauseGenerator.java => 
clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEvent.java
 (81%)
 create mode 100644 
clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventHandler.java



[kafka] branch trunk updated: KAFKA-14248; Fix flaky test PlaintextAdminIntegrationTest.testCreateTopicsReturnsConfigs (#12669)

2022-09-29 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new bc95aa21169 KAFKA-14248; Fix flaky test 
PlaintextAdminIntegrationTest.testCreateTopicsReturnsConfigs (#12669)
bc95aa21169 is described below

commit bc95aa21169b8b5b9b8a4b609e88cc125157234b
Author: Divij Vaidya 
AuthorDate: Fri Sep 30 01:24:03 2022 +0200

KAFKA-14248; Fix flaky test 
PlaintextAdminIntegrationTest.testCreateTopicsReturnsConfigs (#12669)

The test is failing intermittently because we do not wait for propagation 
of the altered config (LogRetentionTimeMillisProp) across all brokers before 
proceeding ahead with the test.

This PR makes the following changes:
1. Wait for propagation of altered configuration to propagate to all 
brokers.
2. Use the existing `killBroker` utility method which waits for shutdown 
using `awaitshutdown`.
3. Improve code readability by using `TestUtils.incrementalAlterConfigs` to 
send alter config requests.

Reviewers: Jason Gustafson 
---
 .../kafka/api/PlaintextAdminIntegrationTest.scala  | 82 ++
 1 file changed, 38 insertions(+), 44 deletions(-)

diff --git 
a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala 
b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala
index 7121f98bb9c..1656af08bc1 100644
--- 
a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala
+++ 
b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala
@@ -33,7 +33,6 @@ import kafka.server.{Defaults, DynamicConfig, KafkaConfig}
 import kafka.utils.TestUtils._
 import kafka.utils.{Log4jController, TestInfoUtils, TestUtils}
 import org.apache.kafka.clients.HostResolver
-import org.apache.kafka.clients.admin.AlterConfigOp.OpType
 import org.apache.kafka.clients.admin.ConfigEntry.ConfigSource
 import org.apache.kafka.clients.admin._
 import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer}
@@ -159,22 +158,6 @@ class PlaintextAdminIntegrationTest extends 
BaseAdminIntegrationTest {
 waitForTopics(client, List(), topics)
   }
 
-  @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName)
-  @ValueSource(strings = Array("zk")) // KRaft mode will be supported in 
KAFKA-13910
-  def testMetadataRefresh(quorum: String): Unit = {
-client = Admin.create(createConfig)
-val topics = Seq("mytopic")
-val newTopics = Seq(new NewTopic("mytopic", 3, 3.toShort))
-client.createTopics(newTopics.asJava).all.get()
-waitForTopics(client, expectedPresent = topics, expectedMissing = List())
-
-val controller = brokers.find(_.config.brokerId == 
brokers.flatMap(_.metadataCache.getControllerId).head).get
-controller.shutdown()
-controller.awaitShutdown()
-val topicDesc = client.describeTopics(topics.asJava).allTopicNames.get()
-assertEquals(topics.toSet, topicDesc.keySet.asScala)
-  }
-
   /**
 * describe should not auto create topics
 */
@@ -821,10 +804,10 @@ class PlaintextAdminIntegrationTest extends 
BaseAdminIntegrationTest {
   @ValueSource(strings = Array("zk", "kraft"))
   def testReplicaCanFetchFromLogStartOffsetAfterDeleteRecords(quorum: String): 
Unit = {
 val leaders = createTopic(topic, replicationFactor = brokerCount)
-val followerIndex = if (leaders(0) != brokers(0).config.brokerId) 0 else 1
+val followerIndex = if (leaders(0) != brokers.head.config.brokerId) 0 else 
1
 
 def waitForFollowerLog(expectedStartOffset: Long, expectedEndOffset: 
Long): Unit = {
-  TestUtils.waitUntilTrue(() => 
brokers(followerIndex).replicaManager.localLog(topicPartition) != None,
+  TestUtils.waitUntilTrue(() => 
brokers(followerIndex).replicaManager.localLog(topicPartition).isDefined,
   "Expected follower to create replica for 
partition")
 
   // wait until the follower discovers that log start offset moved beyond 
its HW
@@ -862,6 +845,7 @@ class PlaintextAdminIntegrationTest extends 
BaseAdminIntegrationTest {
 val result1 = client.deleteRecords(Map(topicPartition -> 
RecordsToDelete.beforeOffset(117L)).asJava)
 result1.all().get()
 restartDeadBrokers()
+TestUtils.waitForBrokersInIsr(client, topicPartition, Set(followerIndex))
 waitForFollowerLog(expectedStartOffset=117L, expectedEndOffset=200L)
   }
 
@@ -1522,7 +1506,7 @@ class PlaintextAdminIntegrationTest extends 
BaseAdminIntegrationTest {
 // Now change the preferred leader to 1
 changePreferredLeader(prefer1)
 // but shut it down...
-brokers(1).shutdown()
+killBroker(1)
 TestUtils.waitForBrokersOutOfIsr(client, Set(partition1, partition2), 
Set(1))
 
 def assertPreferredLeaderNotAvailable(
@@ -1576,9 +1560,9 @@ 

[kafka] branch 3.3 updated: MINOR: Small update docs/design.html grammar and typo (#12691)

2022-09-27 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch 3.3
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.3 by this push:
 new 07a448295f7 MINOR: Small update docs/design.html grammar and typo 
(#12691)
07a448295f7 is described below

commit 07a448295f75754bc2aa7ed8c9e959c17e51e800
Author: Chase Thomas 
AuthorDate: Tue Sep 27 11:46:25 2022 -0700

MINOR: Small update docs/design.html grammar and typo (#12691)

Reviewers: Jason Gustafson 
---
 docs/design.html | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/docs/design.html b/docs/design.html
index b37300f23e5..9485ab9c500 100644
--- a/docs/design.html
+++ b/docs/design.html
@@ -322,7 +322,7 @@
 Followers consume messages from the leader just as a normal Kafka consumer 
would and apply them to their own log. Having the followers pull from the 
leader has the nice property of allowing the follower to naturally
 batch together log entries they are applying to their log.
 
-As with most distributed systems automatically handling failures requires 
having a precise definition of what it means for a node to be "alive." In 
Kafka, a special node
+As with most distributed systems, automatically handling failures requires 
a precise definition of what it means for a node to be "alive." In Kafka, a 
special node
 known as the "controller" is responsible for managing the registration of 
brokers in the cluster. Broker liveness has two conditions:
 
   Brokers must maintain an active session with the controller in order 
to receive regular metadata updates.
@@ -335,7 +335,7 @@
 
 For clusters using Zookeeper, liveness is determined indirectly through 
the existence of an ephemeral node which is created by the broker on
 initialization of its Zookeeper session. If the broker loses its session 
after failing to send heartbeats to Zookeeper before expiration of
-zookeeper.session.timeout.ms, the the node gets deleted. The 
controller would then notice the node deletion through a Zookeeper watch
+zookeeper.session.timeout.ms, then the node gets deleted. The 
controller would then notice the node deletion through a Zookeeper watch
 and mark the broker offline.
 
 We refer to nodes satisfying these two conditions as being "in sync" to 
avoid the vagueness of "alive" or "failed". The leader keeps track of the set 
of "in sync" replicas,



[kafka] branch trunk updated (5e322deb9a8 -> d2f900b055d)

2022-09-27 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a change to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


from 5e322deb9a8 MINOR: Improve unit test coverage of LeaderAndIsr class 
(#12689)
 add d2f900b055d MINOR: Small update docs/design.html grammar and typo 
(#12691)

No new revisions were added by this update.

Summary of changes:
 docs/design.html | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)



[kafka] branch 3.3 updated: MINOR: Update design docs to avoid zookeeper-specific assumptions (#12690)

2022-09-26 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch 3.3
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.3 by this push:
 new 1ce7bd7f29f MINOR: Update design docs to avoid zookeeper-specific 
assumptions (#12690)
1ce7bd7f29f is described below

commit 1ce7bd7f29f8d8d5f2b6679a7174722915ba7c36
Author: Jason Gustafson 
AuthorDate: Mon Sep 26 13:01:07 2022 -0700

MINOR: Update design docs to avoid zookeeper-specific assumptions (#12690)

Update a few cases in the documentation which do not make sense for KRaft.

Reviewers: José Armando García Sancio 
---
 docs/design.html | 36 ++--
 1 file changed, 26 insertions(+), 10 deletions(-)

diff --git a/docs/design.html b/docs/design.html
index 6e32b2d7f6f..b37300f23e5 100644
--- a/docs/design.html
+++ b/docs/design.html
@@ -322,18 +322,33 @@
 Followers consume messages from the leader just as a normal Kafka consumer 
would and apply them to their own log. Having the followers pull from the 
leader has the nice property of allowing the follower to naturally
 batch together log entries they are applying to their log.
 
-As with most distributed systems automatically handling failures requires 
having a precise definition of what it means for a node to be "alive". For 
Kafka node liveness has two conditions
+As with most distributed systems automatically handling failures requires 
having a precise definition of what it means for a node to be "alive." In 
Kafka, a special node
+known as the "controller" is responsible for managing the registration of 
brokers in the cluster. Broker liveness has two conditions:
 
-A node must be able to maintain its session with ZooKeeper (via 
ZooKeeper's heartbeat mechanism)
-If it is a follower it must replicate the writes happening on the 
leader and not fall "too far" behind
+  Brokers must maintain an active session with the controller in order 
to receive regular metadata updates.
+  Brokers acting as followers must replicate the writes from the 
leader and not fall "too far" behind.
 
-We refer to nodes satisfying these two conditions as being "in sync" to 
avoid the vagueness of "alive" or "failed". The leader keeps track of the set 
of "in sync" nodes. If a follower dies, gets stuck, or falls
-behind, the leader will remove it from the list of in sync replicas. The 
determination of stuck and lagging replicas is controlled by the 
replica.lag.time.max.ms configuration.
+
+What is meant by an "active session" depends on the cluster configuration. 
For KRaft clusters, an active session is maintained by 
+sending periodic heartbeats to the controller. If the controller fails to 
receive a heartbeat before the timeout configured by 
+broker.session.timeout.ms expires, then the node is 
considered offline.
+
+For clusters using Zookeeper, liveness is determined indirectly through 
the existence of an ephemeral node which is created by the broker on
+initialization of its Zookeeper session. If the broker loses its session 
after failing to send heartbeats to Zookeeper before expiration of
+zookeeper.session.timeout.ms, the the node gets deleted. The 
controller would then notice the node deletion through a Zookeeper watch
+and mark the broker offline.
+
+We refer to nodes satisfying these two conditions as being "in sync" to 
avoid the vagueness of "alive" or "failed". The leader keeps track of the set 
of "in sync" replicas,
+which is known as the ISR. If either of these conditions fail to be 
satisified, then the broker will be removed from the ISR. For example,
+if a follower dies, then the controller will notice the failure through 
the loss of its session, and will remove the broker from the ISR.
+On the other hand, if the follower lags too far behind the leader but 
still has an active session, then the leader can also remove it from the ISR.
+The determination of lagging replicas is controlled through the 
replica.lag.time.max.ms configuration. 
+Replicas that cannot catch up to the end of the log on the leader within 
the max time set by this configuration are removed from the ISR.
 
 In distributed systems terminology we only attempt to handle a 
"fail/recover" model of failures where nodes suddenly cease working and then 
later recover (perhaps without knowing that they have died). Kafka does not
 handle so-called "Byzantine" failures in which nodes produce arbitrary or 
malicious responses (perhaps due to bugs or foul play).
 
-We can now more precisely define that a message is considered committed 
when all in sync replicas for that partition have app

[kafka] branch trunk updated (07a31599c3f -> 1c0f8f90e2a)

2022-09-26 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a change to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


from 07a31599c3f KAFKA-10199: Fix switching to updating standbys if standby 
is removed (#12687)
 add 1c0f8f90e2a MINOR: Update design docs to avoid zookeeper-specific 
assumptions (#12690)

No new revisions were added by this update.

Summary of changes:
 docs/design.html | 36 ++--
 1 file changed, 26 insertions(+), 10 deletions(-)



[kafka] branch trunk updated: MINOR: Update security docs for kraft Authorizer configuration (#12673)

2022-09-21 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 3549a5524ec MINOR: Update security docs for kraft Authorizer 
configuration (#12673)
3549a5524ec is described below

commit 3549a5524ec3a9f0011a0991c665983f67de2122
Author: Jason Gustafson 
AuthorDate: Wed Sep 21 19:38:59 2022 -0700

MINOR: Update security docs for kraft Authorizer configuration (#12673)

Update security documentation to describe how to configure the KRaft 
`Authorizer` implementation and include a note about principal forwarding.

Additionally, this patch renames 
`KafkaConfig.Defaults.DefaultPrincipalSerde` to `DefaultPrincipalBuilder` since 
the former is somewhat misleading.

Reviewers: David Arthur 
---
 core/src/main/scala/kafka/server/KafkaConfig.scala |  4 +--
 docs/security.html | 29 --
 2 files changed, 29 insertions(+), 4 deletions(-)

diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala 
b/core/src/main/scala/kafka/server/KafkaConfig.scala
index caf9a7f682a..be91c06bf9b 100755
--- a/core/src/main/scala/kafka/server/KafkaConfig.scala
+++ b/core/src/main/scala/kafka/server/KafkaConfig.scala
@@ -263,7 +263,7 @@ object Defaults {
 /** * General Security configuration ***/
   val ConnectionsMaxReauthMsDefault = 0L
   val DefaultServerMaxMaxReceiveSize = 
BrokerSecurityConfigs.DEFAULT_SASL_SERVER_MAX_RECEIVE_SIZE
-  val DefaultPrincipalSerde = classOf[DefaultKafkaPrincipalBuilder]
+  val DefaultPrincipalBuilder = classOf[DefaultKafkaPrincipalBuilder]
 
   /** * Sasl configuration ***/
   val SaslMechanismInterBrokerProtocol = SaslConfigs.DEFAULT_SASL_MECHANISM
@@ -1345,7 +1345,7 @@ object KafkaConfig {
   .define(securityProviderClassProp, STRING, null, LOW, 
securityProviderClassDoc)
 
   /** * SSL Configuration /
-  .define(PrincipalBuilderClassProp, CLASS, 
Defaults.DefaultPrincipalSerde, MEDIUM, PrincipalBuilderClassDoc)
+  .define(PrincipalBuilderClassProp, CLASS, 
Defaults.DefaultPrincipalBuilder, MEDIUM, PrincipalBuilderClassDoc)
   .define(SslProtocolProp, STRING, Defaults.SslProtocol, MEDIUM, 
SslProtocolDoc)
   .define(SslProviderProp, STRING, null, MEDIUM, SslProviderDoc)
   .define(SslEnabledProtocolsProp, LIST, Defaults.SslEnabledProtocols, 
MEDIUM, SslEnabledProtocolsDoc)
diff --git a/docs/security.html b/docs/security.html
index 49754ed13f6..10e60cbb88c 100644
--- a/docs/security.html
+++ b/docs/security.html
@@ -1136,13 +1136,38 @@ sasl.mechanism.inter.broker.protocol=GSSAPI (or one of 
the other enabled mechani
 
 
 7.4 Authorization and 
ACLs
-Kafka ships with a pluggable Authorizer and an out-of-box authorizer 
implementation that uses zookeeper to store all the acls. The Authorizer is 
configured by setting authorizer.class.name in server.properties. To 
enable the out of the box implementation use:
+Kafka ships with a pluggable authorization framework, which is configured 
with the authorizer.class.name property in the server confgiuration.
+Configured implementations must extend 
org.apache.kafka.server.authorizer.Authorizer.
+Kafka provides default implementations which store ACLs in the cluster 
metadata (either Zookeeper or the KRaft metadata log).
+
+For Zookeeper-based clusters, the provided implementation is configured as 
follows:
 authorizer.class.name=kafka.security.authorizer.AclAuthorizer
-Kafka acls are defined in the general format of "Principal P is 
[Allowed/Denied] Operation O From Host H on any Resource R matching 
ResourcePattern RP". You can read more about the acl structure in https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+Interface;>KIP-11
 and resource patterns in https://cwiki.apache.org/confluence/display/KAFKA/KIP-290%3A+Support+for+Prefixed+ACLs;>KIP-290.
 In order to add, remove or list acls you can use th [...]
+For KRaft clusters, use the following configuration on all nodes (brokers, 
controllers, or combined broker/controller nodes):
+authorizer.class.name=org.apache.kafka.metadata.authorizer.StandardAuthorizer
+
+Kafka ACLs are defined in the general format of "Principal {P} is 
[Allowed|Denied] Operation {O} From Host {H} on any Resource {R} matching 
ResourcePattern {RP}".
+You can read more about the ACL structure in https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+Interface;>KIP-11
 and
+resource patterns in https://cwiki.apache.org/confluence/display/KAFKA/KIP-290%3A+Support+for+Prefixed+ACLs;>KIP-290.
+In order to add, remove, or list ACLs, you can use the Kafka ACL CLI 
kafka-acls.sh. By default, if no ResourcePatterns match a specific 
Resource R,
+then R has n

[kafka] branch trunk updated: KAFKA-14236; ListGroups request produces too much Denied logs in authorizer (#12652)

2022-09-21 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new cb9557a9905 KAFKA-14236; ListGroups request produces too much Denied 
logs in authorizer (#12652)
cb9557a9905 is described below

commit cb9557a99050bf317cb347cc7756a61f25ce032d
Author: aLeX 
AuthorDate: Thu Sep 22 02:42:30 2022 +0200

KAFKA-14236; ListGroups request produces too much Denied logs in authorizer 
(#12652)

Avoid `is Denied Operation` logs when calling ListGroups api, by adding 
`logIfDenied = false` to `authorize` calls.

Reviewers: Jason Gustafson 
---
 core/src/main/scala/kafka/server/KafkaApis.scala | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala 
b/core/src/main/scala/kafka/server/KafkaApis.scala
index 7b3f8269f4a..bae20f25667 100644
--- a/core/src/main/scala/kafka/server/KafkaApis.scala
+++ b/core/src/main/scala/kafka/server/KafkaApis.scala
@@ -1635,12 +1635,12 @@ class KafkaApis(val requestChannel: RequestChannel,
 )
 }
 val (error, groups) = groupCoordinator.handleListGroups(states)
-if (authHelper.authorize(request.context, DESCRIBE, CLUSTER, CLUSTER_NAME))
+if (authHelper.authorize(request.context, DESCRIBE, CLUSTER, CLUSTER_NAME, 
logIfDenied = false))
   // With describe cluster access all groups are returned. We keep this 
alternative for backward compatibility.
   requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs =>
 createResponse(requestThrottleMs, groups, error))
 else {
-  val filteredGroups = groups.filter(group => 
authHelper.authorize(request.context, DESCRIBE, GROUP, group.groupId))
+  val filteredGroups = groups.filter(group => 
authHelper.authorize(request.context, DESCRIBE, GROUP, group.groupId, 
logIfDenied = false))
   requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs =>
 createResponse(requestThrottleMs, filteredGroups, error))
 }



[kafka] branch trunk updated (5587c65fd3c -> 695424fa9d5)

2022-09-21 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a change to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


from 5587c65fd3c MINOR: Add configurable max receive size for SASL 
authentication requests
 add 695424fa9d5 MINOR: Mention deprecation of authorizer flags in security 
documentation (#12668)

No new revisions were added by this update.

Summary of changes:
 docs/security.html | 48 
 1 file changed, 24 insertions(+), 24 deletions(-)



[kafka] branch 3.3 updated: KAFKA-14240; Validate kraft snapshot state on startup (#12653)

2022-09-19 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch 3.3
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.3 by this push:
 new 0c08c80afa KAFKA-14240; Validate kraft snapshot state on startup 
(#12653)
0c08c80afa is described below

commit 0c08c80afa8a00aa520ec407d9bce6537395fc23
Author: Jason Gustafson 
AuthorDate: Mon Sep 19 11:52:48 2022 -0700

KAFKA-14240; Validate kraft snapshot state on startup (#12653)

We should prevent the metadata log from initializing in a known bad state. 
If the log start offset of the first segment is greater than 0, then must be a 
snapshot an offset greater than or equal to it order to ensure that the 
initialized state is complete.

Reviewers: José Armando García Sancio 
---
 core/src/main/scala/kafka/log/LogLoader.scala  |  7 +-
 .../main/scala/kafka/raft/KafkaMetadataLog.scala   | 43 ---
 .../scala/kafka/raft/KafkaMetadataLogTest.scala| 85 +-
 .../java/org/apache/kafka/snapshot/Snapshots.java  | 15 ++--
 .../org/apache/kafka/snapshot/SnapshotsTest.java   |  6 +-
 5 files changed, 132 insertions(+), 24 deletions(-)

diff --git a/core/src/main/scala/kafka/log/LogLoader.scala 
b/core/src/main/scala/kafka/log/LogLoader.scala
index 25ee89c72b..f8da67656f 100644
--- a/core/src/main/scala/kafka/log/LogLoader.scala
+++ b/core/src/main/scala/kafka/log/LogLoader.scala
@@ -19,7 +19,6 @@ package kafka.log
 
 import java.io.{File, IOException}
 import java.nio.file.{Files, NoSuchFileException}
-
 import kafka.common.LogSegmentOffsetOverflowException
 import kafka.log.UnifiedLog.{CleanedFileSuffix, DeletedFileSuffix, 
SwapFileSuffix, isIndexFile, isLogFile, offsetFromFile}
 import kafka.server.{LogDirFailureChannel, LogOffsetMetadata}
@@ -28,6 +27,7 @@ import kafka.utils.{CoreUtils, Logging, Scheduler}
 import org.apache.kafka.common.TopicPartition
 import org.apache.kafka.common.errors.InvalidOffsetException
 import org.apache.kafka.common.utils.Time
+import org.apache.kafka.snapshot.Snapshots
 
 import java.util.concurrent.{ConcurrentHashMap, ConcurrentMap}
 import scala.collection.{Set, mutable}
@@ -229,7 +229,10 @@ class LogLoader(
   if (!file.canRead)
 throw new IOException(s"Could not read file $file")
   val filename = file.getName
-  if (filename.endsWith(DeletedFileSuffix)) {
+
+  // Delete stray files marked for deletion, but skip KRaft snapshots.
+  // These are handled in the recovery logic in `KafkaMetadataLog`.
+  if (filename.endsWith(DeletedFileSuffix) && 
!filename.endsWith(Snapshots.DELETE_SUFFIX)) {
 debug(s"Deleting stray temporary file ${file.getAbsolutePath}")
 Files.deleteIfExists(file.toPath)
   } else if (filename.endsWith(CleanedFileSuffix)) {
diff --git a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala 
b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala
index d112f3b581..95d96b3399 100644
--- a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala
+++ b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala
@@ -16,7 +16,7 @@
  */
 package kafka.raft
 
-import kafka.log.{AppendOrigin, Defaults, UnifiedLog, LogConfig, 
LogOffsetSnapshot, SnapshotGenerated}
+import kafka.log.{AppendOrigin, Defaults, LogConfig, LogOffsetSnapshot, 
SnapshotGenerated, UnifiedLog}
 import kafka.server.KafkaConfig.{MetadataLogSegmentBytesProp, 
MetadataLogSegmentMinBytesProp}
 import kafka.server.{BrokerTopicStats, FetchHighWatermark, FetchLogEnd, 
KafkaConfig, LogDirFailureChannel, RequestLocal}
 import kafka.utils.{CoreUtils, Logging, Scheduler}
@@ -26,7 +26,7 @@ import org.apache.kafka.common.record.{ControlRecordUtils, 
MemoryRecords, Record
 import org.apache.kafka.common.utils.{BufferSupplier, Time}
 import org.apache.kafka.common.{KafkaException, TopicPartition, Uuid}
 import org.apache.kafka.raft.{Isolation, KafkaRaftClient, LogAppendInfo, 
LogFetchInfo, LogOffsetMetadata, OffsetAndEpoch, OffsetMetadata, ReplicatedLog, 
ValidOffsetAndEpoch}
-import org.apache.kafka.snapshot.{FileRawSnapshotReader, 
FileRawSnapshotWriter, RawSnapshotReader, RawSnapshotWriter, Snapshots}
+import org.apache.kafka.snapshot.{FileRawSnapshotReader, 
FileRawSnapshotWriter, RawSnapshotReader, RawSnapshotWriter, SnapshotPath, 
Snapshots}
 
 import java.io.File
 import java.nio.file.{Files, NoSuchFileException, Path}
@@ -546,7 +546,7 @@ case class MetadataLogConfig(logSegmentBytes: Int,
  fileDeleteDelayMs: Int,
  nodeId: Int)
 
-object KafkaMetadataLog {
+object KafkaMetadataLog extends Logging {
   def apply(
 topicPartition: TopicPartition,
 topicId: Uuid,
@@ -623,7 +623,9 @@ object KafkaMetadataLog {
   private def recoverSnapshots(
 log: UnifiedLog
   ): mutable.TreeMap[OffsetAndEpoch, Option[FileRawSnapshotReader]] = {
-val snapshots = mutable.TreeMap.e

[kafka] branch trunk updated: KAFKA-14240; Validate kraft snapshot state on startup (#12653)

2022-09-19 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 8c8b5366a6 KAFKA-14240; Validate kraft snapshot state on startup 
(#12653)
8c8b5366a6 is described below

commit 8c8b5366a69a14def32bffe01faeeea448d0c9d6
Author: Jason Gustafson 
AuthorDate: Mon Sep 19 11:52:48 2022 -0700

KAFKA-14240; Validate kraft snapshot state on startup (#12653)

We should prevent the metadata log from initializing in a known bad state. 
If the log start offset of the first segment is greater than 0, then must be a 
snapshot an offset greater than or equal to it order to ensure that the 
initialized state is complete.

Reviewers: José Armando García Sancio 
---
 core/src/main/scala/kafka/log/LogLoader.scala  |  7 +-
 .../main/scala/kafka/raft/KafkaMetadataLog.scala   | 43 ---
 .../scala/kafka/raft/KafkaMetadataLogTest.scala| 85 +-
 .../java/org/apache/kafka/snapshot/Snapshots.java  | 15 ++--
 .../org/apache/kafka/snapshot/SnapshotsTest.java   |  6 +-
 5 files changed, 132 insertions(+), 24 deletions(-)

diff --git a/core/src/main/scala/kafka/log/LogLoader.scala 
b/core/src/main/scala/kafka/log/LogLoader.scala
index 25ee89c72b..f8da67656f 100644
--- a/core/src/main/scala/kafka/log/LogLoader.scala
+++ b/core/src/main/scala/kafka/log/LogLoader.scala
@@ -19,7 +19,6 @@ package kafka.log
 
 import java.io.{File, IOException}
 import java.nio.file.{Files, NoSuchFileException}
-
 import kafka.common.LogSegmentOffsetOverflowException
 import kafka.log.UnifiedLog.{CleanedFileSuffix, DeletedFileSuffix, 
SwapFileSuffix, isIndexFile, isLogFile, offsetFromFile}
 import kafka.server.{LogDirFailureChannel, LogOffsetMetadata}
@@ -28,6 +27,7 @@ import kafka.utils.{CoreUtils, Logging, Scheduler}
 import org.apache.kafka.common.TopicPartition
 import org.apache.kafka.common.errors.InvalidOffsetException
 import org.apache.kafka.common.utils.Time
+import org.apache.kafka.snapshot.Snapshots
 
 import java.util.concurrent.{ConcurrentHashMap, ConcurrentMap}
 import scala.collection.{Set, mutable}
@@ -229,7 +229,10 @@ class LogLoader(
   if (!file.canRead)
 throw new IOException(s"Could not read file $file")
   val filename = file.getName
-  if (filename.endsWith(DeletedFileSuffix)) {
+
+  // Delete stray files marked for deletion, but skip KRaft snapshots.
+  // These are handled in the recovery logic in `KafkaMetadataLog`.
+  if (filename.endsWith(DeletedFileSuffix) && 
!filename.endsWith(Snapshots.DELETE_SUFFIX)) {
 debug(s"Deleting stray temporary file ${file.getAbsolutePath}")
 Files.deleteIfExists(file.toPath)
   } else if (filename.endsWith(CleanedFileSuffix)) {
diff --git a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala 
b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala
index d112f3b581..95d96b3399 100644
--- a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala
+++ b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala
@@ -16,7 +16,7 @@
  */
 package kafka.raft
 
-import kafka.log.{AppendOrigin, Defaults, UnifiedLog, LogConfig, 
LogOffsetSnapshot, SnapshotGenerated}
+import kafka.log.{AppendOrigin, Defaults, LogConfig, LogOffsetSnapshot, 
SnapshotGenerated, UnifiedLog}
 import kafka.server.KafkaConfig.{MetadataLogSegmentBytesProp, 
MetadataLogSegmentMinBytesProp}
 import kafka.server.{BrokerTopicStats, FetchHighWatermark, FetchLogEnd, 
KafkaConfig, LogDirFailureChannel, RequestLocal}
 import kafka.utils.{CoreUtils, Logging, Scheduler}
@@ -26,7 +26,7 @@ import org.apache.kafka.common.record.{ControlRecordUtils, 
MemoryRecords, Record
 import org.apache.kafka.common.utils.{BufferSupplier, Time}
 import org.apache.kafka.common.{KafkaException, TopicPartition, Uuid}
 import org.apache.kafka.raft.{Isolation, KafkaRaftClient, LogAppendInfo, 
LogFetchInfo, LogOffsetMetadata, OffsetAndEpoch, OffsetMetadata, ReplicatedLog, 
ValidOffsetAndEpoch}
-import org.apache.kafka.snapshot.{FileRawSnapshotReader, 
FileRawSnapshotWriter, RawSnapshotReader, RawSnapshotWriter, Snapshots}
+import org.apache.kafka.snapshot.{FileRawSnapshotReader, 
FileRawSnapshotWriter, RawSnapshotReader, RawSnapshotWriter, SnapshotPath, 
Snapshots}
 
 import java.io.File
 import java.nio.file.{Files, NoSuchFileException, Path}
@@ -546,7 +546,7 @@ case class MetadataLogConfig(logSegmentBytes: Int,
  fileDeleteDelayMs: Int,
  nodeId: Int)
 
-object KafkaMetadataLog {
+object KafkaMetadataLog extends Logging {
   def apply(
 topicPartition: TopicPartition,
 topicId: Uuid,
@@ -623,7 +623,9 @@ object KafkaMetadataLog {
   private def recoverSnapshots(
 log: UnifiedLog
   ): mutable.TreeMap[OffsetAndEpoch, Option[FileRawSnapshotReader]] = {
-val snapshots = mutable.TreeMap.e

[kafka] branch trunk updated (b166ac43cb -> a8fcbcc08f)

2022-09-18 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a change to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


from b166ac43cb KAFKA-14238;  KRaft metadata log should not delete segment 
past the latest snapshot (#12655)
 add a8fcbcc08f MINOR: Set display granularity in gradle test logging 
(#12657)

No new revisions were added by this update.

Summary of changes:
 build.gradle | 3 +++
 1 file changed, 3 insertions(+)



[kafka] branch 3.3 updated: KAFKA-14238; KRaft metadata log should not delete segment past the latest snapshot (#12655)

2022-09-17 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch 3.3
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.3 by this push:
 new 74c4bbfaf9 KAFKA-14238;  KRaft metadata log should not delete segment 
past the latest snapshot (#12655)
74c4bbfaf9 is described below

commit 74c4bbfaf9a7da16baf0219df2613901aa706639
Author: José Armando García Sancio 
AuthorDate: Sat Sep 17 06:30:50 2022 -0700

KAFKA-14238;  KRaft metadata log should not delete segment past the latest 
snapshot (#12655)

Disable segment deletion based on size and time by setting the KRaft 
metadata log's `RetentionMsProp` and `RetentionBytesProp` to `-1`. This will 
cause `UnifiedLog.deleteRetentionMsBreachedSegments` and 
`UnifiedLog.deleteRetentionSizeBreachedSegments` to short circuit instead of 
deleting segments.

Without this changes the included test would fail. This happens because 
`deleteRetentionMsBreachedSegments` is able to delete past the 
`logStartOffset`. Deleting past the `logStartOffset` would violate the 
invariant that if the `logStartOffset` is greater than 0 then there is a 
snapshot with an end offset greater than or equal to the log start offset.

Reviewers: Luke Chen , Jason Gustafson 

---
 .../main/scala/kafka/raft/KafkaMetadataLog.scala   | 26 ---
 .../scala/kafka/raft/KafkaMetadataLogTest.scala| 54 +-
 2 files changed, 72 insertions(+), 8 deletions(-)

diff --git a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala 
b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala
index 83b8bee444..d112f3b581 100644
--- a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala
+++ b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala
@@ -382,7 +382,7 @@ final class KafkaMetadataLog private (
   }
 
   /**
-   * Perform cleaning of old snapshots and log segments based on size.
+   * Perform cleaning of old snapshots and log segments based on size and time.
*
* If our configured retention size has been violated, we perform cleaning 
as follows:
*
@@ -556,15 +556,29 @@ object KafkaMetadataLog {
 config: MetadataLogConfig
   ): KafkaMetadataLog = {
 val props = new Properties()
-props.put(LogConfig.MaxMessageBytesProp, 
config.maxBatchSizeInBytes.toString)
-props.put(LogConfig.SegmentBytesProp, Int.box(config.logSegmentBytes))
-props.put(LogConfig.SegmentMsProp, Long.box(config.logSegmentMillis))
-props.put(LogConfig.FileDeleteDelayMsProp, 
Int.box(Defaults.FileDeleteDelayMs))
+props.setProperty(LogConfig.MaxMessageBytesProp, 
config.maxBatchSizeInBytes.toString)
+props.setProperty(LogConfig.SegmentBytesProp, 
config.logSegmentBytes.toString)
+props.setProperty(LogConfig.SegmentMsProp, 
config.logSegmentMillis.toString)
+props.setProperty(LogConfig.FileDeleteDelayMsProp, 
Defaults.FileDeleteDelayMs.toString)
+
+// Disable time and byte retention when deleting segments
+props.setProperty(LogConfig.RetentionMsProp, "-1")
+props.setProperty(LogConfig.RetentionBytesProp, "-1")
 LogConfig.validateValues(props)
 val defaultLogConfig = LogConfig(props)
 
 if (config.logSegmentBytes < config.logSegmentMinBytes) {
-  throw new InvalidConfigurationException(s"Cannot set 
$MetadataLogSegmentBytesProp below ${config.logSegmentMinBytes}")
+  throw new InvalidConfigurationException(
+s"Cannot set $MetadataLogSegmentBytesProp below 
${config.logSegmentMinBytes}: ${config.logSegmentBytes}"
+  )
+} else if (defaultLogConfig.retentionMs >= 0) {
+  throw new InvalidConfigurationException(
+s"Cannot set ${LogConfig.RetentionMsProp} above -1: 
${defaultLogConfig.retentionMs}."
+  )
+} else if (defaultLogConfig.retentionSize >= 0) {
+  throw new InvalidConfigurationException(
+s"Cannot set ${LogConfig.RetentionBytesProp} above -1: 
${defaultLogConfig.retentionSize}."
+  )
 }
 
 val log = UnifiedLog(
diff --git a/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala 
b/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala
index bf37719f73..49b1206606 100644
--- a/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala
+++ b/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala
@@ -66,13 +66,13 @@ final class KafkaMetadataLogTest {
 props.put(MetadataLogSegmentMillisProp, Int.box(10 * 1024))
 assertThrows(classOf[InvalidConfigurationException], () => {
   val kafkaConfig = KafkaConfig.fromProps(props)
-  val metadataConfig = MetadataLogConfig.apply(kafkaConfig, 
KafkaRaftClient.MAX_BATCH_SIZE_BYTES, KafkaRaftClient.MAX_FETCH_SIZE_BYTES)
+  val metadataConfig = MetadataLogConfig(kafkaConfig, 
KafkaRaftClient.MAX_BATCH_SIZE_BYTES, KafkaRaftClient.MAX_FETCH_SIZE_BYTES)
   buildMetadataLog(tempDir, mockTime, metadataConfig)
 })
 

[kafka] branch trunk updated: KAFKA-14238; KRaft metadata log should not delete segment past the latest snapshot (#12655)

2022-09-17 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new b166ac43cb KAFKA-14238;  KRaft metadata log should not delete segment 
past the latest snapshot (#12655)
b166ac43cb is described below

commit b166ac43cb646d9589623abfe283874d59ce923b
Author: José Armando García Sancio 
AuthorDate: Sat Sep 17 06:30:50 2022 -0700

KAFKA-14238;  KRaft metadata log should not delete segment past the latest 
snapshot (#12655)

Disable segment deletion based on size and time by setting the KRaft 
metadata log's `RetentionMsProp` and `RetentionBytesProp` to `-1`. This will 
cause `UnifiedLog.deleteRetentionMsBreachedSegments` and 
`UnifiedLog.deleteRetentionSizeBreachedSegments` to short circuit instead of 
deleting segments.

Without this changes the included test would fail. This happens because 
`deleteRetentionMsBreachedSegments` is able to delete past the 
`logStartOffset`. Deleting past the `logStartOffset` would violate the 
invariant that if the `logStartOffset` is greater than 0 then there is a 
snapshot with an end offset greater than or equal to the log start offset.

Reviewers: Luke Chen , Jason Gustafson 

---
 .../main/scala/kafka/raft/KafkaMetadataLog.scala   | 26 ---
 .../scala/kafka/raft/KafkaMetadataLogTest.scala| 54 +-
 2 files changed, 72 insertions(+), 8 deletions(-)

diff --git a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala 
b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala
index 83b8bee444..d112f3b581 100644
--- a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala
+++ b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala
@@ -382,7 +382,7 @@ final class KafkaMetadataLog private (
   }
 
   /**
-   * Perform cleaning of old snapshots and log segments based on size.
+   * Perform cleaning of old snapshots and log segments based on size and time.
*
* If our configured retention size has been violated, we perform cleaning 
as follows:
*
@@ -556,15 +556,29 @@ object KafkaMetadataLog {
 config: MetadataLogConfig
   ): KafkaMetadataLog = {
 val props = new Properties()
-props.put(LogConfig.MaxMessageBytesProp, 
config.maxBatchSizeInBytes.toString)
-props.put(LogConfig.SegmentBytesProp, Int.box(config.logSegmentBytes))
-props.put(LogConfig.SegmentMsProp, Long.box(config.logSegmentMillis))
-props.put(LogConfig.FileDeleteDelayMsProp, 
Int.box(Defaults.FileDeleteDelayMs))
+props.setProperty(LogConfig.MaxMessageBytesProp, 
config.maxBatchSizeInBytes.toString)
+props.setProperty(LogConfig.SegmentBytesProp, 
config.logSegmentBytes.toString)
+props.setProperty(LogConfig.SegmentMsProp, 
config.logSegmentMillis.toString)
+props.setProperty(LogConfig.FileDeleteDelayMsProp, 
Defaults.FileDeleteDelayMs.toString)
+
+// Disable time and byte retention when deleting segments
+props.setProperty(LogConfig.RetentionMsProp, "-1")
+props.setProperty(LogConfig.RetentionBytesProp, "-1")
 LogConfig.validateValues(props)
 val defaultLogConfig = LogConfig(props)
 
 if (config.logSegmentBytes < config.logSegmentMinBytes) {
-  throw new InvalidConfigurationException(s"Cannot set 
$MetadataLogSegmentBytesProp below ${config.logSegmentMinBytes}")
+  throw new InvalidConfigurationException(
+s"Cannot set $MetadataLogSegmentBytesProp below 
${config.logSegmentMinBytes}: ${config.logSegmentBytes}"
+  )
+} else if (defaultLogConfig.retentionMs >= 0) {
+  throw new InvalidConfigurationException(
+s"Cannot set ${LogConfig.RetentionMsProp} above -1: 
${defaultLogConfig.retentionMs}."
+  )
+} else if (defaultLogConfig.retentionSize >= 0) {
+  throw new InvalidConfigurationException(
+s"Cannot set ${LogConfig.RetentionBytesProp} above -1: 
${defaultLogConfig.retentionSize}."
+  )
 }
 
 val log = UnifiedLog(
diff --git a/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala 
b/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala
index 94bf45391e..a8e7b2c324 100644
--- a/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala
+++ b/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala
@@ -66,13 +66,13 @@ final class KafkaMetadataLogTest {
 props.put(MetadataLogSegmentMillisProp, Int.box(10 * 1024))
 assertThrows(classOf[InvalidConfigurationException], () => {
   val kafkaConfig = KafkaConfig.fromProps(props)
-  val metadataConfig = MetadataLogConfig.apply(kafkaConfig, 
KafkaRaftClient.MAX_BATCH_SIZE_BYTES, KafkaRaftClient.MAX_FETCH_SIZE_BYTES)
+  val metadataConfig = MetadataLogConfig(kafkaConfig, 
KafkaRaftClient.MAX_BATCH_SIZE_BYTES, KafkaRaftClient.MAX_FETCH_SIZE_BYTES)
   buildMetadataLog(tempDir, mockTime, metadataConfig

[kafka] branch trunk updated (86645cb40a7 -> 5f01fed2060)

2022-09-13 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a change to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


from 86645cb40a7 KAFKA-14073; Log the reason for snapshot (#12414)
 add 5f01fed2060 MINOR: Small cleanups in FetcherTest following KAFKA-14196 
(#12629)

No new revisions were added by this update.

Summary of changes:
 .../java/org/apache/kafka/clients/consumer/internals/FetcherTest.java | 4 +---
 1 file changed, 1 insertion(+), 3 deletions(-)



[kafka] branch 3.2 updated: KAFKA-14196; Do not continue fetching partitions awaiting auto-commit prior to revocation (#12603)

2022-09-12 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch 3.2
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.2 by this push:
 new 56baf6448f0 KAFKA-14196; Do not continue fetching partitions awaiting 
auto-commit prior to revocation (#12603)
56baf6448f0 is described below

commit 56baf6448f0540d30e82c858e8e242094e95df7f
Author: Philip Nee 
AuthorDate: Mon Sep 12 21:02:13 2022 -0700

KAFKA-14196; Do not continue fetching partitions awaiting auto-commit prior 
to revocation (#12603)

When auto-commit is enabled with the "eager" rebalance strategy, the 
consumer will commit all offsets prior to revocation. Following recent changes, 
this offset commit is done asynchronously, which means there is an opportunity 
for fetches to continue returning data to the application. When this happens, 
the progress is lost following revocation, which results in duplicate 
consumption. This patch fixes the problem by adding a flag in 
`SubscriptionState` to ensure that partitions which [...]

Reviewers: Luke Chen , Jason Gustafson 

---
 .../consumer/internals/ConsumerCoordinator.java| 17 ++
 .../consumer/internals/SubscriptionState.java  | 12 +++-
 .../clients/consumer/internals/FetcherTest.java| 70 ++
 .../consumer/internals/SubscriptionStateTest.java  | 10 
 4 files changed, 108 insertions(+), 1 deletion(-)

diff --git 
a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
 
b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
index 9838e7dc8fe..a614504704f 100644
--- 
a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
+++ 
b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
@@ -759,6 +759,7 @@ public final class ConsumerCoordinator extends 
AbstractCoordinator {
 // async commit offsets prior to rebalance if auto-commit enabled
 // and there is no in-flight offset commit request
 if (autoCommitEnabled && autoCommitOffsetRequestFuture == null) {
+maybeMarkPartitionsPendingRevocation();
 autoCommitOffsetRequestFuture = maybeAutoCommitOffsetsAsync();
 }
 
@@ -859,6 +860,22 @@ public final class ConsumerCoordinator extends 
AbstractCoordinator {
 return true;
 }
 
+private void maybeMarkPartitionsPendingRevocation() {
+if (protocol != RebalanceProtocol.EAGER) {
+return;
+}
+
+// When asynchronously committing offsets prior to the revocation of a 
set of partitions, there will be a
+// window of time between when the offset commit is sent and when it 
returns and revocation completes. It is
+// possible for pending fetches for these partitions to return during 
this time, which means the application's
+// position may get ahead of the committed position prior to 
revocation. This can cause duplicate consumption.
+// To prevent this, we mark the partitions as "pending revocation," 
which stops the Fetcher from sending new
+// fetches or returning data from previous fetches to the user.
+Set partitions = subscriptions.assignedPartitions();
+log.debug("Marking assigned partitions pending for revocation: {}", 
partitions);
+subscriptions.markPendingRevocation(partitions);
+}
+
 @Override
 public void onLeavePrepare() {
 // Save the current Generation, as the hb thread can change it at any 
time
diff --git 
a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java
 
b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java
index 416468d945f..7f8fe8479e9 100644
--- 
a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java
+++ 
b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java
@@ -738,6 +738,10 @@ public class SubscriptionState {
 assignedState(tp).pause();
 }
 
+public synchronized void markPendingRevocation(Set tps) {
+tps.forEach(tp -> assignedState(tp).markPendingRevocation());
+}
+
 public synchronized void resume(TopicPartition tp) {
 assignedState(tp).resume();
 }
@@ -769,6 +773,7 @@ public class SubscriptionState {
 private Long logStartOffset; // the log start offset
 private Long lastStableOffset;
 private boolean paused;  // whether this partition has been paused by 
the user
+private boolean pendingRevocation;
 private OffsetResetStrategy resetStrategy;  // the strategy to use if 
the offset needs resetting
 private Long nextRetryTimeMs;
 private Integer preferredReadReplica;
@@ -777,6 +782,7 @@ public class SubscriptionState

[kafka] branch 3.3 updated: KAFKA-14196; Do not continue fetching partitions awaiting auto-commit prior to revocation (#12603)

2022-09-12 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch 3.3
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.3 by this push:
 new 14df199af04 KAFKA-14196; Do not continue fetching partitions awaiting 
auto-commit prior to revocation (#12603)
14df199af04 is described below

commit 14df199af0433f2718b71b976aab3b3a7002d4b8
Author: Philip Nee 
AuthorDate: Mon Sep 12 21:02:13 2022 -0700

KAFKA-14196; Do not continue fetching partitions awaiting auto-commit prior 
to revocation (#12603)

When auto-commit is enabled with the "eager" rebalance strategy, the 
consumer will commit all offsets prior to revocation. Following recent changes, 
this offset commit is done asynchronously, which means there is an opportunity 
for fetches to continue returning data to the application. When this happens, 
the progress is lost following revocation, which results in duplicate 
consumption. This patch fixes the problem by adding a flag in 
`SubscriptionState` to ensure that partitions which [...]

Reviewers: Luke Chen , Jason Gustafson 

---
 .../consumer/internals/ConsumerCoordinator.java| 17 ++
 .../consumer/internals/SubscriptionState.java  | 12 +++-
 .../clients/consumer/internals/FetcherTest.java| 70 ++
 .../consumer/internals/SubscriptionStateTest.java  | 10 
 4 files changed, 108 insertions(+), 1 deletion(-)

diff --git 
a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
 
b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
index 5228c60e0fb..00af72a807b 100644
--- 
a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
+++ 
b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
@@ -759,6 +759,7 @@ public final class ConsumerCoordinator extends 
AbstractCoordinator {
 // async commit offsets prior to rebalance if auto-commit enabled
 // and there is no in-flight offset commit request
 if (autoCommitEnabled && autoCommitOffsetRequestFuture == null) {
+maybeMarkPartitionsPendingRevocation();
 autoCommitOffsetRequestFuture = maybeAutoCommitOffsetsAsync();
 }
 
@@ -859,6 +860,22 @@ public final class ConsumerCoordinator extends 
AbstractCoordinator {
 return true;
 }
 
+private void maybeMarkPartitionsPendingRevocation() {
+if (protocol != RebalanceProtocol.EAGER) {
+return;
+}
+
+// When asynchronously committing offsets prior to the revocation of a 
set of partitions, there will be a
+// window of time between when the offset commit is sent and when it 
returns and revocation completes. It is
+// possible for pending fetches for these partitions to return during 
this time, which means the application's
+// position may get ahead of the committed position prior to 
revocation. This can cause duplicate consumption.
+// To prevent this, we mark the partitions as "pending revocation," 
which stops the Fetcher from sending new
+// fetches or returning data from previous fetches to the user.
+Set partitions = subscriptions.assignedPartitions();
+log.debug("Marking assigned partitions pending for revocation: {}", 
partitions);
+subscriptions.markPendingRevocation(partitions);
+}
+
 @Override
 public void onLeavePrepare() {
 // Save the current Generation, as the hb thread can change it at any 
time
diff --git 
a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java
 
b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java
index 0225822a194..03b664e1392 100644
--- 
a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java
+++ 
b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java
@@ -738,6 +738,10 @@ public class SubscriptionState {
 assignedState(tp).pause();
 }
 
+public synchronized void markPendingRevocation(Set tps) {
+tps.forEach(tp -> assignedState(tp).markPendingRevocation());
+}
+
 public synchronized void resume(TopicPartition tp) {
 assignedState(tp).resume();
 }
@@ -769,6 +773,7 @@ public class SubscriptionState {
 private Long logStartOffset; // the log start offset
 private Long lastStableOffset;
 private boolean paused;  // whether this partition has been paused by 
the user
+private boolean pendingRevocation;
 private OffsetResetStrategy resetStrategy;  // the strategy to use if 
the offset needs resetting
 private Long nextRetryTimeMs;
 private Integer preferredReadReplica;
@@ -777,6 +782,7 @@ public class SubscriptionState

[kafka] branch trunk updated (b9774c0b025 -> 536cdf692fb)

2022-09-12 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a change to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


from b9774c0b025 KAFKA-14215; Ensure forwarded requests are applied to 
broker request quota (#12624)
 add 536cdf692fb KAFKA-14196; Do not continue fetching partitions awaiting 
auto-commit prior to revocation (#12603)

No new revisions were added by this update.

Summary of changes:
 .../consumer/internals/ConsumerCoordinator.java| 17 ++
 .../consumer/internals/SubscriptionState.java  | 12 +++-
 .../clients/consumer/internals/FetcherTest.java| 70 ++
 .../consumer/internals/SubscriptionStateTest.java  | 10 
 4 files changed, 108 insertions(+), 1 deletion(-)



[kafka] branch 3.3 updated: KAFKA-14215; Ensure forwarded requests are applied to broker request quota (#12624)

2022-09-12 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch 3.3
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.3 by this push:
 new 85fc2671001 KAFKA-14215; Ensure forwarded requests are applied to 
broker request quota (#12624)
85fc2671001 is described below

commit 85fc267100161441342eeaa0a8fcb8625b300148
Author: Jason Gustafson 
AuthorDate: Mon Sep 12 20:50:33 2022 -0700

KAFKA-14215; Ensure forwarded requests are applied to broker request quota 
(#12624)

Currently forwarded requests are not applied to any quotas on either the 
controller or the broker. The controller-side throttling requires the 
controller to apply the quota changes from the log to the quota managers, which 
will be done separately. In this patch, we change the response logic on the 
broker side to also apply the broker's request quota. The enforced throttle 
time is the maximum of the throttle returned from the controller (which is 0 
until we fix the aforementioned issue [...]

Reviewers: David Arthur 
---
 .../kafka/common/requests/AbstractResponse.java| 12 +
 .../common/requests/AddOffsetsToTxnResponse.java   |  5 ++
 .../requests/AddPartitionsToTxnResponse.java   |  5 ++
 .../requests/AllocateProducerIdsResponse.java  |  5 ++
 .../common/requests/AlterClientQuotasResponse.java |  5 ++
 .../common/requests/AlterConfigsResponse.java  |  5 ++
 .../AlterPartitionReassignmentsResponse.java   |  5 ++
 .../common/requests/AlterPartitionResponse.java|  5 ++
 .../requests/AlterReplicaLogDirsResponse.java  |  5 ++
 .../AlterUserScramCredentialsResponse.java |  5 ++
 .../kafka/common/requests/ApiVersionsResponse.java |  5 ++
 .../common/requests/BeginQuorumEpochResponse.java  |  5 ++
 .../common/requests/BrokerHeartbeatResponse.java   |  5 ++
 .../requests/BrokerRegistrationResponse.java   |  5 ++
 .../requests/ControlledShutdownResponse.java   |  5 ++
 .../kafka/common/requests/CreateAclsResponse.java  |  5 ++
 .../requests/CreateDelegationTokenResponse.java|  5 ++
 .../common/requests/CreatePartitionsResponse.java  |  5 ++
 .../common/requests/CreateTopicsResponse.java  |  5 ++
 .../kafka/common/requests/DeleteAclsResponse.java  |  5 ++
 .../common/requests/DeleteGroupsResponse.java  |  5 ++
 .../common/requests/DeleteRecordsResponse.java |  5 ++
 .../common/requests/DeleteTopicsResponse.java  |  5 ++
 .../common/requests/DescribeAclsResponse.java  |  5 ++
 .../requests/DescribeClientQuotasResponse.java |  5 ++
 .../common/requests/DescribeClusterResponse.java   |  5 ++
 .../common/requests/DescribeConfigsResponse.java   |  5 ++
 .../requests/DescribeDelegationTokenResponse.java  |  5 ++
 .../common/requests/DescribeGroupsResponse.java|  5 ++
 .../common/requests/DescribeLogDirsResponse.java   |  5 ++
 .../common/requests/DescribeProducersResponse.java |  5 ++
 .../common/requests/DescribeQuorumResponse.java|  5 ++
 .../requests/DescribeTransactionsResponse.java |  6 +++
 .../DescribeUserScramCredentialsResponse.java  |  5 ++
 .../common/requests/ElectLeadersResponse.java  |  5 ++
 .../common/requests/EndQuorumEpochResponse.java|  5 ++
 .../kafka/common/requests/EndTxnResponse.java  |  4 ++
 .../kafka/common/requests/EnvelopeResponse.java|  5 ++
 .../requests/ExpireDelegationTokenResponse.java|  5 ++
 .../kafka/common/requests/FetchResponse.java   |  5 ++
 .../common/requests/FetchSnapshotResponse.java |  5 ++
 .../common/requests/FindCoordinatorResponse.java   |  5 ++
 .../kafka/common/requests/HeartbeatResponse.java   |  5 ++
 .../requests/IncrementalAlterConfigsResponse.java  |  5 ++
 .../common/requests/InitProducerIdResponse.java|  5 ++
 .../kafka/common/requests/JoinGroupResponse.java   |  5 ++
 .../common/requests/LeaderAndIsrResponse.java  |  5 ++
 .../kafka/common/requests/LeaveGroupResponse.java  |  5 ++
 .../kafka/common/requests/ListGroupsResponse.java  |  5 ++
 .../kafka/common/requests/ListOffsetsResponse.java |  5 ++
 .../ListPartitionReassignmentsResponse.java|  5 ++
 .../common/requests/ListTransactionsResponse.java  |  5 ++
 .../kafka/common/requests/MetadataResponse.java|  5 ++
 .../common/requests/OffsetCommitResponse.java  |  5 ++
 .../common/requests/OffsetDeleteResponse.java  |  5 ++
 .../kafka/common/requests/OffsetFetchResponse.java |  5 ++
 .../requests/OffsetsForLeaderEpochResponse.java|  5 ++
 .../kafka/common/requests/ProduceResponse.java |  5 ++
 .../requests/RenewDelegationTokenResponse.java |  5 ++
 .../common/requests/SaslAuthenticateResponse.java  |  5 ++
 .../common/requests/SaslHandshakeResponse.java |  5 ++
 .../kafka/common/requests/StopReplicaResponse.java |  5 ++
 .../kafka/common/requests/SyncGroupResponse.java   |  5 ++
 .../common/requests/TxnOffsetCommitResponse.java   |  5 ++
 .../common/requests

[kafka] branch trunk updated (d4fc3186b40 -> b9774c0b025)

2022-09-12 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a change to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


from d4fc3186b40 MINOR: Replace usage of File.createTempFile() with 
TestUtils.tempFile() (#12591)
 add b9774c0b025 KAFKA-14215; Ensure forwarded requests are applied to 
broker request quota (#12624)

No new revisions were added by this update.

Summary of changes:
 .../kafka/common/requests/AbstractResponse.java| 12 +
 .../common/requests/AddOffsetsToTxnResponse.java   |  5 ++
 .../requests/AddPartitionsToTxnResponse.java   |  5 ++
 .../requests/AllocateProducerIdsResponse.java  |  5 ++
 .../common/requests/AlterClientQuotasResponse.java |  5 ++
 .../common/requests/AlterConfigsResponse.java  |  5 ++
 .../AlterPartitionReassignmentsResponse.java   |  5 ++
 .../common/requests/AlterPartitionResponse.java|  5 ++
 .../requests/AlterReplicaLogDirsResponse.java  |  5 ++
 .../AlterUserScramCredentialsResponse.java |  5 ++
 .../kafka/common/requests/ApiVersionsResponse.java |  5 ++
 .../common/requests/BeginQuorumEpochResponse.java  |  5 ++
 .../common/requests/BrokerHeartbeatResponse.java   |  5 ++
 .../requests/BrokerRegistrationResponse.java   |  5 ++
 .../requests/ControlledShutdownResponse.java   |  5 ++
 .../kafka/common/requests/CreateAclsResponse.java  |  5 ++
 .../requests/CreateDelegationTokenResponse.java|  5 ++
 .../common/requests/CreatePartitionsResponse.java  |  5 ++
 .../common/requests/CreateTopicsResponse.java  |  5 ++
 .../kafka/common/requests/DeleteAclsResponse.java  |  5 ++
 .../common/requests/DeleteGroupsResponse.java  |  5 ++
 .../common/requests/DeleteRecordsResponse.java |  5 ++
 .../common/requests/DeleteTopicsResponse.java  |  5 ++
 .../common/requests/DescribeAclsResponse.java  |  5 ++
 .../requests/DescribeClientQuotasResponse.java |  5 ++
 .../common/requests/DescribeClusterResponse.java   |  5 ++
 .../common/requests/DescribeConfigsResponse.java   |  5 ++
 .../requests/DescribeDelegationTokenResponse.java  |  5 ++
 .../common/requests/DescribeGroupsResponse.java|  5 ++
 .../common/requests/DescribeLogDirsResponse.java   |  5 ++
 .../common/requests/DescribeProducersResponse.java |  5 ++
 .../common/requests/DescribeQuorumResponse.java|  5 ++
 .../requests/DescribeTransactionsResponse.java |  6 +++
 .../DescribeUserScramCredentialsResponse.java  |  5 ++
 .../common/requests/ElectLeadersResponse.java  |  5 ++
 .../common/requests/EndQuorumEpochResponse.java|  5 ++
 .../kafka/common/requests/EndTxnResponse.java  |  4 ++
 .../kafka/common/requests/EnvelopeResponse.java|  5 ++
 .../requests/ExpireDelegationTokenResponse.java|  5 ++
 .../kafka/common/requests/FetchResponse.java   |  5 ++
 .../common/requests/FetchSnapshotResponse.java |  5 ++
 .../common/requests/FindCoordinatorResponse.java   |  5 ++
 .../kafka/common/requests/HeartbeatResponse.java   |  5 ++
 .../requests/IncrementalAlterConfigsResponse.java  |  5 ++
 .../common/requests/InitProducerIdResponse.java|  5 ++
 .../kafka/common/requests/JoinGroupResponse.java   |  5 ++
 .../common/requests/LeaderAndIsrResponse.java  |  5 ++
 .../kafka/common/requests/LeaveGroupResponse.java  |  5 ++
 .../kafka/common/requests/ListGroupsResponse.java  |  5 ++
 .../kafka/common/requests/ListOffsetsResponse.java |  5 ++
 .../ListPartitionReassignmentsResponse.java|  5 ++
 .../common/requests/ListTransactionsResponse.java  |  5 ++
 .../kafka/common/requests/MetadataResponse.java|  5 ++
 .../common/requests/OffsetCommitResponse.java  |  5 ++
 .../common/requests/OffsetDeleteResponse.java  |  5 ++
 .../kafka/common/requests/OffsetFetchResponse.java |  5 ++
 .../requests/OffsetsForLeaderEpochResponse.java|  5 ++
 .../kafka/common/requests/ProduceResponse.java |  5 ++
 .../requests/RenewDelegationTokenResponse.java |  5 ++
 .../common/requests/SaslAuthenticateResponse.java  |  5 ++
 .../common/requests/SaslHandshakeResponse.java |  5 ++
 .../kafka/common/requests/StopReplicaResponse.java |  5 ++
 .../kafka/common/requests/SyncGroupResponse.java   |  5 ++
 .../common/requests/TxnOffsetCommitResponse.java   |  5 ++
 .../common/requests/UnregisterBrokerResponse.java  |  5 ++
 .../common/requests/UpdateFeaturesResponse.java|  5 ++
 .../common/requests/UpdateMetadataResponse.java|  5 ++
 .../apache/kafka/common/requests/VoteResponse.java |  5 ++
 .../common/requests/WriteTxnMarkersResponse.java   |  5 ++
 .../scala/kafka/server/RequestHandlerHelper.scala  | 11 +++--
 .../scala/unit/kafka/server/KafkaApisTest.scala| 55 +-
 71 files changed, 412 insertions(+), 6 deletions(-)



[kafka] branch 3.3 updated: KAFKA-14201; Consumer should not send group instance ID if committing with empty member ID (#12599)

2022-09-08 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch 3.3
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.3 by this push:
 new 7131724819 KAFKA-14201; Consumer should not send group instance ID if 
committing with empty member ID (#12599)
7131724819 is described below

commit 7131724819d35ee08ff84a4cb9b8ca88bacb1311
Author: David Jacot 
AuthorDate: Fri Sep 9 00:05:40 2022 +0200

KAFKA-14201; Consumer should not send group instance ID if committing with 
empty member ID (#12599)

The consumer group instance ID is used to support a notion of "static" 
consumer groups. The idea is to be able to identify the same group instance 
across restarts so that a rebalance is not needed. However, if the user sets 
`group.instance.id` in the consumer configuration, but uses "simple" assignment 
with `assign()`, then the instance ID nevertheless is sent in the OffsetCommit 
request to the coordinator. This may result in a surprising UNKNOWN_MEMBER_ID 
error.

This PR fixes the issue on the client side by not setting the group 
instance id if the member id is empty (no generation).

Reviewers: Jason Gustafson 
---
 .../consumer/internals/ConsumerCoordinator.java|  5 -
 .../internals/ConsumerCoordinatorTest.java | 26 ++
 2 files changed, 30 insertions(+), 1 deletion(-)

diff --git 
a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
 
b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
index 9838e7dc8f..5228c60e0f 100644
--- 
a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
+++ 
b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
@@ -1272,8 +1272,10 @@ public final class ConsumerCoordinator extends 
AbstractCoordinator {
 }
 
 final Generation generation;
+final String groupInstanceId;
 if (subscriptions.hasAutoAssignedPartitions()) {
 generation = generationIfStable();
+groupInstanceId = rebalanceConfig.groupInstanceId.orElse(null);
 // if the generation is null, we are not part of an active group 
(and we expect to be).
 // the only thing we can do is fail the commit and let the user 
rejoin the group in poll().
 if (generation == null) {
@@ -1293,6 +1295,7 @@ public final class ConsumerCoordinator extends 
AbstractCoordinator {
 }
 } else {
 generation = Generation.NO_GENERATION;
+groupInstanceId = null;
 }
 
 OffsetCommitRequest.Builder builder = new OffsetCommitRequest.Builder(
@@ -1300,7 +1303,7 @@ public final class ConsumerCoordinator extends 
AbstractCoordinator {
 .setGroupId(this.rebalanceConfig.groupId)
 .setGenerationId(generation.generationId)
 .setMemberId(generation.memberId)
-
.setGroupInstanceId(rebalanceConfig.groupInstanceId.orElse(null))
+.setGroupInstanceId(groupInstanceId)
 .setTopics(new 
ArrayList<>(requestTopicDataMap.values()))
 );
 
diff --git 
a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java
 
b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java
index d948990d69..5e080b7721 100644
--- 
a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java
+++ 
b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java
@@ -2821,6 +2821,32 @@ public abstract class ConsumerCoordinatorTest {
 assertEquals(newGen, coordinator.generation());
 }
 
+@Test
+public void testCommitOffsetShouldNotSetInstanceIdIfMemberIdIsUnknown() {
+rebalanceConfig = buildRebalanceConfig(groupInstanceId);
+ConsumerCoordinator coordinator = buildCoordinator(
+rebalanceConfig,
+new Metrics(),
+assignors,
+false,
+subscriptions
+);
+
+client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
+coordinator.ensureCoordinatorReady(time.timer(5000));
+
+client.prepareResponse(body -> {
+OffsetCommitRequestData data = ((OffsetCommitRequest) body).data();
+return data.groupInstanceId() == null && data.memberId().isEmpty();
+}, offsetCommitResponse(Collections.emptyMap()));
+
+RequestFuture future = 
coordinator.sendOffsetCommitRequest(singletonMap(t1p,
+new OffsetAndMetadata(100L, "metadata")));
+
+assertTrue(consumerClient.poll(future, time.timer(5000)));
+assertFalse(future.

[kafka] branch trunk updated: KAFKA-14201; Consumer should not send group instance ID if committing with empty member ID (#12599)

2022-09-08 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new b7f20be809 KAFKA-14201; Consumer should not send group instance ID if 
committing with empty member ID (#12599)
b7f20be809 is described below

commit b7f20be809c47202ace4b54e69be5428012df1a9
Author: David Jacot 
AuthorDate: Fri Sep 9 00:05:40 2022 +0200

KAFKA-14201; Consumer should not send group instance ID if committing with 
empty member ID (#12599)

The consumer group instance ID is used to support a notion of "static" 
consumer groups. The idea is to be able to identify the same group instance 
across restarts so that a rebalance is not needed. However, if the user sets 
`group.instance.id` in the consumer configuration, but uses "simple" assignment 
with `assign()`, then the instance ID nevertheless is sent in the OffsetCommit 
request to the coordinator. This may result in a surprising UNKNOWN_MEMBER_ID 
error.

This PR fixes the issue on the client side by not setting the group 
instance id if the member id is empty (no generation).

Reviewers: Jason Gustafson 
---
 .../consumer/internals/ConsumerCoordinator.java|  5 -
 .../internals/ConsumerCoordinatorTest.java | 26 ++
 2 files changed, 30 insertions(+), 1 deletion(-)

diff --git 
a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
 
b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
index 9838e7dc8f..5228c60e0f 100644
--- 
a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
+++ 
b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
@@ -1272,8 +1272,10 @@ public final class ConsumerCoordinator extends 
AbstractCoordinator {
 }
 
 final Generation generation;
+final String groupInstanceId;
 if (subscriptions.hasAutoAssignedPartitions()) {
 generation = generationIfStable();
+groupInstanceId = rebalanceConfig.groupInstanceId.orElse(null);
 // if the generation is null, we are not part of an active group 
(and we expect to be).
 // the only thing we can do is fail the commit and let the user 
rejoin the group in poll().
 if (generation == null) {
@@ -1293,6 +1295,7 @@ public final class ConsumerCoordinator extends 
AbstractCoordinator {
 }
 } else {
 generation = Generation.NO_GENERATION;
+groupInstanceId = null;
 }
 
 OffsetCommitRequest.Builder builder = new OffsetCommitRequest.Builder(
@@ -1300,7 +1303,7 @@ public final class ConsumerCoordinator extends 
AbstractCoordinator {
 .setGroupId(this.rebalanceConfig.groupId)
 .setGenerationId(generation.generationId)
 .setMemberId(generation.memberId)
-
.setGroupInstanceId(rebalanceConfig.groupInstanceId.orElse(null))
+.setGroupInstanceId(groupInstanceId)
 .setTopics(new 
ArrayList<>(requestTopicDataMap.values()))
 );
 
diff --git 
a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java
 
b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java
index 829fd2dbf0..06584aef46 100644
--- 
a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java
+++ 
b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java
@@ -2820,6 +2820,32 @@ public abstract class ConsumerCoordinatorTest {
 assertEquals(newGen, coordinator.generation());
 }
 
+@Test
+public void testCommitOffsetShouldNotSetInstanceIdIfMemberIdIsUnknown() {
+rebalanceConfig = buildRebalanceConfig(groupInstanceId);
+ConsumerCoordinator coordinator = buildCoordinator(
+rebalanceConfig,
+new Metrics(),
+assignors,
+false,
+subscriptions
+);
+
+client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
+coordinator.ensureCoordinatorReady(time.timer(5000));
+
+client.prepareResponse(body -> {
+OffsetCommitRequestData data = ((OffsetCommitRequest) body).data();
+return data.groupInstanceId() == null && data.memberId().isEmpty();
+}, offsetCommitResponse(Collections.emptyMap()));
+
+RequestFuture future = 
coordinator.sendOffsetCommitRequest(singletonMap(t1p,
+new OffsetAndMetadata(100L, "metadata")));
+
+assertTrue(consumerClient.poll(future, time.timer(5000)));
+assertFalse(future.

[kafka] branch 3.3 updated: KAFKA-14201; Consumer should not send group instance ID if committing with empty member ID (server side) (#12598)

2022-09-08 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch 3.3
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.3 by this push:
 new cfa1f098d6 KAFKA-14201; Consumer should not send group instance ID if 
committing with empty member ID (server side) (#12598)
cfa1f098d6 is described below

commit cfa1f098d6f0a47816d56e29d5dca01fb5cbf8a3
Author: David Jacot 
AuthorDate: Thu Sep 8 23:35:59 2022 +0200

KAFKA-14201; Consumer should not send group instance ID if committing with 
empty member ID (server side) (#12598)

The consumer group instance ID is used to support a notion of "static" 
consumer groups. The idea is to be able to identify the same group instance 
across restarts so that a rebalance is not needed. However, if the user sets 
`group.instance.id` in the consumer configuration, but uses "simple" assignment 
with `assign()`, then the instance ID nevertheless is sent in the OffsetCommit 
request to the coordinator. This may result in a surprising UNKNOWN_MEMBER_ID 
error.

This PR attempts to fix this issue for existing consumers by relaxing the 
validation in this case. One way is to simply ignore the member id and the 
static id when the generation id is -1. -1 signals that the request comes from 
either the admin client or a consumer which does not use the group management. 
This does not apply to transactional offsets commit.

Reviewers: Jason Gustafson 
---
 .../kafka/coordinator/group/GroupCoordinator.scala |  5 +
 .../coordinator/group/GroupCoordinatorTest.scala   | 24 +-
 2 files changed, 28 insertions(+), 1 deletion(-)

diff --git a/core/src/main/scala/kafka/coordinator/group/GroupCoordinator.scala 
b/core/src/main/scala/kafka/coordinator/group/GroupCoordinator.scala
index 6bf337d679..68dd8d4a06 100644
--- a/core/src/main/scala/kafka/coordinator/group/GroupCoordinator.scala
+++ b/core/src/main/scala/kafka/coordinator/group/GroupCoordinator.scala
@@ -982,6 +982,11 @@ class GroupCoordinator(val brokerId: Int,
   ): Option[Errors] = {
 if (group.is(Dead)) {
   Some(Errors.COORDINATOR_NOT_AVAILABLE)
+} else if (generationId < 0 && group.is(Empty)) {
+  // When the generation id is -1, the request comes from either the admin 
client
+  // or a consumer which does not use the group management facility. In 
this case,
+  // the request can commit offsets if the group is empty.
+  None
 } else if (generationId >= 0 || memberId != 
JoinGroupRequest.UNKNOWN_MEMBER_ID || groupInstanceId.isDefined) {
   validateCurrentMember(
 group,
diff --git 
a/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala 
b/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala
index 164c9ab1fe..a13793823c 100644
--- 
a/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala
+++ 
b/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala
@@ -1263,7 +1263,6 @@ class GroupCoordinatorTest {
 group.transitionTo(PreparingRebalance)
 group.transitionTo(Empty)
 
-
 // Illegal state exception shall trigger since follower id resides in 
pending member bucket.
 val expectedException = assertThrows(classOf[IllegalStateException],
   () => staticJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, 
followerInstanceId, protocolType, protocolSuperset, clockAdvance = 1))
@@ -2941,6 +2940,29 @@ class GroupCoordinatorTest {
 assertEquals(Errors.ILLEGAL_GENERATION, commitOffsetResult(tp))
   }
 
+  @Test
+  def testManualCommitOffsetShouldNotValidateMemberIdAndInstanceId(): Unit = {
+val tp = new TopicPartition("topic", 0)
+
+var commitOffsetResult = commitOffsets(
+  groupId,
+  JoinGroupRequest.UNKNOWN_MEMBER_ID,
+  -1,
+  Map(tp -> offsetAndMetadata(0)),
+  Some("instance-id")
+)
+assertEquals(Errors.NONE, commitOffsetResult(tp))
+
+commitOffsetResult = commitOffsets(
+  groupId,
+  "unknown",
+  -1,
+  Map(tp -> offsetAndMetadata(0)),
+  None
+)
+assertEquals(Errors.NONE, commitOffsetResult(tp))
+  }
+
   @Test
   def testTxnCommitOffsetWithFencedInstanceId(): Unit = {
 val tp = new TopicPartition("topic", 0)



[kafka] branch trunk updated: KAFKA-14201; Consumer should not send group instance ID if committing with empty member ID (server side) (#12598)

2022-09-08 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 706adc39f6 KAFKA-14201; Consumer should not send group instance ID if 
committing with empty member ID (server side) (#12598)
706adc39f6 is described below

commit 706adc39f6aff378f98000b0466181d7fcb1d997
Author: David Jacot 
AuthorDate: Thu Sep 8 23:35:59 2022 +0200

KAFKA-14201; Consumer should not send group instance ID if committing with 
empty member ID (server side) (#12598)

The consumer group instance ID is used to support a notion of "static" 
consumer groups. The idea is to be able to identify the same group instance 
across restarts so that a rebalance is not needed. However, if the user sets 
`group.instance.id` in the consumer configuration, but uses "simple" assignment 
with `assign()`, then the instance ID nevertheless is sent in the OffsetCommit 
request to the coordinator. This may result in a surprising UNKNOWN_MEMBER_ID 
error.

This PR attempts to fix this issue for existing consumers by relaxing the 
validation in this case. One way is to simply ignore the member id and the 
static id when the generation id is -1. -1 signals that the request comes from 
either the admin client or a consumer which does not use the group management. 
This does not apply to transactional offsets commit.

Reviewers: Jason Gustafson 
---
 .../kafka/coordinator/group/GroupCoordinator.scala |  5 +
 .../coordinator/group/GroupCoordinatorTest.scala   | 24 +-
 2 files changed, 28 insertions(+), 1 deletion(-)

diff --git a/core/src/main/scala/kafka/coordinator/group/GroupCoordinator.scala 
b/core/src/main/scala/kafka/coordinator/group/GroupCoordinator.scala
index 8d8f86f774..de4dbf3b0a 100644
--- a/core/src/main/scala/kafka/coordinator/group/GroupCoordinator.scala
+++ b/core/src/main/scala/kafka/coordinator/group/GroupCoordinator.scala
@@ -982,6 +982,11 @@ class GroupCoordinator(val brokerId: Int,
   ): Option[Errors] = {
 if (group.is(Dead)) {
   Some(Errors.COORDINATOR_NOT_AVAILABLE)
+} else if (generationId < 0 && group.is(Empty)) {
+  // When the generation id is -1, the request comes from either the admin 
client
+  // or a consumer which does not use the group management facility. In 
this case,
+  // the request can commit offsets if the group is empty.
+  None
 } else if (generationId >= 0 || memberId != 
JoinGroupRequest.UNKNOWN_MEMBER_ID || groupInstanceId.isDefined) {
   validateCurrentMember(
 group,
diff --git 
a/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala 
b/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala
index dd3524c1e3..596cd0efba 100644
--- 
a/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala
+++ 
b/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala
@@ -1263,7 +1263,6 @@ class GroupCoordinatorTest {
 group.transitionTo(PreparingRebalance)
 group.transitionTo(Empty)
 
-
 // Illegal state exception shall trigger since follower id resides in 
pending member bucket.
 val expectedException = assertThrows(classOf[IllegalStateException],
   () => staticJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, 
followerInstanceId, protocolType, protocolSuperset, clockAdvance = 1))
@@ -2941,6 +2940,29 @@ class GroupCoordinatorTest {
 assertEquals(Errors.ILLEGAL_GENERATION, commitOffsetResult(tp))
   }
 
+  @Test
+  def testManualCommitOffsetShouldNotValidateMemberIdAndInstanceId(): Unit = {
+val tp = new TopicPartition("topic", 0)
+
+var commitOffsetResult = commitOffsets(
+  groupId,
+  JoinGroupRequest.UNKNOWN_MEMBER_ID,
+  -1,
+  Map(tp -> offsetAndMetadata(0)),
+  Some("instance-id")
+)
+assertEquals(Errors.NONE, commitOffsetResult(tp))
+
+commitOffsetResult = commitOffsets(
+  groupId,
+  "unknown",
+  -1,
+  Map(tp -> offsetAndMetadata(0)),
+  None
+)
+assertEquals(Errors.NONE, commitOffsetResult(tp))
+  }
+
   @Test
   def testTxnCommitOffsetWithFencedInstanceId(): Unit = {
 val tp = new TopicPartition("topic", 0)



[kafka] branch 3.3 updated: MINOR: Dump log tool should support bootstrap checkpoint (#12556)

2022-08-25 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch 3.3
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.3 by this push:
 new 8151af1ebfa MINOR: Dump log tool should support bootstrap checkpoint 
(#12556)
8151af1ebfa is described below

commit 8151af1ebfa2d4f6ba09396a5dff76f289a6f526
Author: Jason Gustafson 
AuthorDate: Thu Aug 25 09:17:55 2022 -0700

MINOR: Dump log tool should support bootstrap checkpoint (#12556)

This patch adds support to `kafka-dump-log.sh` to print the 
`bootstrap.checkpoint` file from KIP-778: 
https://cwiki.apache.org/confluence/display/KAFKA/KIP-778:+KRaft+Upgrades.

Reviewers:  dengziming , Luke Chen 

---
 core/src/main/scala/kafka/tools/DumpLogSegments.scala | 9 +++--
 1 file changed, 7 insertions(+), 2 deletions(-)

diff --git a/core/src/main/scala/kafka/tools/DumpLogSegments.scala 
b/core/src/main/scala/kafka/tools/DumpLogSegments.scala
index b57342ff29b..e4d31baf1b6 100755
--- a/core/src/main/scala/kafka/tools/DumpLogSegments.scala
+++ b/core/src/main/scala/kafka/tools/DumpLogSegments.scala
@@ -30,6 +30,7 @@ import 
org.apache.kafka.common.metadata.{MetadataJsonConverters, MetadataRecordT
 import org.apache.kafka.common.protocol.ByteBufferAccessor
 import org.apache.kafka.common.record._
 import org.apache.kafka.common.utils.Utils
+import org.apache.kafka.controller.BootstrapMetadata
 import org.apache.kafka.metadata.MetadataRecordSerde
 import org.apache.kafka.snapshot.Snapshots
 
@@ -253,8 +254,12 @@ object DumpLogSegments {
   val startOffset = file.getName.split("\\.")(0).toLong
   println(s"Log starting offset: $startOffset")
 } else if (file.getName.endsWith(Snapshots.SUFFIX)) {
-  val path = Snapshots.parse(file.toPath).get()
-  println(s"Snapshot end offset: ${path.snapshotId.offset}, epoch: 
${path.snapshotId.epoch}")
+  if (file.getName == BootstrapMetadata.BOOTSTRAP_FILE) {
+println("KRaft bootstrap snapshot")
+  } else {
+val path = Snapshots.parse(file.toPath).get()
+println(s"Snapshot end offset: ${path.snapshotId.offset}, epoch: 
${path.snapshotId.epoch}")
+  }
 }
 val fileRecords = FileRecords.open(file, false).slice(0, maxBytes)
 try {



[kafka] branch trunk updated: MINOR: Dump log tool should support bootstrap checkpoint (#12556)

2022-08-25 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 289ab1494d0 MINOR: Dump log tool should support bootstrap checkpoint 
(#12556)
289ab1494d0 is described below

commit 289ab1494d00d4267f7a6d82f3d11884c6f47302
Author: Jason Gustafson 
AuthorDate: Thu Aug 25 09:17:55 2022 -0700

MINOR: Dump log tool should support bootstrap checkpoint (#12556)

This patch adds support to `kafka-dump-log.sh` to print the 
`bootstrap.checkpoint` file from KIP-778: 
https://cwiki.apache.org/confluence/display/KAFKA/KIP-778:+KRaft+Upgrades.

Reviewers:  dengziming , Luke Chen 

---
 core/src/main/scala/kafka/tools/DumpLogSegments.scala | 9 +++--
 1 file changed, 7 insertions(+), 2 deletions(-)

diff --git a/core/src/main/scala/kafka/tools/DumpLogSegments.scala 
b/core/src/main/scala/kafka/tools/DumpLogSegments.scala
index b57342ff29b..e4d31baf1b6 100755
--- a/core/src/main/scala/kafka/tools/DumpLogSegments.scala
+++ b/core/src/main/scala/kafka/tools/DumpLogSegments.scala
@@ -30,6 +30,7 @@ import 
org.apache.kafka.common.metadata.{MetadataJsonConverters, MetadataRecordT
 import org.apache.kafka.common.protocol.ByteBufferAccessor
 import org.apache.kafka.common.record._
 import org.apache.kafka.common.utils.Utils
+import org.apache.kafka.controller.BootstrapMetadata
 import org.apache.kafka.metadata.MetadataRecordSerde
 import org.apache.kafka.snapshot.Snapshots
 
@@ -253,8 +254,12 @@ object DumpLogSegments {
   val startOffset = file.getName.split("\\.")(0).toLong
   println(s"Log starting offset: $startOffset")
 } else if (file.getName.endsWith(Snapshots.SUFFIX)) {
-  val path = Snapshots.parse(file.toPath).get()
-  println(s"Snapshot end offset: ${path.snapshotId.offset}, epoch: 
${path.snapshotId.epoch}")
+  if (file.getName == BootstrapMetadata.BOOTSTRAP_FILE) {
+println("KRaft bootstrap snapshot")
+  } else {
+val path = Snapshots.parse(file.toPath).get()
+println(s"Snapshot end offset: ${path.snapshotId.offset}, epoch: 
${path.snapshotId.epoch}")
+  }
 }
 val fileRecords = FileRecords.open(file, false).slice(0, maxBytes)
 try {



[kafka] branch 3.3 updated: MINOR: A few cleanups for DescribeQuorum APIs (#12548)

2022-08-24 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch 3.3
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.3 by this push:
 new 12759d656c5 MINOR: A few cleanups for DescribeQuorum APIs (#12548)
12759d656c5 is described below

commit 12759d656c5d077bffa50f2db0be78f49e390ac8
Author: Jason Gustafson 
AuthorDate: Wed Aug 24 13:12:14 2022 -0700

MINOR: A few cleanups for DescribeQuorum APIs (#12548)

A few small cleanups in the `DescribeQuorum` API and handling logic:

- Change field types in `QuorumInfo`:
  - `leaderId`: `Integer` -> `int`
  - `leaderEpoch`: `Integer` -> `long` (to allow for type expansion in the 
future)
  - `highWatermark`: `Long` -> `long`
- Use field names `lastFetchTimestamp` and `lastCaughtUpTimestamp` 
consistently
- Move construction of `DescribeQuorumResponseData.PartitionData` into 
`LeaderState`
- Consolidate fetch time/offset update logic into 
`LeaderState.ReplicaState.updateFollowerState`

Reviewers: Luke Chen , José Armando García Sancio 

---
 .../kafka/clients/admin/KafkaAdminClient.java  |  19 +-
 .../org/apache/kafka/clients/admin/QuorumInfo.java |  68 +++--
 .../common/requests/DescribeQuorumResponse.java|  22 +-
 .../kafka/clients/admin/KafkaAdminClientTest.java  |   6 +-
 .../scala/kafka/admin/MetadataQuorumCommand.scala  |  10 +-
 .../kafka/server/KRaftClusterTest.scala|   8 +-
 .../org/apache/kafka/raft/KafkaRaftClient.java |  13 +-
 .../java/org/apache/kafka/raft/LeaderState.java| 233 +++---
 .../org/apache/kafka/raft/LeaderStateTest.java | 337 +++--
 .../apache/kafka/raft/RaftClientTestContext.java   |  15 +-
 .../kafka/raft/internals/KafkaRaftMetricsTest.java |   4 +-
 11 files changed, 456 insertions(+), 279 deletions(-)

diff --git 
a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java 
b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
index e5df779b616..1b837fda223 100644
--- a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
+++ b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
@@ -4357,12 +4357,21 @@ public class KafkaAdminClient extends AdminClient {
 }
 
 private QuorumInfo createQuorumResult(final 
DescribeQuorumResponseData.PartitionData partition) {
+List voters = 
partition.currentVoters().stream()
+.map(this::translateReplicaState)
+.collect(Collectors.toList());
+
+List observers = 
partition.observers().stream()
+.map(this::translateReplicaState)
+.collect(Collectors.toList());
+
 return new QuorumInfo(
-partition.leaderId(),
-partition.leaderEpoch(),
-partition.highWatermark(),
-partition.currentVoters().stream().map(v -> 
translateReplicaState(v)).collect(Collectors.toList()),
-partition.observers().stream().map(o -> 
translateReplicaState(o)).collect(Collectors.toList()));
+partition.leaderId(),
+partition.leaderEpoch(),
+partition.highWatermark(),
+voters,
+observers
+);
 }
 
 @Override
diff --git 
a/clients/src/main/java/org/apache/kafka/clients/admin/QuorumInfo.java 
b/clients/src/main/java/org/apache/kafka/clients/admin/QuorumInfo.java
index 3a0b6cf6f74..f9e4f8c11c9 100644
--- a/clients/src/main/java/org/apache/kafka/clients/admin/QuorumInfo.java
+++ b/clients/src/main/java/org/apache/kafka/clients/admin/QuorumInfo.java
@@ -24,13 +24,19 @@ import java.util.OptionalLong;
  * This class is used to describe the state of the quorum received in 
DescribeQuorumResponse.
  */
 public class QuorumInfo {
-private final Integer leaderId;
-private final Integer leaderEpoch;
-private final Long highWatermark;
+private final int leaderId;
+private final long leaderEpoch;
+private final long highWatermark;
 private final List voters;
 private final List observers;
 
-QuorumInfo(Integer leaderId, Integer leaderEpoch, Long highWatermark, 
List voters, List observers) {
+QuorumInfo(
+int leaderId,
+long leaderEpoch,
+long highWatermark,
+List voters,
+List observers
+) {
 this.leaderId = leaderId;
 this.leaderEpoch = leaderEpoch;
 this.highWatermark = highWatermark;
@@ -38,15 +44,15 @@ public class QuorumInfo {
 this.observers = observers;
 }
 
-public Integer leaderId() {
+public int leaderId() {
 return leaderId;
 }
 
-public Integer leaderEpoch() {
+public l

[kafka] branch trunk updated: MINOR: A few cleanups for DescribeQuorum APIs (#12548)

2022-08-24 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 5c52c61a46f MINOR: A few cleanups for DescribeQuorum APIs (#12548)
5c52c61a46f is described below

commit 5c52c61a46f103cda7fa48bd96584dafe984d6d3
Author: Jason Gustafson 
AuthorDate: Wed Aug 24 13:12:14 2022 -0700

MINOR: A few cleanups for DescribeQuorum APIs (#12548)

A few small cleanups in the `DescribeQuorum` API and handling logic:

- Change field types in `QuorumInfo`:
  - `leaderId`: `Integer` -> `int`
  - `leaderEpoch`: `Integer` -> `long` (to allow for type expansion in the 
future)
  - `highWatermark`: `Long` -> `long`
- Use field names `lastFetchTimestamp` and `lastCaughtUpTimestamp` 
consistently
- Move construction of `DescribeQuorumResponseData.PartitionData` into 
`LeaderState`
- Consolidate fetch time/offset update logic into 
`LeaderState.ReplicaState.updateFollowerState`

Reviewers: Luke Chen , José Armando García Sancio 

---
 .../kafka/clients/admin/KafkaAdminClient.java  |  19 +-
 .../org/apache/kafka/clients/admin/QuorumInfo.java |  68 +++--
 .../common/requests/DescribeQuorumResponse.java|  22 +-
 .../kafka/clients/admin/KafkaAdminClientTest.java  |   6 +-
 .../scala/kafka/admin/MetadataQuorumCommand.scala  |  10 +-
 .../kafka/server/KRaftClusterTest.scala|   8 +-
 .../org/apache/kafka/raft/KafkaRaftClient.java |  13 +-
 .../java/org/apache/kafka/raft/LeaderState.java| 233 +++---
 .../org/apache/kafka/raft/LeaderStateTest.java | 337 +++--
 .../apache/kafka/raft/RaftClientTestContext.java   |  15 +-
 .../kafka/raft/internals/KafkaRaftMetricsTest.java |   4 +-
 11 files changed, 456 insertions(+), 279 deletions(-)

diff --git 
a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java 
b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
index 95fb0ed0d70..92e41ed1f2d 100644
--- a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
+++ b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
@@ -4355,12 +4355,21 @@ public class KafkaAdminClient extends AdminClient {
 }
 
 private QuorumInfo createQuorumResult(final 
DescribeQuorumResponseData.PartitionData partition) {
+List voters = 
partition.currentVoters().stream()
+.map(this::translateReplicaState)
+.collect(Collectors.toList());
+
+List observers = 
partition.observers().stream()
+.map(this::translateReplicaState)
+.collect(Collectors.toList());
+
 return new QuorumInfo(
-partition.leaderId(),
-partition.leaderEpoch(),
-partition.highWatermark(),
-partition.currentVoters().stream().map(v -> 
translateReplicaState(v)).collect(Collectors.toList()),
-partition.observers().stream().map(o -> 
translateReplicaState(o)).collect(Collectors.toList()));
+partition.leaderId(),
+partition.leaderEpoch(),
+partition.highWatermark(),
+voters,
+observers
+);
 }
 
 @Override
diff --git 
a/clients/src/main/java/org/apache/kafka/clients/admin/QuorumInfo.java 
b/clients/src/main/java/org/apache/kafka/clients/admin/QuorumInfo.java
index 3a0b6cf6f74..f9e4f8c11c9 100644
--- a/clients/src/main/java/org/apache/kafka/clients/admin/QuorumInfo.java
+++ b/clients/src/main/java/org/apache/kafka/clients/admin/QuorumInfo.java
@@ -24,13 +24,19 @@ import java.util.OptionalLong;
  * This class is used to describe the state of the quorum received in 
DescribeQuorumResponse.
  */
 public class QuorumInfo {
-private final Integer leaderId;
-private final Integer leaderEpoch;
-private final Long highWatermark;
+private final int leaderId;
+private final long leaderEpoch;
+private final long highWatermark;
 private final List voters;
 private final List observers;
 
-QuorumInfo(Integer leaderId, Integer leaderEpoch, Long highWatermark, 
List voters, List observers) {
+QuorumInfo(
+int leaderId,
+long leaderEpoch,
+long highWatermark,
+List voters,
+List observers
+) {
 this.leaderId = leaderId;
 this.leaderEpoch = leaderEpoch;
 this.highWatermark = highWatermark;
@@ -38,15 +44,15 @@ public class QuorumInfo {
 this.observers = observers;
 }
 
-public Integer leaderId() {
+public int leaderId() {
 return leaderId;
 }
 
-public Integer leaderEpoch() {
+public l

[kafka] branch 3.3 updated: KAFKA-13914: Add command line tool kafka-metadata-quorum.sh (#12469)

2022-08-20 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch 3.3
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.3 by this push:
 new 7d055d6996 KAFKA-13914: Add command line tool kafka-metadata-quorum.sh 
(#12469)
7d055d6996 is described below

commit 7d055d69967bf62e7cc1b8e14a81f38e3e608472
Author: dengziming 
AuthorDate: Sat Aug 20 23:37:26 2022 +0800

KAFKA-13914: Add command line tool kafka-metadata-quorum.sh (#12469)

 Add `MetadataQuorumCommand` to describe quorum status, I'm trying to use 
arg4j style command format, currently, we only support one sub-command which is 
"describe" and we can specify 2 arguments which are --status and --replication.

```
# describe quorum status
kafka-metadata-quorum.sh --bootstrap-server localhost:9092 describe 
--replication

ReplicaId   LogEndOffsetLag LastFetchTimeMs LastCaughtUpTimeMs  
Status
0   10  0   -1  -1  
 Leader
1   10  0   -1  -1  
 Follower
2   10  0   -1  -1  
 Follower

kafka-metadata-quorum.sh --bootstrap-server localhost:9092 describe --status
ClusterId: fMCL8kv1SWm87L_Md-I2hg
LeaderId: 3002
LeaderEpoch:  2
HighWatermark:  10
MaxFollowerLag: 0
MaxFollowerLagTimeMs:   -1
CurrentVoters:[3000,3001,3002]
CurrentObservers:  [0,1,2]

# specify AdminClient properties
kafka-metadata-quorum.sh --bootstrap-server localhost:9092 --command-config 
config.properties describe --status
```

Reviewers: Jason Gustafson 
---
 bin/kafka-metadata-quorum.sh   |  17 ++
 bin/windows/kafka-metatada-quorum.bat  |  17 ++
 build.gradle   |   1 +
 checkstyle/import-control.xml  |   1 +
 .../kafka/clients/admin/KafkaAdminClient.java  |   2 +
 .../org/apache/kafka/clients/admin/QuorumInfo.java |  14 +-
 .../kafka/clients/admin/KafkaAdminClientTest.java  |   2 +-
 .../scala/kafka/admin/MetadataQuorumCommand.scala  | 172 ++
 .../test/junit/RaftClusterInvocationContext.java   |   4 +
 .../kafka/admin/MetadataQuorumCommandTest.scala| 192 +
 .../kafka/server/DescribeQuorumRequestTest.scala   |   2 +
 .../org/apache/kafka/server/util}/ToolsUtils.java  |  51 +-
 .../apache/kafka/tools/ProducerPerformance.java|   1 +
 .../apache/kafka/tools/TransactionsCommand.java|  46 +
 14 files changed, 474 insertions(+), 48 deletions(-)

diff --git a/bin/kafka-metadata-quorum.sh b/bin/kafka-metadata-quorum.sh
new file mode 100755
index 00..24bedbded1
--- /dev/null
+++ b/bin/kafka-metadata-quorum.sh
@@ -0,0 +1,17 @@
+#!/bin/bash
+# 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.
+
+exec $(dirname $0)/kafka-run-class.sh kafka.admin.MetadataQuorumCommand "$@"
diff --git a/bin/windows/kafka-metatada-quorum.bat 
b/bin/windows/kafka-metatada-quorum.bat
new file mode 100644
index 00..4ea8e3109f
--- /dev/null
+++ b/bin/windows/kafka-metatada-quorum.bat
@@ -0,0 +1,17 @@
+@echo off
+rem Licensed to the Apache Software Foundation (ASF) under one or more
+rem contributor license agreements.  See the NOTICE file distributed with
+rem this work for additional information regarding copyright ownership.
+rem The ASF licenses this file to You under the Apache License, Version 2.0
+rem (the "License"); you may not use this file except in compliance with
+rem the License.  You may obtain a copy of the License at
+rem
+rem http://www.apache.org/licenses/LICENSE-2.0
+rem
+rem Unless required by applicable law or agreed to in writing, software
+rem distributed under the License is distributed on an "AS IS" BASIS,
+rem 

[kafka] branch trunk updated: KAFKA-13914: Add command line tool kafka-metadata-quorum.sh (#12469)

2022-08-20 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 150fd5b0b1 KAFKA-13914: Add command line tool kafka-metadata-quorum.sh 
(#12469)
150fd5b0b1 is described below

commit 150fd5b0b18c4761d8f7d7ba9a480aa9f622024f
Author: dengziming 
AuthorDate: Sat Aug 20 23:37:26 2022 +0800

KAFKA-13914: Add command line tool kafka-metadata-quorum.sh (#12469)

 Add `MetadataQuorumCommand` to describe quorum status, I'm trying to use 
arg4j style command format, currently, we only support one sub-command which is 
"describe" and we can specify 2 arguments which are --status and --replication.

```
# describe quorum status
kafka-metadata-quorum.sh --bootstrap-server localhost:9092 describe 
--replication

ReplicaId   LogEndOffsetLag LastFetchTimeMs LastCaughtUpTimeMs  
Status
0   10  0   -1  -1  
 Leader
1   10  0   -1  -1  
 Follower
2   10  0   -1  -1  
 Follower

kafka-metadata-quorum.sh --bootstrap-server localhost:9092 describe --status
ClusterId: fMCL8kv1SWm87L_Md-I2hg
LeaderId: 3002
LeaderEpoch:  2
HighWatermark:  10
MaxFollowerLag: 0
MaxFollowerLagTimeMs:   -1
CurrentVoters:[3000,3001,3002]
CurrentObservers:  [0,1,2]

# specify AdminClient properties
kafka-metadata-quorum.sh --bootstrap-server localhost:9092 --command-config 
config.properties describe --status
```

Reviewers: Jason Gustafson 
---
 bin/kafka-metadata-quorum.sh   |  17 ++
 bin/windows/kafka-metatada-quorum.bat  |  17 ++
 build.gradle   |   1 +
 checkstyle/import-control.xml  |   1 +
 .../kafka/clients/admin/KafkaAdminClient.java  |   2 +
 .../org/apache/kafka/clients/admin/QuorumInfo.java |  14 +-
 .../kafka/clients/admin/KafkaAdminClientTest.java  |   2 +-
 .../scala/kafka/admin/MetadataQuorumCommand.scala  | 172 ++
 .../test/junit/RaftClusterInvocationContext.java   |   4 +
 .../kafka/admin/MetadataQuorumCommandTest.scala| 192 +
 .../kafka/server/DescribeQuorumRequestTest.scala   |   2 +
 .../org/apache/kafka/server/util}/ToolsUtils.java  |  51 +-
 .../apache/kafka/tools/ProducerPerformance.java|   1 +
 .../apache/kafka/tools/TransactionsCommand.java|  46 +
 14 files changed, 474 insertions(+), 48 deletions(-)

diff --git a/bin/kafka-metadata-quorum.sh b/bin/kafka-metadata-quorum.sh
new file mode 100755
index 00..24bedbded1
--- /dev/null
+++ b/bin/kafka-metadata-quorum.sh
@@ -0,0 +1,17 @@
+#!/bin/bash
+# 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.
+
+exec $(dirname $0)/kafka-run-class.sh kafka.admin.MetadataQuorumCommand "$@"
diff --git a/bin/windows/kafka-metatada-quorum.bat 
b/bin/windows/kafka-metatada-quorum.bat
new file mode 100644
index 00..4ea8e3109f
--- /dev/null
+++ b/bin/windows/kafka-metatada-quorum.bat
@@ -0,0 +1,17 @@
+@echo off
+rem Licensed to the Apache Software Foundation (ASF) under one or more
+rem contributor license agreements.  See the NOTICE file distributed with
+rem this work for additional information regarding copyright ownership.
+rem The ASF licenses this file to You under the Apache License, Version 2.0
+rem (the "License"); you may not use this file except in compliance with
+rem the License.  You may obtain a copy of the License at
+rem
+rem http://www.apache.org/licenses/LICENSE-2.0
+rem
+rem Unless required by applicable law or agreed to in writing, software
+rem distributed under the License is distributed on an "AS IS" BASIS,
+rem 

[kafka] branch 3.3 updated: KAFKA-13888; Implement `LastFetchTimestamp` and in `LastCaughtUpTimestamp` for DescribeQuorumResponse [KIP-836] (#12508)

2022-08-19 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch 3.3
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.3 by this push:
 new b4b57dbcff KAFKA-13888; Implement `LastFetchTimestamp` and  in 
`LastCaughtUpTimestamp` for DescribeQuorumResponse [KIP-836] (#12508)
b4b57dbcff is described below

commit b4b57dbcffdf54583e75b6cdedbf0a4fa941981a
Author: Niket 
AuthorDate: Fri Aug 19 15:09:09 2022 -0700

KAFKA-13888; Implement `LastFetchTimestamp` and  in `LastCaughtUpTimestamp` 
for DescribeQuorumResponse [KIP-836] (#12508)

This commit implements the newly added fields `LastFetchTimestamp` and 
`LastCaughtUpTimestamp` for KIP-836: 
https://cwiki.apache.org/confluence/display/KAFKA/KIP-836:+Addition+of+Information+in+DescribeQuorumResponse+about+Voter+Lag.

Reviewers: Jason Gustafson 
---
 .../main/scala/kafka/network/RequestChannel.scala  |   9 +-
 .../kafka/server/KRaftClusterTest.scala|  55 +++-
 .../kafka/server/DescribeQuorumRequestTest.scala   |   9 +-
 .../org/apache/kafka/raft/KafkaRaftClient.java |  16 +--
 .../java/org/apache/kafka/raft/LeaderState.java|  94 ++
 .../org/apache/kafka/raft/KafkaRaftClientTest.java |  22 +++-
 .../org/apache/kafka/raft/LeaderStateTest.java | 140 +
 .../kafka/raft/internals/KafkaRaftMetricsTest.java |   2 +-
 8 files changed, 281 insertions(+), 66 deletions(-)

diff --git a/core/src/main/scala/kafka/network/RequestChannel.scala 
b/core/src/main/scala/kafka/network/RequestChannel.scala
index 4fa611206a..2200757c70 100644
--- a/core/src/main/scala/kafka/network/RequestChannel.scala
+++ b/core/src/main/scala/kafka/network/RequestChannel.scala
@@ -121,10 +121,17 @@ object RequestChannel extends Logging {
 
 def isForwarded: Boolean = envelope.isDefined
 
+private def shouldReturnNotController(response: AbstractResponse): Boolean 
= {
+  response match {
+case describeQuorumResponse: DescribeQuorumResponse => 
response.errorCounts.containsKey(Errors.NOT_LEADER_OR_FOLLOWER)
+case _ => response.errorCounts.containsKey(Errors.NOT_CONTROLLER)
+  }
+}
+
 def buildResponseSend(abstractResponse: AbstractResponse): Send = {
   envelope match {
 case Some(request) =>
-  val envelopeResponse = if 
(abstractResponse.errorCounts().containsKey(Errors.NOT_CONTROLLER)) {
+  val envelopeResponse = if 
(shouldReturnNotController(abstractResponse)) {
 // Since it's a NOT_CONTROLLER error response, we need to make 
envelope response with NOT_CONTROLLER error
 // to notify the requester (i.e. BrokerToControllerRequestThread) 
to update active controller
 new EnvelopeResponse(new EnvelopeResponseData()
diff --git 
a/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala 
b/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala
index 509facf921..a16cf821d4 100644
--- a/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala
+++ b/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala
@@ -21,7 +21,7 @@ import kafka.network.SocketServer
 import kafka.server.IntegrationTestUtils.connectAndReceive
 import kafka.testkit.{BrokerNode, KafkaClusterTestKit, TestKitNodes}
 import kafka.utils.TestUtils
-import org.apache.kafka.clients.admin.{Admin, AlterConfigOp, Config, 
ConfigEntry, NewPartitionReassignment, NewTopic}
+import org.apache.kafka.clients.admin.{Admin, AdminClientConfig, 
AlterConfigOp, Config, ConfigEntry, DescribeMetadataQuorumOptions, 
NewPartitionReassignment, NewTopic}
 import org.apache.kafka.common.{TopicPartition, TopicPartitionInfo}
 import org.apache.kafka.common.message.DescribeClusterRequestData
 import org.apache.kafka.common.network.ListenerName
@@ -32,7 +32,7 @@ import org.junit.jupiter.api.Assertions._
 import org.junit.jupiter.api.{Tag, Test, Timeout}
 
 import java.util
-import java.util.{Arrays, Collections, Optional}
+import java.util.{Arrays, Collections, Optional, OptionalLong, Properties}
 import org.apache.kafka.clients.admin.AlterConfigOp.OpType
 import org.apache.kafka.common.config.ConfigResource
 import org.apache.kafka.common.config.ConfigResource.Type
@@ -778,4 +778,55 @@ class KRaftClusterTest {
   cluster.close()
 }
   }
+  def createAdminClient(cluster: KafkaClusterTestKit): Admin = {
+var props: Properties = null
+props = cluster.clientProperties()
+props.put(AdminClientConfig.CLIENT_ID_CONFIG, this.getClass.getName)
+Admin.create(props)
+  }
+
+  @Test
+  def testDescribeQuorumRequestToBrokers() : Unit = {
+val cluster = new KafkaClusterTestKit.Builder(
+  new TestKitNodes.Builder().
+setNumBrokerNodes(4).
+setNumControllerNodes(3).build()).build()
+try {
+  cluster.format
+  cluster.startup
+  for (i <- 0 to 3) {
+TestUtils.w

[kafka] branch trunk updated: KAFKA-13888; Implement `LastFetchTimestamp` and in `LastCaughtUpTimestamp` for DescribeQuorumResponse [KIP-836] (#12508)

2022-08-19 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new c7f051914e KAFKA-13888; Implement `LastFetchTimestamp` and  in 
`LastCaughtUpTimestamp` for DescribeQuorumResponse [KIP-836] (#12508)
c7f051914e is described below

commit c7f051914e5d20e137901b6d687d6e7cf8775df1
Author: Niket 
AuthorDate: Fri Aug 19 15:09:09 2022 -0700

KAFKA-13888; Implement `LastFetchTimestamp` and  in `LastCaughtUpTimestamp` 
for DescribeQuorumResponse [KIP-836] (#12508)

This commit implements the newly added fields `LastFetchTimestamp` and 
`LastCaughtUpTimestamp` for KIP-836: 
https://cwiki.apache.org/confluence/display/KAFKA/KIP-836:+Addition+of+Information+in+DescribeQuorumResponse+about+Voter+Lag.

Reviewers: Jason Gustafson 
---
 .../main/scala/kafka/network/RequestChannel.scala  |   9 +-
 .../kafka/server/KRaftClusterTest.scala|  55 +++-
 .../kafka/server/DescribeQuorumRequestTest.scala   |   9 +-
 .../org/apache/kafka/raft/KafkaRaftClient.java |  16 +--
 .../java/org/apache/kafka/raft/LeaderState.java|  94 ++
 .../org/apache/kafka/raft/KafkaRaftClientTest.java |  22 +++-
 .../org/apache/kafka/raft/LeaderStateTest.java | 140 +
 .../kafka/raft/internals/KafkaRaftMetricsTest.java |   2 +-
 8 files changed, 281 insertions(+), 66 deletions(-)

diff --git a/core/src/main/scala/kafka/network/RequestChannel.scala 
b/core/src/main/scala/kafka/network/RequestChannel.scala
index 4fa611206a..2200757c70 100644
--- a/core/src/main/scala/kafka/network/RequestChannel.scala
+++ b/core/src/main/scala/kafka/network/RequestChannel.scala
@@ -121,10 +121,17 @@ object RequestChannel extends Logging {
 
 def isForwarded: Boolean = envelope.isDefined
 
+private def shouldReturnNotController(response: AbstractResponse): Boolean 
= {
+  response match {
+case describeQuorumResponse: DescribeQuorumResponse => 
response.errorCounts.containsKey(Errors.NOT_LEADER_OR_FOLLOWER)
+case _ => response.errorCounts.containsKey(Errors.NOT_CONTROLLER)
+  }
+}
+
 def buildResponseSend(abstractResponse: AbstractResponse): Send = {
   envelope match {
 case Some(request) =>
-  val envelopeResponse = if 
(abstractResponse.errorCounts().containsKey(Errors.NOT_CONTROLLER)) {
+  val envelopeResponse = if 
(shouldReturnNotController(abstractResponse)) {
 // Since it's a NOT_CONTROLLER error response, we need to make 
envelope response with NOT_CONTROLLER error
 // to notify the requester (i.e. BrokerToControllerRequestThread) 
to update active controller
 new EnvelopeResponse(new EnvelopeResponseData()
diff --git 
a/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala 
b/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala
index 509facf921..a16cf821d4 100644
--- a/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala
+++ b/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala
@@ -21,7 +21,7 @@ import kafka.network.SocketServer
 import kafka.server.IntegrationTestUtils.connectAndReceive
 import kafka.testkit.{BrokerNode, KafkaClusterTestKit, TestKitNodes}
 import kafka.utils.TestUtils
-import org.apache.kafka.clients.admin.{Admin, AlterConfigOp, Config, 
ConfigEntry, NewPartitionReassignment, NewTopic}
+import org.apache.kafka.clients.admin.{Admin, AdminClientConfig, 
AlterConfigOp, Config, ConfigEntry, DescribeMetadataQuorumOptions, 
NewPartitionReassignment, NewTopic}
 import org.apache.kafka.common.{TopicPartition, TopicPartitionInfo}
 import org.apache.kafka.common.message.DescribeClusterRequestData
 import org.apache.kafka.common.network.ListenerName
@@ -32,7 +32,7 @@ import org.junit.jupiter.api.Assertions._
 import org.junit.jupiter.api.{Tag, Test, Timeout}
 
 import java.util
-import java.util.{Arrays, Collections, Optional}
+import java.util.{Arrays, Collections, Optional, OptionalLong, Properties}
 import org.apache.kafka.clients.admin.AlterConfigOp.OpType
 import org.apache.kafka.common.config.ConfigResource
 import org.apache.kafka.common.config.ConfigResource.Type
@@ -778,4 +778,55 @@ class KRaftClusterTest {
   cluster.close()
 }
   }
+  def createAdminClient(cluster: KafkaClusterTestKit): Admin = {
+var props: Properties = null
+props = cluster.clientProperties()
+props.put(AdminClientConfig.CLIENT_ID_CONFIG, this.getClass.getName)
+Admin.create(props)
+  }
+
+  @Test
+  def testDescribeQuorumRequestToBrokers() : Unit = {
+val cluster = new KafkaClusterTestKit.Builder(
+  new TestKitNodes.Builder().
+setNumBrokerNodes(4).
+setNumControllerNodes(3).build()).build()
+try {
+  cluster.format
+  cluster.startup
+  for (i <- 0 to 3) {
+TestUtils.w

[kafka] branch 3.3 updated: MINOR: Fix unexpected request error in kraft shutdown (#12538)

2022-08-19 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch 3.3
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.3 by this push:
 new 26ffdd7728 MINOR: Fix unexpected request error in kraft shutdown 
(#12538)
26ffdd7728 is described below

commit 26ffdd772827627af2e3213a395bae56c1b41321
Author: Jason Gustafson 
AuthorDate: Fri Aug 19 12:45:05 2022 -0700

MINOR: Fix unexpected request error in kraft shutdown (#12538)

We have been seeing a few exceptions like the following when running 
integration tests:
```
[2022-08-18 13:02:59,470] ERROR [ControllerApis nodeId=3000] Unexpected 
error handling request RequestHeader(apiKey=FETCH, apiVersion=13, 
clientId=raft-client-0, correlationId=7) -- 
FetchRequestData(clusterId='txpo87ZUSbGSeV2v7H0n_w', replicaId=0, 
maxWaitMs=500, minBytes=0, maxBytes=8388608, isolationLevel=0, sessionId=0, 
sessionEpoch=-1, topics=[FetchTopic(topic='__cluster_metadata', 
topicId=AQ, partitions=[FetchPartition(partition=0, 
currentLeaderEpoch=1, fetchOf [...]
java.util.concurrent.CompletionException: java.util.NoSuchElementException: 
key not found: BROKER_NOT_AVAILABLE
at 
java.base/java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:315)
at 
java.base/java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:320)
at 
java.base/java.util.concurrent.CompletableFuture.uniHandle(CompletableFuture.java:936)
at 
java.base/java.util.concurrent.CompletableFuture$UniHandle.tryFire(CompletableFuture.java:911)
at 
java.base/java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:510)
at 
java.base/java.util.concurrent.CompletableFuture.complete(CompletableFuture.java:2147)
at 
org.apache.kafka.raft.KafkaRaftClient.lambda$handleRequest$19(KafkaRaftClient.java:1666)
at 
java.base/java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:863)
at 
java.base/java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:841)
at 
java.base/java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:510)
at 
java.base/java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:2162)
at 
kafka.raft.TimingWheelExpirationService$TimerTaskCompletableFuture.run(TimingWheelExpirationService.scala:32)
at 
java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:539)
at 
java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)
at 
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)
at 
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
at java.base/java.lang.Thread.run(Thread.java:833)
Caused by: java.util.NoSuchElementException: key not found: 
BROKER_NOT_AVAILABLE
```
There are two causes for this error that I found. First, we were not 
shutting down the timer services in `RaftManager` which are used in the 
purgatory implementation. This meant that operations remaining in purgatory 
could be completed even after `RaftManager` was shutdown. Second, the shutdown 
order in `KafkaClusterTestKit` was problematic. The `RaftManager` instance 
depends on the `SocketServer` in `ControllerServer`, but it was the latter that 
was shutdown first. Instead, we should [...]

Reviewers: Ismael Juma 
---
 core/src/main/scala/kafka/raft/RaftManager.scala   | 10 ---
 .../main/scala/kafka/server/ControllerApis.scala   |  1 +
 .../main/scala/kafka/server/KafkaRaftServer.scala  |  7 +
 .../java/kafka/testkit/KafkaClusterTestKit.java| 35 +-
 4 files changed, 35 insertions(+), 18 deletions(-)

diff --git a/core/src/main/scala/kafka/raft/RaftManager.scala 
b/core/src/main/scala/kafka/raft/RaftManager.scala
index a44d9d8fe0..5b8fe1e827 100644
--- a/core/src/main/scala/kafka/raft/RaftManager.scala
+++ b/core/src/main/scala/kafka/raft/RaftManager.scala
@@ -122,6 +122,8 @@ class KafkaRaftManager[T](
   private val dataDir = createDataDir()
   override val replicatedLog: ReplicatedLog = buildMetadataLog()
   private val netChannel = buildNetworkChannel()
+  private val expirationTimer = new SystemTimer("raft-expiration-executor")
+  private val expirationService = new 
TimingWheelExpirationService(expirationTimer)
   override val client: KafkaRaftClient[T] = buildRaftClient()
   private val raftIoThread = new RaftIoThread(client, threadNamePrefix)
 
@@ -133,10 +135,10 @@ class KafkaRaftManager[T](
 case spec: InetAddressSpec =>
   netChannel.updateEndpoint(voterAddressEntry.getKey, spec)
 case _: UnknownAddressSpec =>
-  logger

[kafka] branch trunk updated: MINOR: Fix unexpected request error in kraft shutdown (#12538)

2022-08-19 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new a724166fcc MINOR: Fix unexpected request error in kraft shutdown 
(#12538)
a724166fcc is described below

commit a724166fcc9d260df4a9f05357d96535da7c1627
Author: Jason Gustafson 
AuthorDate: Fri Aug 19 12:45:05 2022 -0700

MINOR: Fix unexpected request error in kraft shutdown (#12538)

We have been seeing a few exceptions like the following when running 
integration tests:
```
[2022-08-18 13:02:59,470] ERROR [ControllerApis nodeId=3000] Unexpected 
error handling request RequestHeader(apiKey=FETCH, apiVersion=13, 
clientId=raft-client-0, correlationId=7) -- 
FetchRequestData(clusterId='txpo87ZUSbGSeV2v7H0n_w', replicaId=0, 
maxWaitMs=500, minBytes=0, maxBytes=8388608, isolationLevel=0, sessionId=0, 
sessionEpoch=-1, topics=[FetchTopic(topic='__cluster_metadata', 
topicId=AQ, partitions=[FetchPartition(partition=0, 
currentLeaderEpoch=1, fetchOf [...]
java.util.concurrent.CompletionException: java.util.NoSuchElementException: 
key not found: BROKER_NOT_AVAILABLE
at 
java.base/java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:315)
at 
java.base/java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:320)
at 
java.base/java.util.concurrent.CompletableFuture.uniHandle(CompletableFuture.java:936)
at 
java.base/java.util.concurrent.CompletableFuture$UniHandle.tryFire(CompletableFuture.java:911)
at 
java.base/java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:510)
at 
java.base/java.util.concurrent.CompletableFuture.complete(CompletableFuture.java:2147)
at 
org.apache.kafka.raft.KafkaRaftClient.lambda$handleRequest$19(KafkaRaftClient.java:1666)
at 
java.base/java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:863)
at 
java.base/java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:841)
at 
java.base/java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:510)
at 
java.base/java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:2162)
at 
kafka.raft.TimingWheelExpirationService$TimerTaskCompletableFuture.run(TimingWheelExpirationService.scala:32)
at 
java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:539)
at 
java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)
at 
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)
at 
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
at java.base/java.lang.Thread.run(Thread.java:833)
Caused by: java.util.NoSuchElementException: key not found: 
BROKER_NOT_AVAILABLE
```
There are two causes for this error that I found. First, we were not 
shutting down the timer services in `RaftManager` which are used in the 
purgatory implementation. This meant that operations remaining in purgatory 
could be completed even after `RaftManager` was shutdown. Second, the shutdown 
order in `KafkaClusterTestKit` was problematic. The `RaftManager` instance 
depends on the `SocketServer` in `ControllerServer`, but it was the latter that 
was shutdown first. Instead, we should [...]

Reviewers: Ismael Juma 
---
 core/src/main/scala/kafka/raft/RaftManager.scala   | 10 ---
 .../main/scala/kafka/server/ControllerApis.scala   |  1 +
 .../main/scala/kafka/server/KafkaRaftServer.scala  |  7 +
 .../java/kafka/testkit/KafkaClusterTestKit.java| 35 +-
 4 files changed, 35 insertions(+), 18 deletions(-)

diff --git a/core/src/main/scala/kafka/raft/RaftManager.scala 
b/core/src/main/scala/kafka/raft/RaftManager.scala
index a44d9d8fe0..5b8fe1e827 100644
--- a/core/src/main/scala/kafka/raft/RaftManager.scala
+++ b/core/src/main/scala/kafka/raft/RaftManager.scala
@@ -122,6 +122,8 @@ class KafkaRaftManager[T](
   private val dataDir = createDataDir()
   override val replicatedLog: ReplicatedLog = buildMetadataLog()
   private val netChannel = buildNetworkChannel()
+  private val expirationTimer = new SystemTimer("raft-expiration-executor")
+  private val expirationService = new 
TimingWheelExpirationService(expirationTimer)
   override val client: KafkaRaftClient[T] = buildRaftClient()
   private val raftIoThread = new RaftIoThread(client, threadNamePrefix)
 
@@ -133,10 +135,10 @@ class KafkaRaftManager[T](
 case spec: InetAddressSpec =>
   netChannel.updateEndpoint(voterAddressEntry.getKey, spec)
 case _: UnknownAddressSpec =>
-  logger

[kafka] branch 3.3 updated: KAFKA-14167; Completion exceptions should not be translated directly to error codes (#12518)

2022-08-17 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch 3.3
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.3 by this push:
 new 81c4426550 KAFKA-14167; Completion exceptions should not be translated 
directly to error codes (#12518)
81c4426550 is described below

commit 81c442655079355f859dcdb495df015a1a1f7baa
Author: Jason Gustafson 
AuthorDate: Wed Aug 17 18:11:42 2022 -0700

KAFKA-14167; Completion exceptions should not be translated directly to 
error codes (#12518)

There are a few cases in `ControllerApis` where we may see an 
`ApiException` wrapped as a `CompletionException`. This can happen in 
`QuorumController.allocateProducerIds` where the returned future is the result 
of calling `thenApply` on the future passed to the controller. The danger when 
this happens is that the `CompletionException` gets passed to 
`Errors.forException`, which translates it to an `UNKNOWN_SERVER_ERROR`. At a 
minimum, I found that the `AllocateProducerIds` and `Update [...]

Interestingly, `DeleteTopics` is not affected by this bug as I originally 
suspected. This is because we have logic in `ApiError.fromThrowable` to check 
for both `CompletionException` and `ExecutionException` and to pull out the 
underlying cause. This patch duplicates this logic from 
`ApiError.fromThrowable` into `Errors.forException` to be sure that we handle 
all cases where exceptions are converted to error codes.

Reviewers: David Arthur 
---
 .../org/apache/kafka/common/protocol/Errors.java   | 21 -
 .../requests/AllocateProducerIdsResponse.java  |  4 +
 .../org/apache/kafka/common/requests/ApiError.java | 10 +--
 .../apache/kafka/common/requests/ApiErrorTest.java |  6 +-
 .../main/scala/kafka/server/ControllerApis.scala   |  7 +-
 .../test/junit/RaftClusterInvocationContext.java   |  4 +-
 .../server/AllocateProducerIdsRequestTest.scala| 98 ++
 .../unit/kafka/server/ControllerApisTest.scala | 78 -
 8 files changed, 205 insertions(+), 23 deletions(-)

diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java 
b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java
index 2ca42bafcf..c220bbcde4 100644
--- a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java
+++ b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java
@@ -132,6 +132,8 @@ import org.slf4j.LoggerFactory;
 
 import java.util.HashMap;
 import java.util.Map;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ExecutionException;
 import java.util.function.Function;
 
 /**
@@ -469,7 +471,8 @@ public enum Errors {
  * If there are multiple matches in the class hierarchy, the first match 
starting from the bottom is used.
  */
 public static Errors forException(Throwable t) {
-Class clazz = t.getClass();
+Throwable cause = maybeUnwrapException(t);
+Class clazz = cause.getClass();
 while (clazz != null) {
 Errors error = classToError.get(clazz);
 if (error != null)
@@ -479,6 +482,22 @@ public enum Errors {
 return UNKNOWN_SERVER_ERROR;
 }
 
+/**
+ * Check if a Throwable is a commonly wrapped exception type (e.g. 
`CompletionException`) and return
+ * the cause if so. This is useful to handle cases where exceptions may be 
raised from a future or a
+ * completion stage (as might be the case for requests sent to the 
controller in `ControllerApis`).
+ *
+ * @param t The Throwable to check
+ * @return The throwable itself or its cause if it is an instance of a 
commonly wrapped exception type
+ */
+public static Throwable maybeUnwrapException(Throwable t) {
+if (t instanceof CompletionException || t instanceof 
ExecutionException) {
+return t.getCause();
+} else {
+return t;
+}
+}
+
 private static String toHtml() {
 final StringBuilder b = new StringBuilder();
 b.append("\n");
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
index 5d48c39e80..41db29158e 100644
--- 
a/clients/src/main/java/org/apache/kafka/common/requests/AllocateProducerIdsResponse.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/requests/AllocateProducerIdsResponse.java
@@ -56,6 +56,10 @@ public class AllocateProducerIdsResponse extends 
AbstractResponse {
 return data.throttleTimeMs();
 }
 
+public Errors error() {
+return Errors.forCode(data.errorCode());
+}
+
 public static AllocateProducerIdsResponse parse(ByteBuffer buffer, short 
version) {
 return new AllocateProducerIdsResponse(new 
AllocateProducerIdsResponseData(
 new ByteBuff

[kafka] branch trunk updated: KAFKA-14167; Completion exceptions should not be translated directly to error codes (#12518)

2022-08-17 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new bc90c29faf KAFKA-14167; Completion exceptions should not be translated 
directly to error codes (#12518)
bc90c29faf is described below

commit bc90c29fafc69747daeecada8bb0c347e138edc8
Author: Jason Gustafson 
AuthorDate: Wed Aug 17 18:11:42 2022 -0700

KAFKA-14167; Completion exceptions should not be translated directly to 
error codes (#12518)

There are a few cases in `ControllerApis` where we may see an 
`ApiException` wrapped as a `CompletionException`. This can happen in 
`QuorumController.allocateProducerIds` where the returned future is the result 
of calling `thenApply` on the future passed to the controller. The danger when 
this happens is that the `CompletionException` gets passed to 
`Errors.forException`, which translates it to an `UNKNOWN_SERVER_ERROR`. At a 
minimum, I found that the `AllocateProducerIds` and `Update [...]

Interestingly, `DeleteTopics` is not affected by this bug as I originally 
suspected. This is because we have logic in `ApiError.fromThrowable` to check 
for both `CompletionException` and `ExecutionException` and to pull out the 
underlying cause. This patch duplicates this logic from 
`ApiError.fromThrowable` into `Errors.forException` to be sure that we handle 
all cases where exceptions are converted to error codes.

Reviewers: David Arthur 
---
 .../org/apache/kafka/common/protocol/Errors.java   | 21 -
 .../requests/AllocateProducerIdsResponse.java  |  4 +
 .../org/apache/kafka/common/requests/ApiError.java | 10 +--
 .../apache/kafka/common/requests/ApiErrorTest.java |  6 +-
 .../main/scala/kafka/server/ControllerApis.scala   |  7 +-
 .../test/junit/RaftClusterInvocationContext.java   |  4 +-
 .../server/AllocateProducerIdsRequestTest.scala| 98 ++
 .../unit/kafka/server/ControllerApisTest.scala | 78 -
 8 files changed, 205 insertions(+), 23 deletions(-)

diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java 
b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java
index 2ca42bafcf..c220bbcde4 100644
--- a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java
+++ b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java
@@ -132,6 +132,8 @@ import org.slf4j.LoggerFactory;
 
 import java.util.HashMap;
 import java.util.Map;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ExecutionException;
 import java.util.function.Function;
 
 /**
@@ -469,7 +471,8 @@ public enum Errors {
  * If there are multiple matches in the class hierarchy, the first match 
starting from the bottom is used.
  */
 public static Errors forException(Throwable t) {
-Class clazz = t.getClass();
+Throwable cause = maybeUnwrapException(t);
+Class clazz = cause.getClass();
 while (clazz != null) {
 Errors error = classToError.get(clazz);
 if (error != null)
@@ -479,6 +482,22 @@ public enum Errors {
 return UNKNOWN_SERVER_ERROR;
 }
 
+/**
+ * Check if a Throwable is a commonly wrapped exception type (e.g. 
`CompletionException`) and return
+ * the cause if so. This is useful to handle cases where exceptions may be 
raised from a future or a
+ * completion stage (as might be the case for requests sent to the 
controller in `ControllerApis`).
+ *
+ * @param t The Throwable to check
+ * @return The throwable itself or its cause if it is an instance of a 
commonly wrapped exception type
+ */
+public static Throwable maybeUnwrapException(Throwable t) {
+if (t instanceof CompletionException || t instanceof 
ExecutionException) {
+return t.getCause();
+} else {
+return t;
+}
+}
+
 private static String toHtml() {
 final StringBuilder b = new StringBuilder();
 b.append("\n");
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
index 5d48c39e80..41db29158e 100644
--- 
a/clients/src/main/java/org/apache/kafka/common/requests/AllocateProducerIdsResponse.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/requests/AllocateProducerIdsResponse.java
@@ -56,6 +56,10 @@ public class AllocateProducerIdsResponse extends 
AbstractResponse {
 return data.throttleTimeMs();
 }
 
+public Errors error() {
+return Errors.forCode(data.errorCode());
+}
+
 public static AllocateProducerIdsResponse parse(ByteBuffer buffer, short 
version) {
 return new AllocateProducerIdsResponse(new 
AllocateProducerIdsResponseData(
  

[kafka] branch 3.3 updated: KAFKA-13940; Return NOT_LEADER_OR_FOLLOWER if DescribeQuorum sent to non-leader (#12517)

2022-08-17 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch 3.3
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.3 by this push:
 new 0ac9c3498c KAFKA-13940; Return NOT_LEADER_OR_FOLLOWER if 
DescribeQuorum sent to non-leader (#12517)
0ac9c3498c is described below

commit 0ac9c3498c761167db1ead46d89dea71b29981d6
Author: Jason Gustafson 
AuthorDate: Wed Aug 17 15:48:32 2022 -0700

KAFKA-13940; Return NOT_LEADER_OR_FOLLOWER if DescribeQuorum sent to 
non-leader (#12517)

Currently the server will return `INVALID_REQUEST` if a `DescribeQuorum` 
request is sent to a node that is not the current leader. In addition to being 
inconsistent with all of the other leader APIs in the raft layer, this error is 
treated as fatal by both the forwarding manager and the admin client. Instead, 
we should return `NOT_LEADER_OR_FOLLOWER` as we do with the other APIs. This 
error is retriable and we can rely on the admin client to retry it after seeing 
this error.

Reviewers: David Jacot 
---
 .../common/requests/DescribeQuorumResponse.java| 16 +++-
 .../kafka/clients/admin/KafkaAdminClientTest.java  | 23 +
 .../org/apache/kafka/raft/KafkaRaftClient.java |  5 +++-
 .../org/apache/kafka/raft/KafkaRaftClientTest.java | 29 ++
 .../apache/kafka/raft/RaftClientTestContext.java   | 27 ++--
 5 files changed, 85 insertions(+), 15 deletions(-)

diff --git 
a/clients/src/main/java/org/apache/kafka/common/requests/DescribeQuorumResponse.java
 
b/clients/src/main/java/org/apache/kafka/common/requests/DescribeQuorumResponse.java
index cbf945b704..06ae681bc5 100644
--- 
a/clients/src/main/java/org/apache/kafka/common/requests/DescribeQuorumResponse.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/requests/DescribeQuorumResponse.java
@@ -37,7 +37,7 @@ import java.util.Map;
  * - {@link Errors#BROKER_NOT_AVAILABLE}
  *
  * Partition level errors:
- * - {@link Errors#INVALID_REQUEST}
+ * - {@link Errors#NOT_LEADER_OR_FOLLOWER}
  * - {@link Errors#UNKNOWN_TOPIC_OR_PARTITION}
  */
 public class DescribeQuorumResponse extends AbstractResponse {
@@ -72,6 +72,19 @@ public class DescribeQuorumResponse extends AbstractResponse 
{
 return DEFAULT_THROTTLE_TIME;
 }
 
+public static DescribeQuorumResponseData singletonErrorResponse(
+TopicPartition topicPartition,
+Errors error
+) {
+return new DescribeQuorumResponseData()
+.setTopics(Collections.singletonList(new 
DescribeQuorumResponseData.TopicData()
+.setTopicName(topicPartition.topic())
+.setPartitions(Collections.singletonList(new 
DescribeQuorumResponseData.PartitionData()
+.setPartitionIndex(topicPartition.partition())
+.setErrorCode(error.code());
+}
+
+
 public static DescribeQuorumResponseData singletonResponse(TopicPartition 
topicPartition,
int leaderId,
int leaderEpoch,
@@ -82,6 +95,7 @@ public class DescribeQuorumResponse extends AbstractResponse {
 .setTopics(Collections.singletonList(new 
DescribeQuorumResponseData.TopicData()
 .setTopicName(topicPartition.topic())
 .setPartitions(Collections.singletonList(new 
DescribeQuorumResponseData.PartitionData()
+.setPartitionIndex(topicPartition.partition())
 .setErrorCode(Errors.NONE.code())
 .setLeaderId(leaderId)
 .setLeaderEpoch(leaderEpoch)
diff --git 
a/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java
 
b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java
index de57813679..5faf53f075 100644
--- 
a/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java
+++ 
b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java
@@ -5192,6 +5192,29 @@ public class KafkaAdminClientTest {
 }
 }
 
+@Test
+public void testDescribeMetadataQuorumRetriableError() throws Exception {
+try (final AdminClientUnitTestEnv env = mockClientEnv()) {
+
env.kafkaClient().setNodeApiVersions(NodeApiVersions.create(ApiKeys.DESCRIBE_QUORUM.id,
+ApiKeys.DESCRIBE_QUORUM.oldestVersion(),
+ApiKeys.DESCRIBE_QUORUM.latestVersion()));
+
+// First request fails with a NOT_LEADER_OR_FOLLOWER error (which 
is retriable)
+env.kafkaClient().prepareResponse(
+body -> body instanceof DescribeQuorumRequest,
+prepareDescribeQuorumResponse(Errors.NONE, 
Errors.NOT_LEADER_OR_FOLLOWER, false, false, false, false, false));
+
+// The second request succe

[kafka] branch trunk updated: KAFKA-13940; Return NOT_LEADER_OR_FOLLOWER if DescribeQuorum sent to non-leader (#12517)

2022-08-17 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new e5b865d6bf KAFKA-13940; Return NOT_LEADER_OR_FOLLOWER if 
DescribeQuorum sent to non-leader (#12517)
e5b865d6bf is described below

commit e5b865d6bf37495e9949878c8206b9459aa5e1f4
Author: Jason Gustafson 
AuthorDate: Wed Aug 17 15:48:32 2022 -0700

KAFKA-13940; Return NOT_LEADER_OR_FOLLOWER if DescribeQuorum sent to 
non-leader (#12517)

Currently the server will return `INVALID_REQUEST` if a `DescribeQuorum` 
request is sent to a node that is not the current leader. In addition to being 
inconsistent with all of the other leader APIs in the raft layer, this error is 
treated as fatal by both the forwarding manager and the admin client. Instead, 
we should return `NOT_LEADER_OR_FOLLOWER` as we do with the other APIs. This 
error is retriable and we can rely on the admin client to retry it after seeing 
this error.

Reviewers: David Jacot 
---
 .../common/requests/DescribeQuorumResponse.java| 16 +++-
 .../kafka/clients/admin/KafkaAdminClientTest.java  | 23 +
 .../org/apache/kafka/raft/KafkaRaftClient.java |  5 +++-
 .../org/apache/kafka/raft/KafkaRaftClientTest.java | 29 ++
 .../apache/kafka/raft/RaftClientTestContext.java   | 27 ++--
 5 files changed, 85 insertions(+), 15 deletions(-)

diff --git 
a/clients/src/main/java/org/apache/kafka/common/requests/DescribeQuorumResponse.java
 
b/clients/src/main/java/org/apache/kafka/common/requests/DescribeQuorumResponse.java
index cbf945b704..06ae681bc5 100644
--- 
a/clients/src/main/java/org/apache/kafka/common/requests/DescribeQuorumResponse.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/requests/DescribeQuorumResponse.java
@@ -37,7 +37,7 @@ import java.util.Map;
  * - {@link Errors#BROKER_NOT_AVAILABLE}
  *
  * Partition level errors:
- * - {@link Errors#INVALID_REQUEST}
+ * - {@link Errors#NOT_LEADER_OR_FOLLOWER}
  * - {@link Errors#UNKNOWN_TOPIC_OR_PARTITION}
  */
 public class DescribeQuorumResponse extends AbstractResponse {
@@ -72,6 +72,19 @@ public class DescribeQuorumResponse extends AbstractResponse 
{
 return DEFAULT_THROTTLE_TIME;
 }
 
+public static DescribeQuorumResponseData singletonErrorResponse(
+TopicPartition topicPartition,
+Errors error
+) {
+return new DescribeQuorumResponseData()
+.setTopics(Collections.singletonList(new 
DescribeQuorumResponseData.TopicData()
+.setTopicName(topicPartition.topic())
+.setPartitions(Collections.singletonList(new 
DescribeQuorumResponseData.PartitionData()
+.setPartitionIndex(topicPartition.partition())
+.setErrorCode(error.code());
+}
+
+
 public static DescribeQuorumResponseData singletonResponse(TopicPartition 
topicPartition,
int leaderId,
int leaderEpoch,
@@ -82,6 +95,7 @@ public class DescribeQuorumResponse extends AbstractResponse {
 .setTopics(Collections.singletonList(new 
DescribeQuorumResponseData.TopicData()
 .setTopicName(topicPartition.topic())
 .setPartitions(Collections.singletonList(new 
DescribeQuorumResponseData.PartitionData()
+.setPartitionIndex(topicPartition.partition())
 .setErrorCode(Errors.NONE.code())
 .setLeaderId(leaderId)
 .setLeaderEpoch(leaderEpoch)
diff --git 
a/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java
 
b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java
index de57813679..5faf53f075 100644
--- 
a/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java
+++ 
b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java
@@ -5192,6 +5192,29 @@ public class KafkaAdminClientTest {
 }
 }
 
+@Test
+public void testDescribeMetadataQuorumRetriableError() throws Exception {
+try (final AdminClientUnitTestEnv env = mockClientEnv()) {
+
env.kafkaClient().setNodeApiVersions(NodeApiVersions.create(ApiKeys.DESCRIBE_QUORUM.id,
+ApiKeys.DESCRIBE_QUORUM.oldestVersion(),
+ApiKeys.DESCRIBE_QUORUM.latestVersion()));
+
+// First request fails with a NOT_LEADER_OR_FOLLOWER error (which 
is retriable)
+env.kafkaClient().prepareResponse(
+body -> body instanceof DescribeQuorumRequest,
+prepareDescribeQuorumResponse(Errors.NONE, 
Errors.NOT_LEADER_OR_FOLLOWER, false, false, false, false, false));
+
+// The second request succe

[kafka] branch trunk updated: HOTFIX: Revert KAFKA-10199 which is causing compilation failures (#12532)

2022-08-17 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 0243bb98a7 HOTFIX: Revert KAFKA-10199 which is causing compilation 
failures (#12532)
0243bb98a7 is described below

commit 0243bb98a7cc6bc4ed3c8f39b3a1ed9f70d0f8bd
Author: Jason Gustafson 
AuthorDate: Wed Aug 17 14:29:49 2022 -0700

HOTFIX: Revert KAFKA-10199 which is causing compilation failures (#12532)

Compilation is failing after these two commits:
```
> Task :streams:compileJava

/Users/jgustafson/Projects/kafka/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java:852:
 error: cannot find symbol
tasks.addPendingTaskToClose(restoringTask.id());
 ^
  symbol:   method 
addPendingTaskToClose(org.apache.kafka.streams.processor.TaskId)
  location: variable tasks of type 
org.apache.kafka.streams.processor.internals.Tasks
1 error
```

Also here:
```

[2022-08-17T20:58:20.912Z] > Task :streams:compileTestJava

[2022-08-17T20:58:20.912Z] 
/home/jenkins/jenkins-agent/workspace/Kafka_kafka-pr_PR-12530/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java:822:
 error: method setupForRevocation(Set,Set) is already defined in 
class TaskManagerTest

[2022-08-17T20:58:20.912Z] private TaskManager setupForRevocation(final 
Set tasksInStateUpdater,
```
 This patch reverts them.

Reviewers: Ismael Juma 
---
 .../streams/processor/internals/TaskManager.java   |  42 +--
 .../kafka/streams/processor/internals/Tasks.java   |  19 +--
 .../processor/internals/TaskManagerTest.java   | 129 +
 3 files changed, 12 insertions(+), 178 deletions(-)

diff --git 
a/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java
 
b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java
index bab05a5184..03c36b0daf 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java
@@ -465,7 +465,7 @@ public class TaskManager {
 standbyTasksToCreate.remove(taskId);
 } else {
 stateUpdater.remove(taskId);
-tasks.addPendingTaskToCloseClean(taskId);
+tasks.addPendingTaskToClose(taskId);
 }
 }
 }
@@ -692,7 +692,7 @@ public class TaskManager {
 
 taskExceptions.putIfAbsent(taskId, e);
 }
-} else if (tasks.removePendingTaskToCloseClean(task.id())) {
+} else if (tasks.removePendingTaskToClose(task.id())) {
 try {
 task.suspend();
 task.closeClean();
@@ -710,8 +710,6 @@ public class TaskManager {
 
 taskExceptions.putIfAbsent(task.id(), e);
 }
-} else if (tasks.removePendingTaskToCloseDirty(task.id())) {
-tasksToCloseDirty.add(task);
 } else if ((inputPartitions = 
tasks.removePendingTaskToUpdateInputPartitions(task.id())) != null) {
 task.updateInputPartitions(inputPartitions, 
topologyMetadata.nodeToSourceTopics(task.id()));
 stateUpdater.add(task);
@@ -757,8 +755,6 @@ public class TaskManager {
 }
 }
 
-removeRevokedTasksFromStateUpdater(remainingRevokedPartitions);
-
 if (!remainingRevokedPartitions.isEmpty()) {
 log.debug("The following revoked partitions {} are missing from 
the current task partitions. It could "
   + "potentially be due to race condition of consumer 
detecting the heartbeat failure, or the tasks " +
@@ -844,20 +840,6 @@ public class TaskManager {
 }
 }
 
-private void removeRevokedTasksFromStateUpdater(final Set 
remainingRevokedPartitions) {
-if (stateUpdater != null) {
-for (final Task restoringTask : stateUpdater.getTasks()) {
-if (restoringTask.isActive()) {
-if 
(remainingRevokedPartitions.containsAll(restoringTask.inputPartitions())) {
-tasks.addPendingTaskToClose(restoringTask.id());
-stateUpdater.remove(restoringTask.id());
-
remainingRevokedPartitions.removeAll(restoringTask.inputPartitions());
-}
-}
-}
-}
-}
-
 private void prepareCommitAndAddOffsetsToMap(final Set 
tasksToPrepare,
  final Map> consumedOffsetsPerTask) {
 for (final Task task : tasksToPrepare) {
@@ -885,15 +867,

[kafka] branch 3.3 updated: KAFKA-14154; Kraft controller should return NOT_CONTROLLER if request epoch is ahead (#12514)

2022-08-15 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch 3.3
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.3 by this push:
 new 16a7713830 KAFKA-14154; Kraft controller should return NOT_CONTROLLER 
if request epoch is ahead (#12514)
16a7713830 is described below

commit 16a77138305c429b3adc62b34c128eb242cadfb5
Author: Jason Gustafson 
AuthorDate: Mon Aug 15 11:34:29 2022 -0700

KAFKA-14154; Kraft controller should return NOT_CONTROLLER if request epoch 
is ahead (#12514)

Similar to https://github.com/apache/kafka/pull/12506. For the Kraft 
controller, we should return NOT_CONTROLLER if the leader/partition epoch in 
the request is ahead of the controller.

Reviewers: José Armando García Sancio 
---
 .../kafka/controller/ReplicationControlManager.java | 21 +++--
 .../controller/ReplicationControlManagerTest.java   | 12 +++-
 2 files changed, 30 insertions(+), 3 deletions(-)

diff --git 
a/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java
 
b/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java
index 4ffb339967..df7097df83 100644
--- 
a/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java
+++ 
b/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java
@@ -125,6 +125,7 @@ import static 
org.apache.kafka.common.protocol.Errors.INVALID_REQUEST;
 import static org.apache.kafka.common.protocol.Errors.INVALID_UPDATE_VERSION;
 import static org.apache.kafka.common.protocol.Errors.NEW_LEADER_ELECTED;
 import static org.apache.kafka.common.protocol.Errors.NONE;
+import static org.apache.kafka.common.protocol.Errors.NOT_CONTROLLER;
 import static 
org.apache.kafka.common.protocol.Errors.NO_REASSIGNMENT_IN_PROGRESS;
 import static org.apache.kafka.common.protocol.Errors.OPERATION_NOT_ATTEMPTED;
 import static 
org.apache.kafka.common.protocol.Errors.TOPIC_AUTHORIZATION_FAILED;
@@ -1071,7 +1072,23 @@ public class ReplicationControlManager {
 
 return UNKNOWN_TOPIC_OR_PARTITION;
 }
-if (partitionData.leaderEpoch() != partition.leaderEpoch) {
+
+// If the partition leader has a higher leader/partition epoch, then 
it is likely
+// that this node is no longer the active controller. We return 
NOT_CONTROLLER in
+// this case to give the leader an opportunity to find the new 
controller.
+if (partitionData.leaderEpoch() > partition.leaderEpoch) {
+log.debug("Rejecting AlterPartition request from node {} for {}-{} 
because " +
+"the current leader epoch is {}, which is greater than the 
local value {}.",
+brokerId, topic.name, partitionId, partition.leaderEpoch, 
partitionData.leaderEpoch());
+return NOT_CONTROLLER;
+}
+if (partitionData.partitionEpoch() > partition.partitionEpoch) {
+log.debug("Rejecting AlterPartition request from node {} for {}-{} 
because " +
+"the current partition epoch is {}, which is greater than 
the local value {}.",
+brokerId, topic.name, partitionId, partition.partitionEpoch, 
partitionData.partitionEpoch());
+return NOT_CONTROLLER;
+}
+if (partitionData.leaderEpoch() < partition.leaderEpoch) {
 log.debug("Rejecting AlterPartition request from node {} for {}-{} 
because " +
 "the current leader epoch is {}, not {}.", brokerId, 
topic.name,
 partitionId, partition.leaderEpoch, 
partitionData.leaderEpoch());
@@ -1085,7 +1102,7 @@ public class ReplicationControlManager {
 
 return INVALID_REQUEST;
 }
-if (partitionData.partitionEpoch() != partition.partitionEpoch) {
+if (partitionData.partitionEpoch() < partition.partitionEpoch) {
 log.info("Rejecting AlterPartition request from node {} for {}-{} 
because " +
 "the current partition epoch is {}, not {}.", brokerId,
 topic.name, partitionId, partition.partitionEpoch,
diff --git 
a/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java
 
b/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java
index d33776ca10..82e378a982 100644
--- 
a/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java
+++ 
b/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java
@@ -122,6 +122,7 @@ import static 
org.apache.kafka.common.protocol.Errors.INVALID_REPLICA_ASSIGNMENT
 import static org.apache.kafka.common.protocol.Errors.INVALID_TOPIC_EXCEPTION;
 import static org.apache.kafka.common.protocol.Errors.NEW

[kafka] branch trunk updated: KAFKA-14154; Kraft controller should return NOT_CONTROLLER if request epoch is ahead (#12514)

2022-08-15 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 5990471b8ca KAFKA-14154; Kraft controller should return NOT_CONTROLLER 
if request epoch is ahead (#12514)
5990471b8ca is described below

commit 5990471b8ca0dc275c9a8ff0cde2a6562ee8199e
Author: Jason Gustafson 
AuthorDate: Mon Aug 15 11:34:29 2022 -0700

KAFKA-14154; Kraft controller should return NOT_CONTROLLER if request epoch 
is ahead (#12514)

Similar to https://github.com/apache/kafka/pull/12506. For the Kraft 
controller, we should return NOT_CONTROLLER if the leader/partition epoch in 
the request is ahead of the controller.

Reviewers: José Armando García Sancio 
---
 .../kafka/controller/ReplicationControlManager.java | 21 +++--
 .../controller/ReplicationControlManagerTest.java   | 12 +++-
 2 files changed, 30 insertions(+), 3 deletions(-)

diff --git 
a/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java
 
b/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java
index 4ffb339967c..df7097df83d 100644
--- 
a/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java
+++ 
b/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java
@@ -125,6 +125,7 @@ import static 
org.apache.kafka.common.protocol.Errors.INVALID_REQUEST;
 import static org.apache.kafka.common.protocol.Errors.INVALID_UPDATE_VERSION;
 import static org.apache.kafka.common.protocol.Errors.NEW_LEADER_ELECTED;
 import static org.apache.kafka.common.protocol.Errors.NONE;
+import static org.apache.kafka.common.protocol.Errors.NOT_CONTROLLER;
 import static 
org.apache.kafka.common.protocol.Errors.NO_REASSIGNMENT_IN_PROGRESS;
 import static org.apache.kafka.common.protocol.Errors.OPERATION_NOT_ATTEMPTED;
 import static 
org.apache.kafka.common.protocol.Errors.TOPIC_AUTHORIZATION_FAILED;
@@ -1071,7 +1072,23 @@ public class ReplicationControlManager {
 
 return UNKNOWN_TOPIC_OR_PARTITION;
 }
-if (partitionData.leaderEpoch() != partition.leaderEpoch) {
+
+// If the partition leader has a higher leader/partition epoch, then 
it is likely
+// that this node is no longer the active controller. We return 
NOT_CONTROLLER in
+// this case to give the leader an opportunity to find the new 
controller.
+if (partitionData.leaderEpoch() > partition.leaderEpoch) {
+log.debug("Rejecting AlterPartition request from node {} for {}-{} 
because " +
+"the current leader epoch is {}, which is greater than the 
local value {}.",
+brokerId, topic.name, partitionId, partition.leaderEpoch, 
partitionData.leaderEpoch());
+return NOT_CONTROLLER;
+}
+if (partitionData.partitionEpoch() > partition.partitionEpoch) {
+log.debug("Rejecting AlterPartition request from node {} for {}-{} 
because " +
+"the current partition epoch is {}, which is greater than 
the local value {}.",
+brokerId, topic.name, partitionId, partition.partitionEpoch, 
partitionData.partitionEpoch());
+return NOT_CONTROLLER;
+}
+if (partitionData.leaderEpoch() < partition.leaderEpoch) {
 log.debug("Rejecting AlterPartition request from node {} for {}-{} 
because " +
 "the current leader epoch is {}, not {}.", brokerId, 
topic.name,
 partitionId, partition.leaderEpoch, 
partitionData.leaderEpoch());
@@ -1085,7 +1102,7 @@ public class ReplicationControlManager {
 
 return INVALID_REQUEST;
 }
-if (partitionData.partitionEpoch() != partition.partitionEpoch) {
+if (partitionData.partitionEpoch() < partition.partitionEpoch) {
 log.info("Rejecting AlterPartition request from node {} for {}-{} 
because " +
 "the current partition epoch is {}, not {}.", brokerId,
 topic.name, partitionId, partition.partitionEpoch,
diff --git 
a/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java
 
b/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java
index d33776ca10e..82e378a9823 100644
--- 
a/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java
+++ 
b/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java
@@ -122,6 +122,7 @@ import static 
org.apache.kafka.common.protocol.Errors.INVALID_REPLICA_ASSIGNMENT
 import static org.apache.kafka.common.protocol.Errors.INVALID_TOPIC_EXCEPTION;
 import static org.apache.kafka.common.protocol.Errors.NEW

[kafka] branch 3.3 updated: KAFKA-14154; Return NOT_CONTROLLER from AlterPartition if leader is ahead of controller (#12506)

2022-08-11 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch 3.3
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.3 by this push:
 new 36ff5d5c98f KAFKA-14154; Return NOT_CONTROLLER from AlterPartition if 
leader is ahead of controller (#12506)
36ff5d5c98f is described below

commit 36ff5d5c98f2649ec2cc5ab302375cd41298a727
Author: Jason Gustafson 
AuthorDate: Thu Aug 11 16:43:12 2022 -0700

KAFKA-14154; Return NOT_CONTROLLER from AlterPartition if leader is ahead 
of controller (#12506)

It is possible for the leader to send an `AlterPartition` request to a 
zombie controller which includes either a partition or leader epoch which is 
larger than what is found in the controller context. Prior to 
https://github.com/apache/kafka/pull/12032, the controller handled this in the 
following way:

1. If the `LeaderAndIsr` state exactly matches the current state on the 
controller excluding the partition epoch, then the `AlterPartition` request is 
considered successful and no error is returned. The risk with this handling is 
that this may cause the leader to incorrectly assume that the state had been 
successfully updated. Since the controller's state is stale, there is no way to 
know what the latest ISR state is.
2. Otherwise, the controller will attempt to update the state in zookeeper 
with the leader/partition epochs from the `AlterPartition` request. This 
operation would fail if the controller's epoch was not still current in 
Zookeeper and the result would be a `NOT_CONTROLLER` error.

Following https://github.com/apache/kafka/pull/12032, the controller's 
validation is stricter. If the partition epoch is larger than expected, then 
the controller will return `INVALID_UPDATE_VERSION` without attempting the 
operation. Similarly, if the leader epoch is larger than expected, the 
controller will return `FENCED_LEADER_EPOCH`. The problem with this new 
handling is that the leader treats the errors from the controller as 
authoritative. For example, if it sees the `FENCED_LEA [...]

In this patch, we want to fix the issues with this handling, but we don't 
want to restore the buggy idempotent check. The approach is straightforward. If 
the controller sees a partition/leader epoch which is larger than what it has 
in the controller context, then it assumes that has become a zombie and returns 
`NOT_CONTROLLER` to the leader. This will cause the leader to attempt to reset 
the controller from its local metadata cache and retry the `AlterPartition` 
request.

Reviewers: David Jacot , José Armando García Sancio 

---
 .../scala/kafka/controller/KafkaController.scala   |  9 +++-
 .../controller/ControllerIntegrationTest.scala | 27 ++
 2 files changed, 30 insertions(+), 6 deletions(-)

diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala 
b/core/src/main/scala/kafka/controller/KafkaController.scala
index 0154d9cbe54..999bcb818e9 100644
--- a/core/src/main/scala/kafka/controller/KafkaController.scala
+++ b/core/src/main/scala/kafka/controller/KafkaController.scala
@@ -2336,7 +2336,14 @@ class KafkaController(val config: KafkaConfig,
   controllerContext.partitionLeadershipInfo(tp) match {
 case Some(leaderIsrAndControllerEpoch) =>
   val currentLeaderAndIsr = leaderIsrAndControllerEpoch.leaderAndIsr
-  if (newLeaderAndIsr.leaderEpoch != currentLeaderAndIsr.leaderEpoch) {
+  if (newLeaderAndIsr.partitionEpoch > 
currentLeaderAndIsr.partitionEpoch
+  || newLeaderAndIsr.leaderEpoch > 
currentLeaderAndIsr.leaderEpoch) {
+// If the partition leader has a higher partition/leader epoch, 
then it is likely
+// that this node is no longer the active controller. We return 
NOT_CONTROLLER in
+// this case to give the leader an opportunity to find the new 
controller.
+partitionResponses(tp) = Left(Errors.NOT_CONTROLLER)
+None
+  } else if (newLeaderAndIsr.leaderEpoch != 
currentLeaderAndIsr.leaderEpoch) {
 partitionResponses(tp) = Left(Errors.FENCED_LEADER_EPOCH)
 None
   } else if 
(newLeaderAndIsr.equalsAllowStalePartitionEpoch(currentLeaderAndIsr)) {
diff --git 
a/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala 
b/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala
index 0c8d000656a..532ff1a946e 100644
--- a/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala
+++ b/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala
@@ -1184,7 +1184,7 @@ class ControllerIntegrationTest extends QuorumTestHarness 
{
 )
 
 assertAlterPartition(
-  partitionError = Errors.INVALID_UPDATE_VERSION,
+  partitionError = Errors.NOT_CONTROLLER,
   partitionEpoch = partition

[kafka] branch trunk updated: KAFKA-14154; Return NOT_CONTROLLER from AlterPartition if leader is ahead of controller (#12506)

2022-08-11 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 520f72995dd KAFKA-14154; Return NOT_CONTROLLER from AlterPartition if 
leader is ahead of controller (#12506)
520f72995dd is described below

commit 520f72995ddc5c687e364b83c7ace45f14fdd701
Author: Jason Gustafson 
AuthorDate: Thu Aug 11 16:43:12 2022 -0700

KAFKA-14154; Return NOT_CONTROLLER from AlterPartition if leader is ahead 
of controller (#12506)

It is possible for the leader to send an `AlterPartition` request to a 
zombie controller which includes either a partition or leader epoch which is 
larger than what is found in the controller context. Prior to 
https://github.com/apache/kafka/pull/12032, the controller handled this in the 
following way:

1. If the `LeaderAndIsr` state exactly matches the current state on the 
controller excluding the partition epoch, then the `AlterPartition` request is 
considered successful and no error is returned. The risk with this handling is 
that this may cause the leader to incorrectly assume that the state had been 
successfully updated. Since the controller's state is stale, there is no way to 
know what the latest ISR state is.
2. Otherwise, the controller will attempt to update the state in zookeeper 
with the leader/partition epochs from the `AlterPartition` request. This 
operation would fail if the controller's epoch was not still current in 
Zookeeper and the result would be a `NOT_CONTROLLER` error.

Following https://github.com/apache/kafka/pull/12032, the controller's 
validation is stricter. If the partition epoch is larger than expected, then 
the controller will return `INVALID_UPDATE_VERSION` without attempting the 
operation. Similarly, if the leader epoch is larger than expected, the 
controller will return `FENCED_LEADER_EPOCH`. The problem with this new 
handling is that the leader treats the errors from the controller as 
authoritative. For example, if it sees the `FENCED_LEA [...]

In this patch, we want to fix the issues with this handling, but we don't 
want to restore the buggy idempotent check. The approach is straightforward. If 
the controller sees a partition/leader epoch which is larger than what it has 
in the controller context, then it assumes that has become a zombie and returns 
`NOT_CONTROLLER` to the leader. This will cause the leader to attempt to reset 
the controller from its local metadata cache and retry the `AlterPartition` 
request.

Reviewers: David Jacot , José Armando García Sancio 

---
 .../scala/kafka/controller/KafkaController.scala   |  9 +++-
 .../controller/ControllerIntegrationTest.scala | 27 ++
 2 files changed, 30 insertions(+), 6 deletions(-)

diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala 
b/core/src/main/scala/kafka/controller/KafkaController.scala
index 0154d9cbe54..999bcb818e9 100644
--- a/core/src/main/scala/kafka/controller/KafkaController.scala
+++ b/core/src/main/scala/kafka/controller/KafkaController.scala
@@ -2336,7 +2336,14 @@ class KafkaController(val config: KafkaConfig,
   controllerContext.partitionLeadershipInfo(tp) match {
 case Some(leaderIsrAndControllerEpoch) =>
   val currentLeaderAndIsr = leaderIsrAndControllerEpoch.leaderAndIsr
-  if (newLeaderAndIsr.leaderEpoch != currentLeaderAndIsr.leaderEpoch) {
+  if (newLeaderAndIsr.partitionEpoch > 
currentLeaderAndIsr.partitionEpoch
+  || newLeaderAndIsr.leaderEpoch > 
currentLeaderAndIsr.leaderEpoch) {
+// If the partition leader has a higher partition/leader epoch, 
then it is likely
+// that this node is no longer the active controller. We return 
NOT_CONTROLLER in
+// this case to give the leader an opportunity to find the new 
controller.
+partitionResponses(tp) = Left(Errors.NOT_CONTROLLER)
+None
+  } else if (newLeaderAndIsr.leaderEpoch != 
currentLeaderAndIsr.leaderEpoch) {
 partitionResponses(tp) = Left(Errors.FENCED_LEADER_EPOCH)
 None
   } else if 
(newLeaderAndIsr.equalsAllowStalePartitionEpoch(currentLeaderAndIsr)) {
diff --git 
a/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala 
b/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala
index 0c8d000656a..532ff1a946e 100644
--- a/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala
+++ b/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala
@@ -1184,7 +1184,7 @@ class ControllerIntegrationTest extends QuorumTestHarness 
{
 )
 
 assertAlterPartition(
-  partitionError = Errors.INVALID_UPDATE_VERSION,
+  partitionError = Errors.NOT_CONTROLLER,
   partitionEpoch = partition

[kafka] branch 3.3 updated: KAFKA-13986; Brokers should include node.id in fetches to metadata quorum (#12498)

2022-08-11 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch 3.3
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.3 by this push:
 new 44229581ed8 KAFKA-13986; Brokers should include node.id in fetches to 
metadata quorum (#12498)
44229581ed8 is described below

commit 44229581ed8994416c41ac6584c150564185f0da
Author: Jason Gustafson 
AuthorDate: Thu Aug 11 14:08:37 2022 -0700

KAFKA-13986; Brokers should include node.id in fetches to metadata quorum 
(#12498)

Currently we do not set the replicaId in fetches from brokers to the 
metadata quorum. It is useful to do so since that allows us to debug 
replication using the `DescribeQuorum` API.

Reviewers: dengziming , José Armando García 
Sancio 
---
 core/src/main/scala/kafka/raft/RaftManager.scala  |  8 +---
 core/src/test/java/kafka/test/ClusterInstance.java| 13 +
 .../kafka/test/junit/RaftClusterInvocationContext.java| 15 +++
 .../java/kafka/test/junit/ZkClusterInvocationContext.java | 13 +
 core/src/test/scala/unit/kafka/raft/RaftManagerTest.scala |  8 
 .../unit/kafka/server/DescribeQuorumRequestTest.scala |  9 +
 6 files changed, 51 insertions(+), 15 deletions(-)

diff --git a/core/src/main/scala/kafka/raft/RaftManager.scala 
b/core/src/main/scala/kafka/raft/RaftManager.scala
index cbb9f7b89bf..a44d9d8fe01 100644
--- a/core/src/main/scala/kafka/raft/RaftManager.scala
+++ b/core/src/main/scala/kafka/raft/RaftManager.scala
@@ -24,7 +24,6 @@ import java.util.concurrent.CompletableFuture
 import kafka.log.UnifiedLog
 import kafka.raft.KafkaRaftManager.RaftIoThread
 import kafka.server.{KafkaConfig, MetaProperties}
-import kafka.server.KafkaRaftServer.ControllerRole
 import kafka.utils.timer.SystemTimer
 import kafka.utils.{KafkaScheduler, Logging, ShutdownableThread}
 import org.apache.kafka.clients.{ApiVersions, ManualMetadataUpdater, 
NetworkClient}
@@ -181,12 +180,7 @@ class KafkaRaftManager[T](
 val expirationTimer = new SystemTimer("raft-expiration-executor")
 val expirationService = new TimingWheelExpirationService(expirationTimer)
 val quorumStateStore = new FileBasedStateStore(new File(dataDir, 
"quorum-state"))
-
-val nodeId = if (config.processRoles.contains(ControllerRole)) {
-  OptionalInt.of(config.nodeId)
-} else {
-  OptionalInt.empty()
-}
+val nodeId = OptionalInt.of(config.nodeId)
 
 val client = new KafkaRaftClient(
   recordSerde,
diff --git a/core/src/test/java/kafka/test/ClusterInstance.java 
b/core/src/test/java/kafka/test/ClusterInstance.java
index a7052857c36..9058508fa94 100644
--- a/core/src/test/java/kafka/test/ClusterInstance.java
+++ b/core/src/test/java/kafka/test/ClusterInstance.java
@@ -27,6 +27,7 @@ import java.util.Collection;
 import java.util.Map;
 import java.util.Optional;
 import java.util.Properties;
+import java.util.Set;
 
 public interface ClusterInstance {
 
@@ -50,6 +51,18 @@ public interface ClusterInstance {
  */
 ClusterConfig config();
 
+/**
+ * Return the set of all controller IDs configured for this test. For 
kraft, this
+ * will return only the nodes which have the "controller" role enabled in 
`process.roles`.
+ * For zookeeper, this will return all broker IDs since they are all 
eligible controllers.
+ */
+Set controllerIds();
+
+/**
+ * Return the set of all broker IDs configured for this test.
+ */
+Set brokerIds();
+
 /**
  * The listener for this cluster as configured by {@link ClusterTest} or 
by {@link ClusterConfig}. If
  * unspecified by those sources, this will return the listener for the 
default security protocol PLAINTEXT
diff --git 
a/core/src/test/java/kafka/test/junit/RaftClusterInvocationContext.java 
b/core/src/test/java/kafka/test/junit/RaftClusterInvocationContext.java
index cef71042d3f..5cd3ec3e246 100644
--- a/core/src/test/java/kafka/test/junit/RaftClusterInvocationContext.java
+++ b/core/src/test/java/kafka/test/junit/RaftClusterInvocationContext.java
@@ -42,6 +42,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Optional;
 import java.util.Properties;
+import java.util.Set;
 import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.atomic.AtomicBoolean;
@@ -192,6 +193,20 @@ public class RaftClusterInvocationContext implements 
TestTemplateInvocationConte
 return clusterConfig;
 }
 
+@Override
+public Set controllerIds() {
+return controllers()
+.map(controllerServer -> controllerServer.config().nodeId())
+.collect(Collectors.toSet());
+}
+
+@Override
+public Set brokerIds() {
+return brokers()
+.map(brokerServe

[kafka] branch trunk updated: KAFKA-13986; Brokers should include node.id in fetches to metadata quorum (#12498)

2022-08-11 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 24b0ebbac16 KAFKA-13986; Brokers should include node.id in fetches to 
metadata quorum (#12498)
24b0ebbac16 is described below

commit 24b0ebbac1676c7a90181bfb83b7297fab8729ae
Author: Jason Gustafson 
AuthorDate: Thu Aug 11 14:08:37 2022 -0700

KAFKA-13986; Brokers should include node.id in fetches to metadata quorum 
(#12498)

Currently we do not set the replicaId in fetches from brokers to the 
metadata quorum. It is useful to do so since that allows us to debug 
replication using the `DescribeQuorum` API.

Reviewers: dengziming , José Armando García 
Sancio 
---
 core/src/main/scala/kafka/raft/RaftManager.scala  |  8 +---
 core/src/test/java/kafka/test/ClusterInstance.java| 13 +
 .../kafka/test/junit/RaftClusterInvocationContext.java| 15 +++
 .../java/kafka/test/junit/ZkClusterInvocationContext.java | 13 +
 core/src/test/scala/unit/kafka/raft/RaftManagerTest.scala |  8 
 .../unit/kafka/server/DescribeQuorumRequestTest.scala |  9 +
 6 files changed, 51 insertions(+), 15 deletions(-)

diff --git a/core/src/main/scala/kafka/raft/RaftManager.scala 
b/core/src/main/scala/kafka/raft/RaftManager.scala
index cbb9f7b89bf..a44d9d8fe01 100644
--- a/core/src/main/scala/kafka/raft/RaftManager.scala
+++ b/core/src/main/scala/kafka/raft/RaftManager.scala
@@ -24,7 +24,6 @@ import java.util.concurrent.CompletableFuture
 import kafka.log.UnifiedLog
 import kafka.raft.KafkaRaftManager.RaftIoThread
 import kafka.server.{KafkaConfig, MetaProperties}
-import kafka.server.KafkaRaftServer.ControllerRole
 import kafka.utils.timer.SystemTimer
 import kafka.utils.{KafkaScheduler, Logging, ShutdownableThread}
 import org.apache.kafka.clients.{ApiVersions, ManualMetadataUpdater, 
NetworkClient}
@@ -181,12 +180,7 @@ class KafkaRaftManager[T](
 val expirationTimer = new SystemTimer("raft-expiration-executor")
 val expirationService = new TimingWheelExpirationService(expirationTimer)
 val quorumStateStore = new FileBasedStateStore(new File(dataDir, 
"quorum-state"))
-
-val nodeId = if (config.processRoles.contains(ControllerRole)) {
-  OptionalInt.of(config.nodeId)
-} else {
-  OptionalInt.empty()
-}
+val nodeId = OptionalInt.of(config.nodeId)
 
 val client = new KafkaRaftClient(
   recordSerde,
diff --git a/core/src/test/java/kafka/test/ClusterInstance.java 
b/core/src/test/java/kafka/test/ClusterInstance.java
index a7052857c36..9058508fa94 100644
--- a/core/src/test/java/kafka/test/ClusterInstance.java
+++ b/core/src/test/java/kafka/test/ClusterInstance.java
@@ -27,6 +27,7 @@ import java.util.Collection;
 import java.util.Map;
 import java.util.Optional;
 import java.util.Properties;
+import java.util.Set;
 
 public interface ClusterInstance {
 
@@ -50,6 +51,18 @@ public interface ClusterInstance {
  */
 ClusterConfig config();
 
+/**
+ * Return the set of all controller IDs configured for this test. For 
kraft, this
+ * will return only the nodes which have the "controller" role enabled in 
`process.roles`.
+ * For zookeeper, this will return all broker IDs since they are all 
eligible controllers.
+ */
+Set controllerIds();
+
+/**
+ * Return the set of all broker IDs configured for this test.
+ */
+Set brokerIds();
+
 /**
  * The listener for this cluster as configured by {@link ClusterTest} or 
by {@link ClusterConfig}. If
  * unspecified by those sources, this will return the listener for the 
default security protocol PLAINTEXT
diff --git 
a/core/src/test/java/kafka/test/junit/RaftClusterInvocationContext.java 
b/core/src/test/java/kafka/test/junit/RaftClusterInvocationContext.java
index cef71042d3f..5cd3ec3e246 100644
--- a/core/src/test/java/kafka/test/junit/RaftClusterInvocationContext.java
+++ b/core/src/test/java/kafka/test/junit/RaftClusterInvocationContext.java
@@ -42,6 +42,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Optional;
 import java.util.Properties;
+import java.util.Set;
 import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.atomic.AtomicBoolean;
@@ -192,6 +193,20 @@ public class RaftClusterInvocationContext implements 
TestTemplateInvocationConte
 return clusterConfig;
 }
 
+@Override
+public Set controllerIds() {
+return controllers()
+.map(controllerServer -> controllerServer.config().nodeId())
+.collect(Collectors.toSet());
+}
+
+@Override
+public Set brokerIds() {
+return brokers()
+.map(brokerServe

[kafka] branch trunk updated: KAFKA-14163; Retry compilation after zinc compile cache error (#12507)

2022-08-11 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 3d9a5f6ae3a KAFKA-14163; Retry compilation after zinc compile cache 
error (#12507)
3d9a5f6ae3a is described below

commit 3d9a5f6ae3a5c67fb60520664e33a1c904dd7a90
Author: Jason Gustafson 
AuthorDate: Thu Aug 11 14:00:06 2022 -0700

KAFKA-14163; Retry compilation after zinc compile cache error (#12507)

We have been seeing builds fail due to errors such as the following:
```
Timeout waiting to lock zinc-1.6.1_2.13.8_17 compiler cache 
(/home/jenkins/.gradle/caches/7.5.1/zinc-1.6.1_2.13.8_17). It is currently in 
use by another Gradle instance.
```
This patch includes a workaround: if the compilation fails due to this zinc 
compile cache error, then we retry it.

Reviewers: Ismael Juma 

Co-authored-by: Lucas Bradstreet 
---
 Jenkinsfile |  2 +-
 retry_zinc  | 49 +
 2 files changed, 50 insertions(+), 1 deletion(-)

diff --git a/Jenkinsfile b/Jenkinsfile
index cce625b8d8e..e1b4c5b3a1d 100644
--- a/Jenkinsfile
+++ b/Jenkinsfile
@@ -19,7 +19,7 @@
 
 def doValidation() {
   sh """
-./gradlew -PscalaVersion=$SCALA_VERSION clean compileJava compileScala 
compileTestJava compileTestScala \
+./retry_zinc ./gradlew -PscalaVersion=$SCALA_VERSION clean compileJava 
compileScala compileTestJava compileTestScala \
 spotlessScalaCheck checkstyleMain checkstyleTest spotbugsMain rat \
 --profile --no-daemon --continue -PxmlSpotBugsReport=true
   """
diff --git a/retry_zinc b/retry_zinc
new file mode 100755
index 000..9d7b5553fa9
--- /dev/null
+++ b/retry_zinc
@@ -0,0 +1,49 @@
+#!/bin/bash
+# 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.
+
+# Hacky workaround for https://github.com/gradle/gradle/issues/3777
+# There is currently no configurable timeout, so we retry builds jenkins when 
we can't get a lock on the zinc compiler cache
+# Hopefully we can remove this in the future, but this will save us from 
having to manually rebuild for the time being.
+# Example:
+# [2021-10-19T17:25:07.234Z] * What went wrong:
+# [2021-10-19T17:25:07.234Z] Execution failed for task 
':streams:streams-scala:compileScala'.
+# [2021-10-19T17:25:07.234Z] > Timeout waiting to lock zinc-1.3.5_2.13.6_8 
compiler cache (/home/jenkins/.gradle/caches/7.0.2/zinc-1.3.5_2.13.6_8). It is 
currently in use by another Gradle instance.
+# [2021-10-19T17:25:07.234Z]   Owner PID: 3999
+# [2021-10-19T17:25:07.234Z]   Our PID: 3973
+# [2021-10-19T17:25:07.234Z]   Owner Operation: 
+# [2021-10-19T17:25:07.234Z]   Our operation: 
+# [2021-10-19T17:25:07.234Z]   Lock file: 
/home/jenkins/.gradle/caches/7.0.2/zinc-1.3.5_2.13.6_8/zinc-1.3.5_2.13.6_8.lock
+
+set -uf -o pipefail
+
+retryable=1
+while [[ "$retryable" != 0 ]]; do
+   retryable=0
+   rm -f buildoutput.log
+
+   "$@" 2>&1 | tee buildoutput.log
+   commandReturnCode=$?
+
+   if [ $commandReturnCode -ne 0 ]; then
+   if grep "Timeout waiting to lock zinc" buildoutput.log; then
+   retryable=1
+   echo 'Retrying due to zinc lock timeout'
+   continue
+   else
+   exit $commandReturnCode
+   fi
+   fi
+done



[kafka] branch 3.3 updated: KAFKA-14144:; Compare AlterPartition LeaderAndIsr before fencing partition epoch (#12489)

2022-08-09 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch 3.3
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.3 by this push:
 new 9fda5fcac6b KAFKA-14144:; Compare AlterPartition LeaderAndIsr before 
fencing partition epoch (#12489)
9fda5fcac6b is described below

commit 9fda5fcac6b3f7727af1c6363562da3aea415f77
Author: David Mao <47232755+sple...@users.noreply.github.com>
AuthorDate: Tue Aug 9 08:55:38 2022 -0700

KAFKA-14144:; Compare AlterPartition LeaderAndIsr before fencing partition 
epoch (#12489)

This PR fixes an AlterPartition regression introduced in 
https://github.com/apache/kafka/pull/12032

When an AlterPartition request succeeds, the partition epoch gets bumped. 
In Zk controller mode the sender also relies on the AlterPartition response to 
be informed of the new partition epoch.
If the sender times out the request before a response is sent, the sender 
will have a stale partition epoch compared to the ZK controller state and will 
be fenced on subsequent AlterPartition request attempts. The sender will not 
receive an updated partition epoch until it receives a LeaderAndIsr request for 
controller-initiated ISR changes.

Reviewers: Jason Gustafson 
---
 core/src/main/scala/kafka/api/LeaderAndIsr.scala   |  5 +-
 .../scala/kafka/controller/KafkaController.scala   | 11 +--
 .../controller/ControllerIntegrationTest.scala | 82 +-
 3 files changed, 58 insertions(+), 40 deletions(-)

diff --git a/core/src/main/scala/kafka/api/LeaderAndIsr.scala 
b/core/src/main/scala/kafka/api/LeaderAndIsr.scala
index dd1d381a144..da68cdb479c 100644
--- a/core/src/main/scala/kafka/api/LeaderAndIsr.scala
+++ b/core/src/main/scala/kafka/api/LeaderAndIsr.scala
@@ -62,7 +62,7 @@ case class LeaderAndIsr(
 if (leader == LeaderAndIsr.NoLeader) None else Some(leader)
   }
 
-  def equalsIgnorePartitionEpoch(other: LeaderAndIsr): Boolean = {
+  def equalsAllowStalePartitionEpoch(other: LeaderAndIsr): Boolean = {
 if (this == other) {
   true
 } else if (other == null) {
@@ -71,7 +71,8 @@ case class LeaderAndIsr(
   leader == other.leader &&
 leaderEpoch == other.leaderEpoch &&
 isr.equals(other.isr) &&
-leaderRecoveryState == other.leaderRecoveryState
+leaderRecoveryState == other.leaderRecoveryState &&
+partitionEpoch <= other.partitionEpoch
 }
   }
 
diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala 
b/core/src/main/scala/kafka/controller/KafkaController.scala
index 8d16eb7e1da..d179bcd6ca4 100644
--- a/core/src/main/scala/kafka/controller/KafkaController.scala
+++ b/core/src/main/scala/kafka/controller/KafkaController.scala
@@ -2339,14 +2339,15 @@ class KafkaController(val config: KafkaConfig,
   if (newLeaderAndIsr.leaderEpoch != currentLeaderAndIsr.leaderEpoch) {
 partitionResponses(tp) = Left(Errors.FENCED_LEADER_EPOCH)
 None
-  } else if (newLeaderAndIsr.partitionEpoch < 
currentLeaderAndIsr.partitionEpoch) {
-partitionResponses(tp) = Left(Errors.INVALID_UPDATE_VERSION)
-None
-  } else if 
(newLeaderAndIsr.equalsIgnorePartitionEpoch(currentLeaderAndIsr)) {
+  } else if 
(newLeaderAndIsr.equalsAllowStalePartitionEpoch(currentLeaderAndIsr)) {
 // If a partition is already in the desired state, just return it
+// this check must be done before fencing based on partition epoch 
to maintain idempotency
 partitionResponses(tp) = Right(currentLeaderAndIsr)
 None
-  } else if (newLeaderAndIsr.leaderRecoveryState == 
LeaderRecoveryState.RECOVERING && newLeaderAndIsr.isr.length > 1) {
+  } else if (newLeaderAndIsr.partitionEpoch != 
currentLeaderAndIsr.partitionEpoch) {
+partitionResponses(tp) = Left(Errors.INVALID_UPDATE_VERSION)
+None
+  }  else if (newLeaderAndIsr.leaderRecoveryState == 
LeaderRecoveryState.RECOVERING && newLeaderAndIsr.isr.length > 1) {
 partitionResponses(tp) = Left(Errors.INVALID_REQUEST)
 info(
   s"Rejecting AlterPartition from node $brokerId for $tp because 
leader is recovering and ISR is greater than 1: " +
diff --git 
a/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala 
b/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala
index 57cbeafd4d0..76188894371 100644
--- a/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala
+++ b/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala
@@ -1002,47 +1002,58 @@ class ControllerIntegrationTest extends 
QuorumTestHarness {
 
 val controller = getController().kafkaController
 val leaderIsrAndControllerEp

[kafka] branch trunk updated: KAFKA-14144:; Compare AlterPartition LeaderAndIsr before fencing partition epoch (#12489)

2022-08-09 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 62c2880a584 KAFKA-14144:; Compare AlterPartition LeaderAndIsr before 
fencing partition epoch (#12489)
62c2880a584 is described below

commit 62c2880a584dee52af52d74d7f0ec44475528c72
Author: David Mao <47232755+sple...@users.noreply.github.com>
AuthorDate: Tue Aug 9 08:55:38 2022 -0700

KAFKA-14144:; Compare AlterPartition LeaderAndIsr before fencing partition 
epoch (#12489)

This PR fixes an AlterPartition regression introduced in 
https://github.com/apache/kafka/pull/12032

When an AlterPartition request succeeds, the partition epoch gets bumped. 
In Zk controller mode the sender also relies on the AlterPartition response to 
be informed of the new partition epoch.
If the sender times out the request before a response is sent, the sender 
will have a stale partition epoch compared to the ZK controller state and will 
be fenced on subsequent AlterPartition request attempts. The sender will not 
receive an updated partition epoch until it receives a LeaderAndIsr request for 
controller-initiated ISR changes.

Reviewers: Jason Gustafson 
---
 core/src/main/scala/kafka/api/LeaderAndIsr.scala   |  5 +-
 .../scala/kafka/controller/KafkaController.scala   | 11 +--
 .../controller/ControllerIntegrationTest.scala | 82 +-
 3 files changed, 58 insertions(+), 40 deletions(-)

diff --git a/core/src/main/scala/kafka/api/LeaderAndIsr.scala 
b/core/src/main/scala/kafka/api/LeaderAndIsr.scala
index dd1d381a144..da68cdb479c 100644
--- a/core/src/main/scala/kafka/api/LeaderAndIsr.scala
+++ b/core/src/main/scala/kafka/api/LeaderAndIsr.scala
@@ -62,7 +62,7 @@ case class LeaderAndIsr(
 if (leader == LeaderAndIsr.NoLeader) None else Some(leader)
   }
 
-  def equalsIgnorePartitionEpoch(other: LeaderAndIsr): Boolean = {
+  def equalsAllowStalePartitionEpoch(other: LeaderAndIsr): Boolean = {
 if (this == other) {
   true
 } else if (other == null) {
@@ -71,7 +71,8 @@ case class LeaderAndIsr(
   leader == other.leader &&
 leaderEpoch == other.leaderEpoch &&
 isr.equals(other.isr) &&
-leaderRecoveryState == other.leaderRecoveryState
+leaderRecoveryState == other.leaderRecoveryState &&
+partitionEpoch <= other.partitionEpoch
 }
   }
 
diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala 
b/core/src/main/scala/kafka/controller/KafkaController.scala
index 8d16eb7e1da..d179bcd6ca4 100644
--- a/core/src/main/scala/kafka/controller/KafkaController.scala
+++ b/core/src/main/scala/kafka/controller/KafkaController.scala
@@ -2339,14 +2339,15 @@ class KafkaController(val config: KafkaConfig,
   if (newLeaderAndIsr.leaderEpoch != currentLeaderAndIsr.leaderEpoch) {
 partitionResponses(tp) = Left(Errors.FENCED_LEADER_EPOCH)
 None
-  } else if (newLeaderAndIsr.partitionEpoch < 
currentLeaderAndIsr.partitionEpoch) {
-partitionResponses(tp) = Left(Errors.INVALID_UPDATE_VERSION)
-None
-  } else if 
(newLeaderAndIsr.equalsIgnorePartitionEpoch(currentLeaderAndIsr)) {
+  } else if 
(newLeaderAndIsr.equalsAllowStalePartitionEpoch(currentLeaderAndIsr)) {
 // If a partition is already in the desired state, just return it
+// this check must be done before fencing based on partition epoch 
to maintain idempotency
 partitionResponses(tp) = Right(currentLeaderAndIsr)
 None
-  } else if (newLeaderAndIsr.leaderRecoveryState == 
LeaderRecoveryState.RECOVERING && newLeaderAndIsr.isr.length > 1) {
+  } else if (newLeaderAndIsr.partitionEpoch != 
currentLeaderAndIsr.partitionEpoch) {
+partitionResponses(tp) = Left(Errors.INVALID_UPDATE_VERSION)
+None
+  }  else if (newLeaderAndIsr.leaderRecoveryState == 
LeaderRecoveryState.RECOVERING && newLeaderAndIsr.isr.length > 1) {
 partitionResponses(tp) = Left(Errors.INVALID_REQUEST)
 info(
   s"Rejecting AlterPartition from node $brokerId for $tp because 
leader is recovering and ISR is greater than 1: " +
diff --git 
a/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala 
b/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala
index 57cbeafd4d0..76188894371 100644
--- a/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala
+++ b/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala
@@ -1002,47 +1002,58 @@ class ControllerIntegrationTest extends 
QuorumTestHarness {
 
 val controller = getController().kafkaController
 val leaderIsrAndControllerEp

[kafka] branch 3.3 updated: KAFKA-14104; Add CRC validation when iterating over Metadata Log Records (#12457)

2022-08-08 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch 3.3
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.3 by this push:
 new 2dafe45f15e KAFKA-14104; Add CRC validation when iterating over 
Metadata Log Records (#12457)
2dafe45f15e is described below

commit 2dafe45f15eb730a7a6bad258defd298ec173a08
Author: Niket 
AuthorDate: Mon Aug 8 15:03:04 2022 -0700

KAFKA-14104; Add CRC validation when iterating over Metadata Log Records 
(#12457)

This commit adds a check to ensure the RecordBatch CRC is valid when
iterating over a Batch of Records using the RecordsIterator. The
RecordsIterator is used by both Snapshot reads and Log Records reads in
Kraft. The check can be turned off by a class parameter and is on by 
default.

Reviewers: José Armando García Sancio 
---
 .../kafka/common/record/DefaultRecordBatch.java|  2 +-
 .../kafka/server/RaftClusterSnapshotTest.scala |  3 +-
 .../kafka/controller/QuorumControllerTest.java |  3 +-
 .../org/apache/kafka/metalog/LocalLogManager.java  |  3 +-
 .../org/apache/kafka/raft/KafkaRaftClient.java | 10 +++-
 .../kafka/raft/internals/RecordsBatchReader.java   |  5 +-
 .../kafka/raft/internals/RecordsIterator.java  | 13 -
 .../kafka/snapshot/RecordsSnapshotReader.java  |  5 +-
 .../apache/kafka/raft/RaftEventSimulationTest.java |  2 +-
 .../raft/internals/RecordsBatchReaderTest.java |  3 +-
 .../kafka/raft/internals/RecordsIteratorTest.java  | 61 +++---
 .../kafka/snapshot/SnapshotWriterReaderTest.java   |  5 +-
 12 files changed, 92 insertions(+), 23 deletions(-)

diff --git 
a/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java 
b/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java
index bd80981d84b..bc8f32491c0 100644
--- 
a/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java
@@ -107,7 +107,7 @@ public class DefaultRecordBatch extends AbstractRecordBatch 
implements MutableRe
 static final int PARTITION_LEADER_EPOCH_LENGTH = 4;
 static final int MAGIC_OFFSET = PARTITION_LEADER_EPOCH_OFFSET + 
PARTITION_LEADER_EPOCH_LENGTH;
 static final int MAGIC_LENGTH = 1;
-static final int CRC_OFFSET = MAGIC_OFFSET + MAGIC_LENGTH;
+public static final int CRC_OFFSET = MAGIC_OFFSET + MAGIC_LENGTH;
 static final int CRC_LENGTH = 4;
 static final int ATTRIBUTES_OFFSET = CRC_OFFSET + CRC_LENGTH;
 static final int ATTRIBUTE_LENGTH = 2;
diff --git 
a/core/src/test/scala/integration/kafka/server/RaftClusterSnapshotTest.scala 
b/core/src/test/scala/integration/kafka/server/RaftClusterSnapshotTest.scala
index 503ce7d2bee..f8dccd17d0d 100644
--- a/core/src/test/scala/integration/kafka/server/RaftClusterSnapshotTest.scala
+++ b/core/src/test/scala/integration/kafka/server/RaftClusterSnapshotTest.scala
@@ -78,7 +78,8 @@ class RaftClusterSnapshotTest {
 raftManager.replicatedLog.latestSnapshot.get(),
 new MetadataRecordSerde(),
 BufferSupplier.create(),
-1
+1,
+true
   )
 ) { snapshot =>
   // Check that the snapshot is non-empty
diff --git 
a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java 
b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java
index a62b1f682f0..2cdec699da2 100644
--- 
a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java
+++ 
b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java
@@ -738,7 +738,8 @@ public class QuorumControllerTest {
 reader,
 new MetadataRecordSerde(),
 BufferSupplier.create(),
-Integer.MAX_VALUE
+Integer.MAX_VALUE,
+true
 );
 }
 
diff --git 
a/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java 
b/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java
index c8e39ae3289..e24d86bd873 100644
--- a/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java
+++ b/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java
@@ -496,7 +496,8 @@ public final class LocalLogManager implements 
RaftClient,
 snapshot.get(),
 new  MetadataRecordSerde(),
 BufferSupplier.create(),
-Integer.MAX_VALUE
+Integer.MAX_VALUE,
+true
 )
 );
 }
diff --git a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java 
b/raft/src/main/java/

[kafka] branch trunk updated: KAFKA-14104; Add CRC validation when iterating over Metadata Log Records (#12457)

2022-08-08 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 48caba93409 KAFKA-14104; Add CRC validation when iterating over 
Metadata Log Records (#12457)
48caba93409 is described below

commit 48caba934091cb4cf5ec5540357fb018da21aa8d
Author: Niket 
AuthorDate: Mon Aug 8 15:03:04 2022 -0700

KAFKA-14104; Add CRC validation when iterating over Metadata Log Records 
(#12457)

This commit adds a check to ensure the RecordBatch CRC is valid when
iterating over a Batch of Records using the RecordsIterator. The
RecordsIterator is used by both Snapshot reads and Log Records reads in
Kraft. The check can be turned off by a class parameter and is on by 
default.

Reviewers: José Armando García Sancio 
---
 .../kafka/common/record/DefaultRecordBatch.java|  2 +-
 .../kafka/server/RaftClusterSnapshotTest.scala |  3 +-
 .../kafka/controller/QuorumControllerTest.java |  3 +-
 .../org/apache/kafka/metalog/LocalLogManager.java  |  3 +-
 .../org/apache/kafka/raft/KafkaRaftClient.java | 10 +++-
 .../kafka/raft/internals/RecordsBatchReader.java   |  5 +-
 .../kafka/raft/internals/RecordsIterator.java  | 13 -
 .../kafka/snapshot/RecordsSnapshotReader.java  |  5 +-
 .../apache/kafka/raft/RaftEventSimulationTest.java |  2 +-
 .../raft/internals/RecordsBatchReaderTest.java |  3 +-
 .../kafka/raft/internals/RecordsIteratorTest.java  | 61 +++---
 .../kafka/snapshot/SnapshotWriterReaderTest.java   |  5 +-
 12 files changed, 92 insertions(+), 23 deletions(-)

diff --git 
a/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java 
b/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java
index bd80981d84b..bc8f32491c0 100644
--- 
a/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java
@@ -107,7 +107,7 @@ public class DefaultRecordBatch extends AbstractRecordBatch 
implements MutableRe
 static final int PARTITION_LEADER_EPOCH_LENGTH = 4;
 static final int MAGIC_OFFSET = PARTITION_LEADER_EPOCH_OFFSET + 
PARTITION_LEADER_EPOCH_LENGTH;
 static final int MAGIC_LENGTH = 1;
-static final int CRC_OFFSET = MAGIC_OFFSET + MAGIC_LENGTH;
+public static final int CRC_OFFSET = MAGIC_OFFSET + MAGIC_LENGTH;
 static final int CRC_LENGTH = 4;
 static final int ATTRIBUTES_OFFSET = CRC_OFFSET + CRC_LENGTH;
 static final int ATTRIBUTE_LENGTH = 2;
diff --git 
a/core/src/test/scala/integration/kafka/server/RaftClusterSnapshotTest.scala 
b/core/src/test/scala/integration/kafka/server/RaftClusterSnapshotTest.scala
index 503ce7d2bee..f8dccd17d0d 100644
--- a/core/src/test/scala/integration/kafka/server/RaftClusterSnapshotTest.scala
+++ b/core/src/test/scala/integration/kafka/server/RaftClusterSnapshotTest.scala
@@ -78,7 +78,8 @@ class RaftClusterSnapshotTest {
 raftManager.replicatedLog.latestSnapshot.get(),
 new MetadataRecordSerde(),
 BufferSupplier.create(),
-1
+1,
+true
   )
 ) { snapshot =>
   // Check that the snapshot is non-empty
diff --git 
a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java 
b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java
index 5e395cebcb0..e8392895626 100644
--- 
a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java
+++ 
b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java
@@ -739,7 +739,8 @@ public class QuorumControllerTest {
 reader,
 new MetadataRecordSerde(),
 BufferSupplier.create(),
-Integer.MAX_VALUE
+Integer.MAX_VALUE,
+true
 );
 }
 
diff --git 
a/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java 
b/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java
index c8e39ae3289..e24d86bd873 100644
--- a/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java
+++ b/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java
@@ -496,7 +496,8 @@ public final class LocalLogManager implements 
RaftClient,
 snapshot.get(),
 new  MetadataRecordSerde(),
 BufferSupplier.create(),
-Integer.MAX_VALUE
+Integer.MAX_VALUE,
+true
 )
 );
 }
diff --git a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java 
b/raft/src/main/java/

[kafka] branch 3.3 updated: KAFKA-14078; Do leader/epoch validation in Fetch before checking for valid replica (#12411)

2022-07-25 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch 3.3
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.3 by this push:
 new ebd63b54bd KAFKA-14078; Do leader/epoch validation in Fetch before 
checking for valid replica (#12411)
ebd63b54bd is described below

commit ebd63b54bddae886f8125d904e1676333e8f4e58
Author: Jason Gustafson 
AuthorDate: Mon Jul 25 13:24:40 2022 -0700

KAFKA-14078; Do leader/epoch validation in Fetch before checking for valid 
replica (#12411)

After the fix for https://github.com/apache/kafka/pull/12150, if a follower 
receives a request from another replica, it will return UNKNOWN_LEADER_EPOCH 
even if the leader epoch matches. We need to do epoch leader/epoch validation 
first before we check whether we have a valid replica.

Reviewers: David Jacot 
---
 core/src/main/scala/kafka/cluster/Partition.scala  | 36 +--
 .../scala/unit/kafka/cluster/PartitionTest.scala   | 41 ++
 2 files changed, 66 insertions(+), 11 deletions(-)

diff --git a/core/src/main/scala/kafka/cluster/Partition.scala 
b/core/src/main/scala/kafka/cluster/Partition.scala
index 319025226c..538c51f903 100755
--- a/core/src/main/scala/kafka/cluster/Partition.scala
+++ b/core/src/main/scala/kafka/cluster/Partition.scala
@@ -1205,22 +1205,32 @@ class Partition(val topicPartition: TopicPartition,
 minOneMessage: Boolean,
 updateFetchState: Boolean
   ): LogReadInfo = {
-def readFromLocalLog(): LogReadInfo = {
+def readFromLocalLog(log: UnifiedLog): LogReadInfo = {
   readRecords(
+log,
 fetchPartitionData.lastFetchedEpoch,
 fetchPartitionData.fetchOffset,
 fetchPartitionData.currentLeaderEpoch,
 maxBytes,
 fetchParams.isolation,
-minOneMessage,
-fetchParams.fetchOnlyLeader
+minOneMessage
   )
 }
 
 if (fetchParams.isFromFollower) {
   // Check that the request is from a valid replica before doing the read
-  val replica = followerReplicaOrThrow(fetchParams.replicaId, 
fetchPartitionData)
-  val logReadInfo = readFromLocalLog()
+  val (replica, logReadInfo) = inReadLock(leaderIsrUpdateLock) {
+val localLog = localLogWithEpochOrThrow(
+  fetchPartitionData.currentLeaderEpoch,
+  fetchParams.fetchOnlyLeader
+)
+val replica = followerReplicaOrThrow(
+  fetchParams.replicaId,
+  fetchPartitionData
+)
+val logReadInfo = readFromLocalLog(localLog)
+(replica, logReadInfo)
+  }
 
   if (updateFetchState && logReadInfo.divergingEpoch.isEmpty) {
 updateFollowerFetchState(
@@ -1234,7 +1244,13 @@ class Partition(val topicPartition: TopicPartition,
 
   logReadInfo
 } else {
-  readFromLocalLog()
+  inReadLock(leaderIsrUpdateLock) {
+val localLog = localLogWithEpochOrThrow(
+  fetchPartitionData.currentLeaderEpoch,
+  fetchParams.fetchOnlyLeader
+)
+readFromLocalLog(localLog)
+  }
 }
   }
 
@@ -1270,16 +1286,14 @@ class Partition(val topicPartition: TopicPartition,
   }
 
   private def readRecords(
+localLog: UnifiedLog,
 lastFetchedEpoch: Optional[Integer],
 fetchOffset: Long,
 currentLeaderEpoch: Optional[Integer],
 maxBytes: Int,
 fetchIsolation: FetchIsolation,
-minOneMessage: Boolean,
-fetchOnlyFromLeader: Boolean
-  ): LogReadInfo = inReadLock(leaderIsrUpdateLock) {
-val localLog = localLogWithEpochOrThrow(currentLeaderEpoch, 
fetchOnlyFromLeader)
-
+minOneMessage: Boolean
+  ): LogReadInfo = {
 // Note we use the log end offset prior to the read. This ensures that any 
appends following
 // the fetch do not prevent a follower from coming into sync.
 val initialHighWatermark = localLog.highWatermark
diff --git a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala 
b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala
index 65a6cdadf4..5038219579 100644
--- a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala
+++ b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala
@@ -238,6 +238,47 @@ class PartitionTest extends AbstractPartitionTest {
 assertEquals(None, partition.futureLog)
   }
 
+  @Test
+  def testReplicaFetchToFollower(): Unit = {
+val controllerEpoch = 3
+val followerId = brokerId + 1
+val leaderId = brokerId + 2
+val replicas = List[Integer](brokerId, followerId, leaderId).asJava
+val isr = List[Integer](brokerId, followerId, leaderId).asJava
+val leaderEpoch = 8
+val partitionEpoch = 1
+
+assertTrue(partition.makeFollower(new LeaderAndIsrPartitionState()
+  .setControllerEpoch(controllerEpoch)
+  .setLeader(leaderId)
+  .setLeaderEpoch(leaderEpoch)
+  .setIsr(isr)
+  .setPartitionEpoch(partitionEpoch)
+  .setReplicas(

[kafka] branch trunk updated: KAFKA-14078; Do leader/epoch validation in Fetch before checking for valid replica (#12411)

2022-07-25 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new a450fb70c1 KAFKA-14078; Do leader/epoch validation in Fetch before 
checking for valid replica (#12411)
a450fb70c1 is described below

commit a450fb70c12ba66257e8f61cc4903290f1e435ea
Author: Jason Gustafson 
AuthorDate: Mon Jul 25 13:24:40 2022 -0700

KAFKA-14078; Do leader/epoch validation in Fetch before checking for valid 
replica (#12411)

After the fix for https://github.com/apache/kafka/pull/12150, if a follower 
receives a request from another replica, it will return UNKNOWN_LEADER_EPOCH 
even if the leader epoch matches. We need to do epoch leader/epoch validation 
first before we check whether we have a valid replica.

Reviewers: David Jacot 
---
 core/src/main/scala/kafka/cluster/Partition.scala  | 36 +--
 .../scala/unit/kafka/cluster/PartitionTest.scala   | 41 ++
 2 files changed, 66 insertions(+), 11 deletions(-)

diff --git a/core/src/main/scala/kafka/cluster/Partition.scala 
b/core/src/main/scala/kafka/cluster/Partition.scala
index 319025226c..538c51f903 100755
--- a/core/src/main/scala/kafka/cluster/Partition.scala
+++ b/core/src/main/scala/kafka/cluster/Partition.scala
@@ -1205,22 +1205,32 @@ class Partition(val topicPartition: TopicPartition,
 minOneMessage: Boolean,
 updateFetchState: Boolean
   ): LogReadInfo = {
-def readFromLocalLog(): LogReadInfo = {
+def readFromLocalLog(log: UnifiedLog): LogReadInfo = {
   readRecords(
+log,
 fetchPartitionData.lastFetchedEpoch,
 fetchPartitionData.fetchOffset,
 fetchPartitionData.currentLeaderEpoch,
 maxBytes,
 fetchParams.isolation,
-minOneMessage,
-fetchParams.fetchOnlyLeader
+minOneMessage
   )
 }
 
 if (fetchParams.isFromFollower) {
   // Check that the request is from a valid replica before doing the read
-  val replica = followerReplicaOrThrow(fetchParams.replicaId, 
fetchPartitionData)
-  val logReadInfo = readFromLocalLog()
+  val (replica, logReadInfo) = inReadLock(leaderIsrUpdateLock) {
+val localLog = localLogWithEpochOrThrow(
+  fetchPartitionData.currentLeaderEpoch,
+  fetchParams.fetchOnlyLeader
+)
+val replica = followerReplicaOrThrow(
+  fetchParams.replicaId,
+  fetchPartitionData
+)
+val logReadInfo = readFromLocalLog(localLog)
+(replica, logReadInfo)
+  }
 
   if (updateFetchState && logReadInfo.divergingEpoch.isEmpty) {
 updateFollowerFetchState(
@@ -1234,7 +1244,13 @@ class Partition(val topicPartition: TopicPartition,
 
   logReadInfo
 } else {
-  readFromLocalLog()
+  inReadLock(leaderIsrUpdateLock) {
+val localLog = localLogWithEpochOrThrow(
+  fetchPartitionData.currentLeaderEpoch,
+  fetchParams.fetchOnlyLeader
+)
+readFromLocalLog(localLog)
+  }
 }
   }
 
@@ -1270,16 +1286,14 @@ class Partition(val topicPartition: TopicPartition,
   }
 
   private def readRecords(
+localLog: UnifiedLog,
 lastFetchedEpoch: Optional[Integer],
 fetchOffset: Long,
 currentLeaderEpoch: Optional[Integer],
 maxBytes: Int,
 fetchIsolation: FetchIsolation,
-minOneMessage: Boolean,
-fetchOnlyFromLeader: Boolean
-  ): LogReadInfo = inReadLock(leaderIsrUpdateLock) {
-val localLog = localLogWithEpochOrThrow(currentLeaderEpoch, 
fetchOnlyFromLeader)
-
+minOneMessage: Boolean
+  ): LogReadInfo = {
 // Note we use the log end offset prior to the read. This ensures that any 
appends following
 // the fetch do not prevent a follower from coming into sync.
 val initialHighWatermark = localLog.highWatermark
diff --git a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala 
b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala
index 65a6cdadf4..5038219579 100644
--- a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala
+++ b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala
@@ -238,6 +238,47 @@ class PartitionTest extends AbstractPartitionTest {
 assertEquals(None, partition.futureLog)
   }
 
+  @Test
+  def testReplicaFetchToFollower(): Unit = {
+val controllerEpoch = 3
+val followerId = brokerId + 1
+val leaderId = brokerId + 2
+val replicas = List[Integer](brokerId, followerId, leaderId).asJava
+val isr = List[Integer](brokerId, followerId, leaderId).asJava
+val leaderEpoch = 8
+val partitionEpoch = 1
+
+assertTrue(partition.makeFollower(new LeaderAndIsrPartitionState()
+  .setControllerEpoch(controllerEpoch)
+  .setLeader(leaderId)
+  .setLeaderEpoch(leaderEpoch)
+  .setIsr(isr)
+  .setPartitionEpoch(partitionEpoch)
+  .setReplicas(

[kafka] 01/02: KAFKA-14035; Fix NPE in `SnapshottableHashTable::mergeFrom()` (#12371)

2022-07-10 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch 3.0
in repository https://gitbox.apache.org/repos/asf/kafka.git

commit 50fa5b40a0fbde2341ddca453ac92733314bfe34
Author: Niket 
AuthorDate: Thu Jun 30 21:03:54 2022 -0700

KAFKA-14035; Fix NPE in `SnapshottableHashTable::mergeFrom()` (#12371)

The NPE causes the kraft controller to be in an inconsistent state.

Reviewers: Jason Gustafson 
---
 .../kafka/timeline/SnapshottableHashTable.java | 22 +-
 .../kafka/timeline/SnapshottableHashTableTest.java | 19 +++
 2 files changed, 32 insertions(+), 9 deletions(-)

diff --git 
a/metadata/src/main/java/org/apache/kafka/timeline/SnapshottableHashTable.java 
b/metadata/src/main/java/org/apache/kafka/timeline/SnapshottableHashTable.java
index cbd0a280fc1..299f65a6f78 100644
--- 
a/metadata/src/main/java/org/apache/kafka/timeline/SnapshottableHashTable.java
+++ 
b/metadata/src/main/java/org/apache/kafka/timeline/SnapshottableHashTable.java
@@ -111,15 +111,19 @@ class SnapshottableHashTable other = (HashTier) source;
-List list = new ArrayList<>();
-Object[] otherElements = other.deltaTable.baseElements();
-for (int slot = 0; slot < otherElements.length; slot++) {
-BaseHashTable.unpackSlot(list, otherElements, slot);
-for (T element : list) {
-// When merging in a later hash tier, we want to keep only 
the elements
-// that were present at our epoch.
-if (element.startEpoch() <= epoch) {
-deltaTable.baseAddOrReplace(element);
+// As an optimization, the deltaTable might not exist for a new key
+// as there is no previous value
+if (other.deltaTable != null) {
+List list = new ArrayList<>();
+Object[] otherElements = other.deltaTable.baseElements();
+for (int slot = 0; slot < otherElements.length; slot++) {
+BaseHashTable.unpackSlot(list, otherElements, slot);
+for (T element : list) {
+// When merging in a later hash tier, we want to keep 
only the elements
+// that were present at our epoch.
+if (element.startEpoch() <= epoch) {
+deltaTable.baseAddOrReplace(element);
+}
 }
 }
 }
diff --git 
a/metadata/src/test/java/org/apache/kafka/timeline/SnapshottableHashTableTest.java
 
b/metadata/src/test/java/org/apache/kafka/timeline/SnapshottableHashTableTest.java
index 7f1ddcc3ff5..1b9dd1559ea 100644
--- 
a/metadata/src/test/java/org/apache/kafka/timeline/SnapshottableHashTableTest.java
+++ 
b/metadata/src/test/java/org/apache/kafka/timeline/SnapshottableHashTableTest.java
@@ -97,6 +97,25 @@ public class SnapshottableHashTableTest {
 new SnapshottableHashTable<>(registry, 1);
 assertEquals(0, table.snapshottableSize(Long.MAX_VALUE));
 }
+@Test
+public void testDeleteOnEmptyDeltaTable() {
+// A simple test case to validate the behavior of the TimelineHashSet
+// when the deltaTable for a snapshot is null
+SnapshotRegistry registry = new SnapshotRegistry(new LogContext());
+TimelineHashSet set = new TimelineHashSet<>(registry, 5);
+
+registry.getOrCreateSnapshot(100);
+set.add("bar");
+registry.getOrCreateSnapshot(200);
+set.add("baz");
+registry.revertToSnapshot(100);
+assertTrue(set.isEmpty());
+set.add("foo");
+registry.getOrCreateSnapshot(300);
+set.remove("bar");
+registry.revertToSnapshot(100);
+assertTrue(set.isEmpty());
+}
 
 @Test
 public void testAddAndRemove() {



[kafka] 02/02: KAFKA-14055; Txn markers should not be removed by matching records in the offset map (#12390)

2022-07-10 Thread jgus
This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch 3.0
in repository https://gitbox.apache.org/repos/asf/kafka.git

commit 77230b567ab51726302466058ca5f5e734e81664
Author: Jason Gustafson 
AuthorDate: Sun Jul 10 10:16:39 2022 -0700

KAFKA-14055; Txn markers should not be removed by matching records in the 
offset map (#12390)

When cleaning a topic with transactional data, if the keys used in the user 
data happen to conflict with the keys in the transaction markers, it is 
possible for the markers to get removed before the corresponding data from the 
transaction is removed. This results in a hanging transaction or the loss of 
the transaction's atomicity since it would effectively get bundled into the 
next transaction in the log. Currently control records are excluded when 
building the offset map, but not whe [...]

Reviewers: Jun Rao 
---
 core/src/main/scala/kafka/log/LogCleaner.scala |  2 +
 .../test/scala/unit/kafka/log/LogCleanerTest.scala | 79 ++
 2 files changed, 69 insertions(+), 12 deletions(-)

diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala 
b/core/src/main/scala/kafka/log/LogCleaner.scala
index 12099f0f2c1..97de1db57c2 100644
--- a/core/src/main/scala/kafka/log/LogCleaner.scala
+++ b/core/src/main/scala/kafka/log/LogCleaner.scala
@@ -678,6 +678,8 @@ private[log] class Cleaner(val id: Int,
 if (discardBatchRecords)
   // The batch is only retained to preserve producer sequence 
information; the records can be removed
   false
+else if (batch.isControlBatch)
+  true
 else
   Cleaner.this.shouldRetainRecord(map, retainDeletesAndTxnMarkers, 
batch, record, stats)
   }
diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala 
b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala
index 5b942342193..253bf5490c5 100755
--- a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala
@@ -1027,6 +1027,50 @@ class LogCleanerTest {
 assertEquals(List(3, 4, 5), offsetsInLog(log))
   }
 
+
+  @Test
+  def testCleaningWithKeysConflictingWithTxnMarkerKeys(): Unit = {
+val cleaner = makeCleaner(10)
+val logProps = new Properties()
+logProps.put(LogConfig.SegmentBytesProp, 1024: java.lang.Integer)
+val log = makeLog(config = LogConfig.fromProps(logConfig.originals, 
logProps))
+val leaderEpoch = 5
+val producerEpoch = 0.toShort
+
+// First we append one committed transaction
+val producerId1 = 1L
+val appendProducer = appendTransactionalAsLeader(log, producerId1, 
producerEpoch, leaderEpoch)
+appendProducer(Seq(1))
+log.appendAsLeader(commitMarker(producerId1, producerEpoch), leaderEpoch, 
origin = AppendOrigin.Coordinator)
+
+// Now we append one transaction with a key which conflicts with the 
COMMIT marker appended above
+def commitRecordKey(): ByteBuffer = {
+  val keySize = ControlRecordType.COMMIT.recordKey().sizeOf()
+  val key = ByteBuffer.allocate(keySize)
+  ControlRecordType.COMMIT.recordKey().writeTo(key)
+  key.flip()
+  key
+}
+
+val producerId2 = 2L
+val records = MemoryRecords.withTransactionalRecords(
+  CompressionType.NONE,
+  producerId2,
+  producerEpoch,
+  0,
+  new SimpleRecord(time.milliseconds(), commitRecordKey(), 
ByteBuffer.wrap("foo".getBytes))
+)
+log.appendAsLeader(records, leaderEpoch, origin = AppendOrigin.Client)
+log.appendAsLeader(commitMarker(producerId2, producerEpoch), leaderEpoch, 
origin = AppendOrigin.Coordinator)
+log.roll()
+assertEquals(List(0, 1, 2, 3), offsetsInLog(log))
+
+// After cleaning, the marker should not be removed
+cleaner.clean(LogToClean(new TopicPartition("test", 0), log, 0L, 
log.activeSegment.baseOffset))
+assertEquals(List(0, 1, 2, 3), lastOffsetsPerBatchInLog(log))
+assertEquals(List(0, 1, 2, 3), offsetsInLog(log))
+  }
+
   @Test
   def testPartialSegmentClean(): Unit = {
 // because loadFactor is 0.75, this means we can fit 1 message in the map
@@ -1917,20 +1961,31 @@ class LogCleanerTest {
   partitionLeaderEpoch, new SimpleRecord(key.toString.getBytes, 
value.toString.getBytes))
   }
 
-  private def appendTransactionalAsLeader(log: Log,
-  producerId: Long,
-  producerEpoch: Short,
-  leaderEpoch: Int = 0,
-  origin: AppendOrigin = 
AppendOrigin.Client): Seq[Int] => LogAppendInfo = {
-appendIdempotentAsLeader(log, producerId, producerEpoch, isTransactional = 
true, origin = origin)
+  private def appendTransactionalAsLeader(
+log: Log,
+producerId: Long,
+producerEpoch: Short,
+leaderEpoch: Int = 0,
+origin: AppendOrigin = AppendOrig

  1   2   3   4   5   6   7   8   9   10   >