(kafka) branch trunk updated: KAFKA-16305: Avoid optimisation in handshakeUnwrap (#15434)

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

rsivaram 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 5d6936a4992  KAFKA-16305: Avoid optimisation in handshakeUnwrap 
(#15434)
5d6936a4992 is described below

commit 5d6936a4992b77ef68da216a7c2dbf1f8c9f909e
Author: Gaurav Narula 
AuthorDate: Wed Feb 28 09:37:58 2024 +

 KAFKA-16305: Avoid optimisation in handshakeUnwrap (#15434)

Performs additional unwrap during handshake after data from client is 
processed to support openssl, which needs the extra unwrap to complete 
handshake.

Reviewers: Ismael Juma , Rajini Sivaram 

---
 .../kafka/common/network/SslTransportLayer.java|  7 +--
 .../common/network/SslTransportLayerTest.java  | 60 ++
 2 files changed, 64 insertions(+), 3 deletions(-)

diff --git 
a/clients/src/main/java/org/apache/kafka/common/network/SslTransportLayer.java 
b/clients/src/main/java/org/apache/kafka/common/network/SslTransportLayer.java
index 904c5216a40..da80e363a95 100644
--- 
a/clients/src/main/java/org/apache/kafka/common/network/SslTransportLayer.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/network/SslTransportLayer.java
@@ -498,13 +498,14 @@ public class SslTransportLayer implements TransportLayer {
 }
 
 /**
- * Perform handshake unwrap
+ * Perform handshake unwrap.
+ * Visible for testing.
  * @param doRead boolean If true, read more from the socket channel
  * @param ignoreHandshakeStatus If true, continue to unwrap if data 
available regardless of handshake status
  * @return SSLEngineResult
  * @throws IOException
  */
-private SSLEngineResult handshakeUnwrap(boolean doRead, boolean 
ignoreHandshakeStatus) throws IOException {
+SSLEngineResult handshakeUnwrap(boolean doRead, boolean 
ignoreHandshakeStatus) throws IOException {
 log.trace("SSLHandshake handshakeUnwrap {}", channelId);
 SSLEngineResult result;
 int read = 0;
@@ -526,7 +527,7 @@ public class SslTransportLayer implements TransportLayer {
 handshakeStatus == HandshakeStatus.NEED_UNWRAP) ||
 (ignoreHandshakeStatus && netReadBuffer.position() != 
position);
 log.trace("SSLHandshake handshakeUnwrap: handshakeStatus {} status 
{}", handshakeStatus, result.getStatus());
-} while (netReadBuffer.position() != 0 && cont);
+} while (cont);
 
 // Throw EOF exception for failed read after processing already 
received data
 // so that handshake failures are reported correctly
diff --git 
a/clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java
 
b/clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java
index d92f4facb3c..8b00bcdb955 100644
--- 
a/clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java
+++ 
b/clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java
@@ -36,6 +36,7 @@ import org.apache.kafka.common.utils.Utils;
 import org.apache.kafka.test.TestSslUtils;
 import org.apache.kafka.test.TestUtils;
 import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Test;
 import org.junit.jupiter.api.extension.ExtensionContext;
 import org.junit.jupiter.params.ParameterizedTest;
 import org.junit.jupiter.params.provider.Arguments;
@@ -51,6 +52,7 @@ import java.nio.ByteBuffer;
 import java.nio.channels.Channels;
 import java.nio.channels.SelectionKey;
 import java.nio.channels.SocketChannel;
+import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -65,13 +67,20 @@ import java.util.stream.Stream;
 
 import javax.net.ssl.SSLContext;
 import javax.net.ssl.SSLEngine;
+import javax.net.ssl.SSLEngineResult;
 import javax.net.ssl.SSLParameters;
+import javax.net.ssl.SSLSession;
 
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertThrows;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 import static org.junit.jupiter.api.Assumptions.assumeTrue;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
 
 /**
  * Tests for the SSL transport layer. These use a test harness that runs a 
simple socket server that echos back responses.
@@ -1467,4 +1476,55 @@ public class SslTransportLayerTest {
 }
 }
 }
+
+/**
+ * SSLEngine implementations may transition from NEED_UNWRAP to NEED_UNWRAP
+ * even after reading all the data from the socket. This test ensures we
+ * con

[kafka] branch trunk updated: KAFKA-15510: Fix follower's lastFetchedEpoch when fetch response has … (#14457)

2023-09-28 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

rsivaram 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 dedfed06f7a KAFKA-15510: Fix follower's lastFetchedEpoch when fetch 
response has … (#14457)
dedfed06f7a is described below

commit dedfed06f7a472424080456c997f5200c6bef196
Author: chern 
AuthorDate: Thu Sep 28 06:14:42 2023 -0700

KAFKA-15510: Fix follower's lastFetchedEpoch when fetch response has … 
(#14457)

When a fetch response has no record for a partition, validBytes is 0. We 
shouldn't set the last fetched epoch to logAppendInfo.lastLeaderEpoch.asScala 
since there is no record and it is Optional.empty. We should use 
currentFetchState.lastFetchedEpoch instead.

Reviewers: Divij Vaidya , Viktor Somogyi-Vass 
, Kamal 
Chandraprakash, Rajini Sivaram 

---
 core/src/main/scala/kafka/server/AbstractFetcherThread.scala | 5 +++--
 core/src/test/scala/unit/kafka/server/ReplicaFetcherThreadTest.scala | 4 +++-
 2 files changed, 6 insertions(+), 3 deletions(-)

diff --git a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala 
b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala
index 1dafb89ef0a..450fcfea461 100755
--- a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala
+++ b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala
@@ -364,10 +364,11 @@ abstract class AbstractFetcherThread(name: String,
 
 // ReplicaDirAlterThread may have removed 
topicPartition from the partitionStates after processing the partition data
 if ((validBytes > 0 || currentFetchState.lag.isEmpty) 
&& partitionStates.contains(topicPartition)) {
+  val lastFetchedEpoch =
+if (logAppendInfo.lastLeaderEpoch.isPresent) 
logAppendInfo.lastLeaderEpoch.asScala else currentFetchState.lastFetchedEpoch
   // Update partitionStates only if there is no 
exception during processPartitionData
   val newFetchState = 
PartitionFetchState(currentFetchState.topicId, nextOffset, Some(lag),
-currentFetchState.currentLeaderEpoch, state = 
Fetching,
-logAppendInfo.lastLeaderEpoch.asScala)
+currentFetchState.currentLeaderEpoch, state = 
Fetching, lastFetchedEpoch)
   partitionStates.updateAndMoveToEnd(topicPartition, 
newFetchState)
   if (validBytes > 0) 
fetcherStats.byteRate.mark(validBytes)
 }
diff --git 
a/core/src/test/scala/unit/kafka/server/ReplicaFetcherThreadTest.scala 
b/core/src/test/scala/unit/kafka/server/ReplicaFetcherThreadTest.scala
index e58532622e3..6a0feaa6456 100644
--- a/core/src/test/scala/unit/kafka/server/ReplicaFetcherThreadTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ReplicaFetcherThreadTest.scala
@@ -749,9 +749,10 @@ class ReplicaFetcherThreadTest {
 val log: UnifiedLog = mock(classOf[UnifiedLog])
 val partition: Partition = mock(classOf[Partition])
 val replicaManager: ReplicaManager = mock(classOf[ReplicaManager])
+val lastFetchedEpoch = 2
 
 when(log.highWatermark).thenReturn(0)
-when(log.latestEpoch).thenReturn(Some(0))
+when(log.latestEpoch).thenReturn(Some(lastFetchedEpoch))
 when(log.endOffsetForEpoch(0)).thenReturn(Some(new OffsetAndEpoch(0, 0)))
 when(log.logEndOffset).thenReturn(0)
 when(log.maybeUpdateHighWatermark(0)).thenReturn(None)
@@ -835,6 +836,7 @@ class ReplicaFetcherThreadTest {
 
 // Lag is set to Some(0).
 assertEquals(Some(0), thread.fetchState(t1p0).flatMap(_.lag))
+assertEquals(Some(lastFetchedEpoch), 
thread.fetchState(t1p0).flatMap(_.lastFetchedEpoch))
   }
 
   @Test



[kafka] branch trunk updated: [KAFKA-15117] In TestSslUtils set SubjectAlternativeNames to null if there are no hostnames (#14440)

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

rsivaram 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 d8f358facc2 [KAFKA-15117] In TestSslUtils set SubjectAlternativeNames 
to null if there are no hostnames (#14440)
d8f358facc2 is described below

commit d8f358facc2a5405d08977f922bc0b1dae8f114e
Author: Purshotam Chauhan 
AuthorDate: Mon Sep 25 22:13:01 2023 +0530

[KAFKA-15117] In TestSslUtils set SubjectAlternativeNames to null if there 
are no hostnames (#14440)

We are currently encoding an empty hostNames array to subjectAltName in the 
keystore. While parsing the certificates in the test this causes the issue - 
Unparseable SubjectAlternativeName extension due to java.io.IOException: No 
data available in passed DER encoded value. Up to Java 17, this parsing error 
was ignored. This PR assigns subjectAltName to null if hostnames are empty.

Co-authored-by: Ismael Juma 
Reviewers: Rajini Sivaram 
---
 .../apache/kafka/common/network/SslTransportLayerTest.java   |  3 ---
 .../src/test/java/org/apache/kafka/test/TestSslUtils.java| 12 
 2 files changed, 8 insertions(+), 7 deletions(-)

diff --git 
a/clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java
 
b/clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java
index f49bf868a46..26987e30da8 100644
--- 
a/clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java
+++ 
b/clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java
@@ -36,8 +36,6 @@ import org.apache.kafka.common.utils.Utils;
 import org.apache.kafka.test.TestSslUtils;
 import org.apache.kafka.test.TestUtils;
 import org.junit.jupiter.api.AfterEach;
-import org.junit.jupiter.api.condition.DisabledOnJre;
-import org.junit.jupiter.api.condition.JRE;
 import org.junit.jupiter.api.extension.ExtensionContext;
 import org.junit.jupiter.params.ParameterizedTest;
 import org.junit.jupiter.params.provider.Arguments;
@@ -195,7 +193,6 @@ public class SslTransportLayerTest {
  */
 @ParameterizedTest
 @ArgumentsSource(SslTransportLayerArgumentsProvider.class)
-@DisabledOnJre(value = {JRE.JAVA_20, JRE.JAVA_21}, disabledReason = 
"KAFKA-15117")
 public void testValidEndpointIdentificationCN(Args args) throws Exception {
 args.serverCertStores = certBuilder(true, "localhost", 
args.useInlinePem).build();
 args.clientCertStores = certBuilder(false, "localhost", 
args.useInlinePem).build();
diff --git a/clients/src/test/java/org/apache/kafka/test/TestSslUtils.java 
b/clients/src/test/java/org/apache/kafka/test/TestSslUtils.java
index 6b7c16b0335..1181fc2 100644
--- a/clients/src/test/java/org/apache/kafka/test/TestSslUtils.java
+++ b/clients/src/test/java/org/apache/kafka/test/TestSslUtils.java
@@ -399,10 +399,14 @@ public class TestSslUtils {
 }
 
 public CertificateBuilder sanDnsNames(String... hostNames) throws 
IOException {
-GeneralName[] altNames = new GeneralName[hostNames.length];
-for (int i = 0; i < hostNames.length; i++)
-altNames[i] = new GeneralName(GeneralName.dNSName, 
hostNames[i]);
-subjectAltName = GeneralNames.getInstance(new 
DERSequence(altNames)).getEncoded();
+if (hostNames.length > 0) {
+GeneralName[] altNames = new GeneralName[hostNames.length];
+for (int i = 0; i < hostNames.length; i++)
+altNames[i] = new GeneralName(GeneralName.dNSName, 
hostNames[i]);
+subjectAltName = GeneralNames.getInstance(new 
DERSequence(altNames)).getEncoded();
+} else {
+subjectAltName = null;
+}
 return this;
 }
 



[kafka] branch trunk updated (e99984248da -> b64ac94a8c6)

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

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


from e99984248da KAFKA-9550 Copying log segments to tiered storage in 
RemoteLogManager (#13487)
 add b64ac94a8c6 KAFKA-14891: Fix rack-aware range assignor to assign 
co-partitioned subsets (#13539)

No new revisions were added by this update.

Summary of changes:
 .../kafka/clients/consumer/RangeAssignor.java  |  2 +-
 .../kafka/clients/consumer/RangeAssignorTest.java  | 30 --
 2 files changed, 29 insertions(+), 3 deletions(-)



[kafka] branch trunk updated (970dea60e86 -> 1f0ae71fb32)

2023-04-03 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

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


from 970dea60e86 KAFKA-14785 (KIP-875): Connect offset read REST API 
(#13434)
 add 1f0ae71fb32 KAFKA-14452: Make sticky assignors rack-aware if client 
rack is configured (KIP-881) (#13350)

No new revisions were added by this update.

Summary of changes:
 .../consumer/CooperativeStickyAssignor.java|   9 +-
 .../kafka/clients/consumer/StickyAssignor.java |   2 +-
 .../consumer/internals/AbstractStickyAssignor.java | 329 ++--
 .../consumer/CooperativeStickyAssignorTest.java|  72 +-
 .../kafka/clients/consumer/StickyAssignorTest.java | 206 ++---
 .../internals/AbstractPartitionAssignorTest.java   |  48 +-
 .../internals/AbstractStickyAssignorTest.java  | 827 ++---
 7 files changed, 1014 insertions(+), 479 deletions(-)



[kafka] branch trunk updated: KAFKA-14867: Trigger rebalance when replica racks change if client.rack is configured (KIP-881) (#13474)

2023-03-31 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

rsivaram 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 3c4472d701a KAFKA-14867: Trigger rebalance when replica racks change 
if client.rack is configured (KIP-881) (#13474)
3c4472d701a is described below

commit 3c4472d701a7e9d9b8714a0b9d87ae190d1679fb
Author: Rajini Sivaram 
AuthorDate: Fri Mar 31 15:01:07 2023 +0100

KAFKA-14867: Trigger rebalance when replica racks change if client.rack is 
configured (KIP-881) (#13474)

When `client.rack` is configured for consumers, we perform rack-aware 
consumer partition assignment to improve locality. After/during reassignments, 
replica racks may change, so to ensure optimal consumer assignment, trigger 
rebalance from the leader when set of racks of any partition changes.

Reviewers: David Jacot 
---
 .../consumer/internals/ConsumerCoordinator.java|  58 +--
 .../internals/ConsumerCoordinatorTest.java | 171 ++---
 .../server/FetchFromFollowerIntegrationTest.scala  |  26 +++-
 3 files changed, 224 insertions(+), 31 deletions(-)

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 fec31fe80f8..1f6c5ef0d75 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
@@ -16,6 +16,7 @@
  */
 package org.apache.kafka.clients.consumer.internals;
 
+import java.util.Arrays;
 import java.util.SortedSet;
 import java.util.TreeSet;
 import org.apache.kafka.clients.GroupRebalanceConfig;
@@ -35,6 +36,7 @@ import 
org.apache.kafka.clients.consumer.internals.Utils.TopicPartitionComparato
 import org.apache.kafka.common.Cluster;
 import org.apache.kafka.common.KafkaException;
 import org.apache.kafka.common.Node;
+import org.apache.kafka.common.PartitionInfo;
 import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.errors.FencedInstanceIdException;
 import org.apache.kafka.common.errors.GroupAuthorizationException;
@@ -174,7 +176,8 @@ public final class ConsumerCoordinator extends 
AbstractCoordinator {
 this.rebalanceConfig = rebalanceConfig;
 this.log = logContext.logger(ConsumerCoordinator.class);
 this.metadata = metadata;
-this.metadataSnapshot = new MetadataSnapshot(subscriptions, 
metadata.fetch(), metadata.updateVersion());
+this.rackId = rackId == null || rackId.isEmpty() ? Optional.empty() : 
Optional.of(rackId);
+this.metadataSnapshot = new MetadataSnapshot(this.rackId, 
subscriptions, metadata.fetch(), metadata.updateVersion());
 this.subscriptions = subscriptions;
 this.defaultOffsetCommitCallback = new DefaultOffsetCommitCallback();
 this.autoCommitEnabled = autoCommitEnabled;
@@ -188,7 +191,6 @@ public final class ConsumerCoordinator extends 
AbstractCoordinator {
 this.groupMetadata = new ConsumerGroupMetadata(rebalanceConfig.groupId,
 JoinGroupRequest.UNKNOWN_GENERATION_ID, 
JoinGroupRequest.UNKNOWN_MEMBER_ID, rebalanceConfig.groupInstanceId);
 this.throwOnFetchStableOffsetsUnsupported = 
throwOnFetchStableOffsetsUnsupported;
-this.rackId = rackId == null || rackId.isEmpty() ? Optional.empty() : 
Optional.of(rackId);
 
 if (autoCommitEnabled)
 this.nextAutoCommitTimer = time.timer(autoCommitIntervalMs);
@@ -489,7 +491,7 @@ public final class ConsumerCoordinator extends 
AbstractCoordinator {
 
 // Update the current snapshot, which will be used to check for 
subscription
 // changes that would require a rebalance (e.g. new partitions).
-metadataSnapshot = new MetadataSnapshot(subscriptions, cluster, 
version);
+metadataSnapshot = new MetadataSnapshot(rackId, subscriptions, 
cluster, version);
 }
 }
 
@@ -1613,14 +1615,18 @@ public final class ConsumerCoordinator extends 
AbstractCoordinator {
 
 private static class MetadataSnapshot {
 private final int version;
-private final Map partitionsPerTopic;
+private final Map> partitionsPerTopic;
 
-private MetadataSnapshot(SubscriptionState subscription, Cluster 
cluster, int version) {
-Map partitionsPerTopic = new HashMap<>();
+private MetadataSnapshot(Optional clientRack, 
SubscriptionState subscription, Cluster cluster, int version) {
+Map> partitionsPerTopic = new 
HashMap<>();
 for (String topic : subscription.metadataTopics()) {
-Integer numPartitions = cluster.partitionCountForTopic(topic);
-if (num

[kafka] branch trunk updated (20e05695f95 -> 1401769d92f)

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

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


from 20e05695f95 KAFKA-14447: remove stale TODO comment (#13258)
 add 1401769d92f KAFKA-14452: Refactor AbstractStickyAssignor to prepare 
for rack-aware assignment (#13349)

No new revisions were added by this update.

Summary of changes:
 .../consumer/internals/AbstractStickyAssignor.java | 1801 ++--
 1 file changed, 911 insertions(+), 890 deletions(-)



[kafka] branch trunk updated: KAFKA-14770: Allow dynamic keystore update for brokers if string representation of DN matches even if canonical DNs don't match (#13346)

2023-03-07 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

rsivaram 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 4d43abf1e09 KAFKA-14770: Allow dynamic keystore update for brokers if 
string representation of DN matches even if canonical DNs don't match (#13346)
4d43abf1e09 is described below

commit 4d43abf1e09e01fc5e7af52f65e3fbae02cf9771
Author: Rajini Sivaram 
AuthorDate: Tue Mar 7 09:41:01 2023 +

KAFKA-14770: Allow dynamic keystore update for brokers if string 
representation of DN matches even if canonical DNs don't match (#13346)

To avoid mistakes during dynamic broker config updates that could 
potentially affect clients, we restrict changes that can be performed 
dynamically without broker restart. For broker keystore updates, we require the 
DN to be the same for the old and new certificates since this could potentially 
contain host names used for host name verification by clients. DNs are compared 
using standard Java implementation of X500Principal.equals() which compares 
canonical names. If tags of fields ch [...]

Reviewers: Manikumar Reddy , Kalpesh Patel 

---
 .../kafka/common/security/ssl/SslFactory.java  |  8 -
 .../kafka/common/security/ssl/SslFactoryTest.java  | 38 ++
 .../java/org/apache/kafka/test/TestSslUtils.java   | 19 +--
 3 files changed, 62 insertions(+), 3 deletions(-)

diff --git 
a/clients/src/main/java/org/apache/kafka/common/security/ssl/SslFactory.java 
b/clients/src/main/java/org/apache/kafka/common/security/ssl/SslFactory.java
index d0cc4cc1e69..65c37aa6b47 100644
--- a/clients/src/main/java/org/apache/kafka/common/security/ssl/SslFactory.java
+++ b/clients/src/main/java/org/apache/kafka/common/security/ssl/SslFactory.java
@@ -312,7 +312,13 @@ public class SslFactory implements Reconfigurable, 
Closeable {
 for (int i = 0; i < newEntries.size(); i++) {
 CertificateEntries newEntry = newEntries.get(i);
 CertificateEntries oldEntry = oldEntries.get(i);
-if (!Objects.equals(newEntry.subjectPrincipal, 
oldEntry.subjectPrincipal)) {
+Principal newPrincipal = newEntry.subjectPrincipal;
+Principal oldPrincipal = oldEntry.subjectPrincipal;
+// Compare principal objects to compare canonical names (e.g. 
to ignore leading/trailing whitespaces).
+// Canonical names may differ if the tags of a field changes 
from one with a printable string representation
+// to one without or vice-versa due to optional conversion to 
hex representation based on the tag. So we
+// also compare Principal.getName which compares the RFC2253 
name. If either matches, allow dynamic update.
+if (!Objects.equals(newPrincipal, oldPrincipal) && 
!newPrincipal.getName().equalsIgnoreCase(oldPrincipal.getName())) {
 throw new ConfigException(String.format("Keystore 
DistinguishedName does not match: " +
 " existing={alias=%s, DN=%s}, new={alias=%s, DN=%s}",
 oldEntry.alias, oldEntry.subjectPrincipal, 
newEntry.alias, newEntry.subjectPrincipal));
diff --git 
a/clients/src/test/java/org/apache/kafka/common/security/ssl/SslFactoryTest.java
 
b/clients/src/test/java/org/apache/kafka/common/security/ssl/SslFactoryTest.java
index 21dcd6e4b0f..7ac707b5de7 100644
--- 
a/clients/src/test/java/org/apache/kafka/common/security/ssl/SslFactoryTest.java
+++ 
b/clients/src/test/java/org/apache/kafka/common/security/ssl/SslFactoryTest.java
@@ -20,7 +20,9 @@ import java.io.File;
 import java.io.IOException;
 import java.nio.file.Files;
 import java.security.GeneralSecurityException;
+import java.security.KeyPair;
 import java.security.KeyStore;
+import java.security.cert.X509Certificate;
 import java.util.Arrays;
 import java.util.Map;
 
@@ -46,6 +48,7 @@ import org.apache.kafka.common.network.Mode;
 import org.apache.kafka.test.TestUtils;
 import org.junit.jupiter.api.Test;
 
+import static 
org.apache.kafka.common.security.ssl.SslFactory.CertificateEntries.ensureCompatible;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertNotEquals;
@@ -521,6 +524,41 @@ public abstract class SslFactoryTest {
 
assertFalse(securityConfig.unused().contains(SslConfigs.SSL_ENGINE_FACTORY_CLASS_CONFIG));
 }
 
+@Test
+public void testDynamicUpdateCompatibility() throws Exception {
+KeyPair keyPair = TestSslUtils.generateKeyPair("RSA");
+KeyStore ks = createKeyStore(keyPair, "*.example.com", "Kafka", true, 
"localhost", "*.example.com");
+ensureCompatible(ks, k

[kafka] branch trunk updated: KAFKA-14451: Rack-aware consumer partition assignment for RangeAssignor (KIP-881) (#12990)

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

rsivaram 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 98d84b17f74 KAFKA-14451: Rack-aware consumer partition assignment for 
RangeAssignor (KIP-881) (#12990)
98d84b17f74 is described below

commit 98d84b17f74b0bfe65163d0ddf88976746de5f7e
Author: Rajini Sivaram 
AuthorDate: Wed Mar 1 21:01:35 2023 +

KAFKA-14451: Rack-aware consumer partition assignment for RangeAssignor 
(KIP-881) (#12990)

Best-effort rack alignment for range assignor when both consumer racks and 
partition racks are available with the protocol changes introduced in KIP-881. 
Rack-aware assignment is enabled by configuring client.rack for consumers. 
Balanced assignment per topic is prioritized over rack-alignment. For topics 
with equal partitions and the same set of subscribers, co-partitioning is 
prioritized over rack-alignment.

Reviewers: David Jacot 
---
 .../kafka/clients/consumer/RangeAssignor.java  | 229 +--
 .../internals/AbstractPartitionAssignor.java   |  66 +++-
 .../kafka/clients/consumer/internals/Utils.java|   4 +-
 .../kafka/clients/consumer/RangeAssignorTest.java  | 436 -
 .../clients/consumer/RoundRobinAssignorTest.java   |  11 +-
 .../internals/AbstractPartitionAssignorTest.java   | 169 
 .../kafka/api/PlaintextConsumerTest.scala  |  36 +-
 .../server/FetchFromFollowerIntegrationTest.scala  |  66 +++-
 8 files changed, 852 insertions(+), 165 deletions(-)

diff --git 
a/clients/src/main/java/org/apache/kafka/clients/consumer/RangeAssignor.java 
b/clients/src/main/java/org/apache/kafka/clients/consumer/RangeAssignor.java
index aec0d3997c4..0b3071a4915 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/RangeAssignor.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/RangeAssignor.java
@@ -17,13 +17,27 @@
 package org.apache.kafka.clients.consumer;
 
 import org.apache.kafka.clients.consumer.internals.AbstractPartitionAssignor;
+import 
org.apache.kafka.clients.consumer.internals.Utils.TopicPartitionComparator;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.PartitionInfo;
 import org.apache.kafka.common.TopicPartition;
 
 import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+import java.util.stream.Collectors;
 
 /**
  * The range assignor works on a per-topic basis. For each topic, we lay 
out the available partitions in numeric order
@@ -63,9 +77,26 @@ import java.util.Map;
  * I0: [t0p0, t0p1, t1p0, t1p1]
  * I1: [t0p2, t1p2]
  * 
+ * 
+ * Rack-aware assignment is used if both consumer and partition replica racks 
are available and
+ * some partitions have replicas only on a subset of racks. We attempt to 
match consumer racks with
+ * partition replica racks on a best-effort basis, prioritizing balanced 
assignment over rack alignment.
+ * Topics with equal partition count and same set of subscribers guarantee 
co-partitioning by prioritizing
+ * co-partitioning over rack alignment. In this case, aligning partition 
replicas of these topics on the
+ * same racks will improve locality for consumers. For example, if partitions 
0 of all topics have a replica
+ * on rack 'a', partition 1 on rack 'b' etc., partition 0 of all topics can be 
assigned to a consumer
+ * on rack 'a', partition 1 to a consumer on rack 'b' and so on.
+ * 
+ * Note that rack-aware assignment currently takes all replicas into account, 
including any offline replicas
+ * and replicas that are not in the ISR. This is based on the assumption that 
these replicas are likely
+ * to join the ISR relatively soon. Since consumers don't rebalance on ISR 
change, this avoids unnecessary
+ * cross-rack traffic for long durations after replicas rejoin the ISR. In the 
future, we may consider
+ * rebalancing when replicas are added or removed to improve consumer rack 
alignment.
+ * 
  */
 public class RangeAssignor extends AbstractPartitionAssignor {
 public static final String RANGE_ASSIGNOR_NAME = "range";
+private static final TopicPartitionComparator PARTITION_COMPARATOR = new 
TopicPartitionComparator();
 
 @Override
 public String name() {
@@ -74,45 +105,193 @@ public class RangeAssignor extends 
AbstractPartitionAssignor {
 
 private Map> consumersPerTopic(Map consumerMetadata) {
 Map> topicToConsumers = new HashMap<>();
-for (Map.Entry subscriptionEntry : 
consumerMetadata.entrySet())

[kafka] branch 3.3 updated: KAFKA-14676: Include all SASL configs in login cache key to ensure clients in a JVM can use different OAuth configs (#13211)

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

rsivaram 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 6a86e54c764 KAFKA-14676: Include all SASL configs in login cache key 
to ensure clients in a JVM can use different OAuth configs (#13211)
6a86e54c764 is described below

commit 6a86e54c7645c3db950640b10340ff347a0bc1a2
Author: Rajini Sivaram 
AuthorDate: Sun Feb 12 19:49:12 2023 +

KAFKA-14676: Include all SASL configs in login cache key to ensure clients 
in a JVM can use different OAuth configs (#13211)

We currently cache login managers in static maps for both static JAAS 
config using system property and for JAAS config specified using Kafka config 
sasl.jaas.config. In addition to the JAAS config, the login manager callback 
handler is included in the key, but all other configs are ignored. This 
implementation is based on the assumption clients that require different logins 
(e.g. username/password) use different JAAS configs, because login properties 
are included in the JAAS config ra [...]

This PR includes all SASL configs prefixed with sasl. to be included in the 
key so that logins are only shared if all the sasl configs are identical.

Reviewers: Manikumar Reddy , Kirk True 

---
 .../security/authenticator/LoginManager.java   | 17 ++---
 .../security/authenticator/LoginManagerTest.java   | 43 ++
 2 files changed, 55 insertions(+), 5 deletions(-)

diff --git 
a/clients/src/main/java/org/apache/kafka/common/security/authenticator/LoginManager.java
 
b/clients/src/main/java/org/apache/kafka/common/security/authenticator/LoginManager.java
index 6613fd147f8..f84a8ac9b75 100644
--- 
a/clients/src/main/java/org/apache/kafka/common/security/authenticator/LoginManager.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/security/authenticator/LoginManager.java
@@ -99,14 +99,14 @@ public class LoginManager {
 LoginManager loginManager;
 Password jaasConfigValue = jaasContext.dynamicJaasConfig();
 if (jaasConfigValue != null) {
-LoginMetadata loginMetadata = new 
LoginMetadata<>(jaasConfigValue, loginClass, loginCallbackClass);
+LoginMetadata loginMetadata = new 
LoginMetadata<>(jaasConfigValue, loginClass, loginCallbackClass, configs);
 loginManager = DYNAMIC_INSTANCES.get(loginMetadata);
 if (loginManager == null) {
 loginManager = new LoginManager(jaasContext, 
saslMechanism, configs, loginMetadata);
 DYNAMIC_INSTANCES.put(loginMetadata, loginManager);
 }
 } else {
-LoginMetadata loginMetadata = new 
LoginMetadata<>(jaasContext.name(), loginClass, loginCallbackClass);
+LoginMetadata loginMetadata = new 
LoginMetadata<>(jaasContext.name(), loginClass, loginCallbackClass, configs);
 loginManager = STATIC_INSTANCES.get(loginMetadata);
 if (loginManager == null) {
 loginManager = new LoginManager(jaasContext, 
saslMechanism, configs, loginMetadata);
@@ -198,17 +198,23 @@ public class LoginManager {
 final T configInfo;
 final Class loginClass;
 final Class loginCallbackClass;
+final Map saslConfigs;
 
 LoginMetadata(T configInfo, Class loginClass,
-  Class 
loginCallbackClass) {
+  Class 
loginCallbackClass,
+  Map configs) {
 this.configInfo = configInfo;
 this.loginClass = loginClass;
 this.loginCallbackClass = loginCallbackClass;
+this.saslConfigs = new HashMap<>();
+configs.entrySet().stream()
+.filter(e -> e.getKey().startsWith("sasl."))
+.forEach(e -> saslConfigs.put(e.getKey(), e.getValue())); 
// value may be null
 }
 
 @Override
 public int hashCode() {
-return Objects.hash(configInfo, loginClass, loginCallbackClass);
+return Objects.hash(configInfo, loginClass, loginCallbackClass, 
saslConfigs);
 }
 
 @Override
@@ -219,7 +225,8 @@ public class LoginManager {
 LoginMetadata loginMetadata = (LoginMetadata) o;
 return Objects.equals(configInfo, loginMetadata.configInfo) &&
Objects.equals(loginClass, loginMetadata.loginClass) &&
-   Objects.equals(loginCallbackClass, 
loginMetadata.loginCallbackClass);
+   Objects.equals(loginCallbackClass, 
loginMetadata.loginCallbackClass) &&
+   Objects.equals(saslConfigs, loginMetadata.saslConfigs);
 }
 }
 }
diff --git 
a/clients/src/test/java/org/apache/kafka/common/security/au

[kafka] branch 3.4 updated: KAFKA-14676: Include all SASL configs in login cache key to ensure clients in a JVM can use different OAuth configs (#13211)

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

rsivaram 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 f7a9f5d6a32 KAFKA-14676: Include all SASL configs in login cache key 
to ensure clients in a JVM can use different OAuth configs (#13211)
f7a9f5d6a32 is described below

commit f7a9f5d6a3203bcf2df75ea52bb0ba5a9f7474a6
Author: Rajini Sivaram 
AuthorDate: Sun Feb 12 19:49:12 2023 +

KAFKA-14676: Include all SASL configs in login cache key to ensure clients 
in a JVM can use different OAuth configs (#13211)

We currently cache login managers in static maps for both static JAAS 
config using system property and for JAAS config specified using Kafka config 
sasl.jaas.config. In addition to the JAAS config, the login manager callback 
handler is included in the key, but all other configs are ignored. This 
implementation is based on the assumption clients that require different logins 
(e.g. username/password) use different JAAS configs, because login properties 
are included in the JAAS config ra [...]

This PR includes all SASL configs prefixed with sasl. to be included in the 
key so that logins are only shared if all the sasl configs are identical.

Reviewers: Manikumar Reddy , Kirk True 

---
 .../security/authenticator/LoginManager.java   | 17 ++---
 .../security/authenticator/LoginManagerTest.java   | 43 ++
 2 files changed, 55 insertions(+), 5 deletions(-)

diff --git 
a/clients/src/main/java/org/apache/kafka/common/security/authenticator/LoginManager.java
 
b/clients/src/main/java/org/apache/kafka/common/security/authenticator/LoginManager.java
index 6613fd147f8..f84a8ac9b75 100644
--- 
a/clients/src/main/java/org/apache/kafka/common/security/authenticator/LoginManager.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/security/authenticator/LoginManager.java
@@ -99,14 +99,14 @@ public class LoginManager {
 LoginManager loginManager;
 Password jaasConfigValue = jaasContext.dynamicJaasConfig();
 if (jaasConfigValue != null) {
-LoginMetadata loginMetadata = new 
LoginMetadata<>(jaasConfigValue, loginClass, loginCallbackClass);
+LoginMetadata loginMetadata = new 
LoginMetadata<>(jaasConfigValue, loginClass, loginCallbackClass, configs);
 loginManager = DYNAMIC_INSTANCES.get(loginMetadata);
 if (loginManager == null) {
 loginManager = new LoginManager(jaasContext, 
saslMechanism, configs, loginMetadata);
 DYNAMIC_INSTANCES.put(loginMetadata, loginManager);
 }
 } else {
-LoginMetadata loginMetadata = new 
LoginMetadata<>(jaasContext.name(), loginClass, loginCallbackClass);
+LoginMetadata loginMetadata = new 
LoginMetadata<>(jaasContext.name(), loginClass, loginCallbackClass, configs);
 loginManager = STATIC_INSTANCES.get(loginMetadata);
 if (loginManager == null) {
 loginManager = new LoginManager(jaasContext, 
saslMechanism, configs, loginMetadata);
@@ -198,17 +198,23 @@ public class LoginManager {
 final T configInfo;
 final Class loginClass;
 final Class loginCallbackClass;
+final Map saslConfigs;
 
 LoginMetadata(T configInfo, Class loginClass,
-  Class 
loginCallbackClass) {
+  Class 
loginCallbackClass,
+  Map configs) {
 this.configInfo = configInfo;
 this.loginClass = loginClass;
 this.loginCallbackClass = loginCallbackClass;
+this.saslConfigs = new HashMap<>();
+configs.entrySet().stream()
+.filter(e -> e.getKey().startsWith("sasl."))
+.forEach(e -> saslConfigs.put(e.getKey(), e.getValue())); 
// value may be null
 }
 
 @Override
 public int hashCode() {
-return Objects.hash(configInfo, loginClass, loginCallbackClass);
+return Objects.hash(configInfo, loginClass, loginCallbackClass, 
saslConfigs);
 }
 
 @Override
@@ -219,7 +225,8 @@ public class LoginManager {
 LoginMetadata loginMetadata = (LoginMetadata) o;
 return Objects.equals(configInfo, loginMetadata.configInfo) &&
Objects.equals(loginClass, loginMetadata.loginClass) &&
-   Objects.equals(loginCallbackClass, 
loginMetadata.loginCallbackClass);
+   Objects.equals(loginCallbackClass, 
loginMetadata.loginCallbackClass) &&
+   Objects.equals(saslConfigs, loginMetadata.saslConfigs);
 }
 }
 }
diff --git 
a/clients/src/test/java/org/apache/kafka/common/security/au

[kafka] branch trunk updated: KAFKA-14676: Include all SASL configs in login cache key to ensure clients in a JVM can use different OAuth configs (#13211)

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

rsivaram 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 ef9c9486dac KAFKA-14676: Include all SASL configs in login cache key 
to ensure clients in a JVM can use different OAuth configs (#13211)
ef9c9486dac is described below

commit ef9c9486dac8d6076e2897580b10a699697205ac
Author: Rajini Sivaram 
AuthorDate: Sun Feb 12 19:49:12 2023 +

KAFKA-14676: Include all SASL configs in login cache key to ensure clients 
in a JVM can use different OAuth configs (#13211)

We currently cache login managers in static maps for both static JAAS 
config using system property and for JAAS config specified using Kafka config 
sasl.jaas.config. In addition to the JAAS config, the login manager callback 
handler is included in the key, but all other configs are ignored. This 
implementation is based on the assumption clients that require different logins 
(e.g. username/password) use different JAAS configs, because login properties 
are included in the JAAS config ra [...]

This PR includes all SASL configs prefixed with sasl. to be included in the 
key so that logins are only shared if all the sasl configs are identical.

Reviewers: Manikumar Reddy , Kirk True 

---
 .../security/authenticator/LoginManager.java   | 17 ++---
 .../security/authenticator/LoginManagerTest.java   | 43 ++
 2 files changed, 55 insertions(+), 5 deletions(-)

diff --git 
a/clients/src/main/java/org/apache/kafka/common/security/authenticator/LoginManager.java
 
b/clients/src/main/java/org/apache/kafka/common/security/authenticator/LoginManager.java
index 6613fd147f8..f84a8ac9b75 100644
--- 
a/clients/src/main/java/org/apache/kafka/common/security/authenticator/LoginManager.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/security/authenticator/LoginManager.java
@@ -99,14 +99,14 @@ public class LoginManager {
 LoginManager loginManager;
 Password jaasConfigValue = jaasContext.dynamicJaasConfig();
 if (jaasConfigValue != null) {
-LoginMetadata loginMetadata = new 
LoginMetadata<>(jaasConfigValue, loginClass, loginCallbackClass);
+LoginMetadata loginMetadata = new 
LoginMetadata<>(jaasConfigValue, loginClass, loginCallbackClass, configs);
 loginManager = DYNAMIC_INSTANCES.get(loginMetadata);
 if (loginManager == null) {
 loginManager = new LoginManager(jaasContext, 
saslMechanism, configs, loginMetadata);
 DYNAMIC_INSTANCES.put(loginMetadata, loginManager);
 }
 } else {
-LoginMetadata loginMetadata = new 
LoginMetadata<>(jaasContext.name(), loginClass, loginCallbackClass);
+LoginMetadata loginMetadata = new 
LoginMetadata<>(jaasContext.name(), loginClass, loginCallbackClass, configs);
 loginManager = STATIC_INSTANCES.get(loginMetadata);
 if (loginManager == null) {
 loginManager = new LoginManager(jaasContext, 
saslMechanism, configs, loginMetadata);
@@ -198,17 +198,23 @@ public class LoginManager {
 final T configInfo;
 final Class loginClass;
 final Class loginCallbackClass;
+final Map saslConfigs;
 
 LoginMetadata(T configInfo, Class loginClass,
-  Class 
loginCallbackClass) {
+  Class 
loginCallbackClass,
+  Map configs) {
 this.configInfo = configInfo;
 this.loginClass = loginClass;
 this.loginCallbackClass = loginCallbackClass;
+this.saslConfigs = new HashMap<>();
+configs.entrySet().stream()
+.filter(e -> e.getKey().startsWith("sasl."))
+.forEach(e -> saslConfigs.put(e.getKey(), e.getValue())); 
// value may be null
 }
 
 @Override
 public int hashCode() {
-return Objects.hash(configInfo, loginClass, loginCallbackClass);
+return Objects.hash(configInfo, loginClass, loginCallbackClass, 
saslConfigs);
 }
 
 @Override
@@ -219,7 +225,8 @@ public class LoginManager {
 LoginMetadata loginMetadata = (LoginMetadata) o;
 return Objects.equals(configInfo, loginMetadata.configInfo) &&
Objects.equals(loginClass, loginMetadata.loginClass) &&
-   Objects.equals(loginCallbackClass, 
loginMetadata.loginCallbackClass);
+   Objects.equals(loginCallbackClass, 
loginMetadata.loginCallbackClass) &&
+   Objects.equals(saslConfigs, loginMetadata.saslConfigs);
 }
 }
 }
diff --git 
a/clients/src/test/java/org/apache/kafka/common/secu

[kafka] branch trunk updated (3799708ff09 -> bc1ce9f0f1b)

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

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


from 3799708ff09 KAFKA-14533: re-enable 'false' and disable the 'true' 
parameter of SmokeTestDriverIntegrationTest (#13156)
 add bc1ce9f0f1b KAFKA-14623: OAuth's HttpAccessTokenRetriever potentially 
leaks secrets in logging (#13119)

No new revisions were added by this update.

Summary of changes:
 .../internals/secured/HttpAccessTokenRetriever.java | 17 -
 1 file changed, 12 insertions(+), 5 deletions(-)



[kafka] branch 3.4 updated: KAFKA-14352: Rack-aware consumer partition assignment protocol changes (KIP-881) (#12954)

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

rsivaram 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 f5f1060a81d KAFKA-14352: Rack-aware consumer partition assignment 
protocol changes (KIP-881) (#12954)
f5f1060a81d is described below

commit f5f1060a81dd200e573ef69574ea75b64b73474c
Author: Rajini Sivaram 
AuthorDate: Wed Dec 7 11:41:21 2022 +

KAFKA-14352: Rack-aware consumer partition assignment protocol changes 
(KIP-881) (#12954)

Reviewers: David Jacot 
---
 checkstyle/suppressions.xml|  2 +-
 .../consumer/ConsumerPartitionAssignor.java| 15 +++--
 .../kafka/clients/consumer/KafkaConsumer.java  |  3 +-
 .../consumer/internals/ConsumerCoordinator.java|  8 ++-
 .../consumer/internals/ConsumerProtocol.java   |  5 +-
 .../common/message/ConsumerProtocolAssignment.json |  3 +-
 .../message/ConsumerProtocolSubscription.json  |  6 +-
 .../consumer/CooperativeStickyAssignorTest.java|  8 +--
 .../kafka/clients/consumer/KafkaConsumerTest.java  |  3 +-
 .../kafka/clients/consumer/StickyAssignorTest.java |  8 +--
 .../internals/ConsumerCoordinatorTest.java | 67 +++---
 .../consumer/internals/ConsumerProtocolTest.java   | 23 ++--
 .../kafka/api/PlaintextConsumerTest.scala  | 17 ++
 13 files changed, 136 insertions(+), 32 deletions(-)

diff --git a/checkstyle/suppressions.xml b/checkstyle/suppressions.xml
index ac0accc17fa..cea6a193790 100644
--- a/checkstyle/suppressions.xml
+++ b/checkstyle/suppressions.xml
@@ -66,7 +66,7 @@
 
 
+  files="(KafkaConsumer|ConsumerCoordinator).java"/>
 
  topics;
 private final ByteBuffer userData;
 private final List ownedPartitions;
+private final Optional rackId;
 private Optional groupInstanceId;
 private final Optional generationId;
 
-public Subscription(List topics, ByteBuffer userData, 
List ownedPartitions, int generationId) {
+public Subscription(List topics, ByteBuffer userData, 
List ownedPartitions, int generationId, Optional 
rackId) {
 this.topics = topics;
 this.userData = userData;
 this.ownedPartitions = ownedPartitions;
 this.groupInstanceId = Optional.empty();
 this.generationId = generationId < 0 ? Optional.empty() : 
Optional.of(generationId);
+this.rackId = rackId;
 }
 
 public Subscription(List topics, ByteBuffer userData, 
List ownedPartitions) {
-this(topics, userData, ownedPartitions, DEFAULT_GENERATION);
+this(topics, userData, ownedPartitions, DEFAULT_GENERATION, 
Optional.empty());
 }
 
 public Subscription(List topics, ByteBuffer userData) {
-this(topics, userData, Collections.emptyList(), 
DEFAULT_GENERATION);
+this(topics, userData, Collections.emptyList(), 
DEFAULT_GENERATION, Optional.empty());
 }
 
 public Subscription(List topics) {
-this(topics, null, Collections.emptyList(), DEFAULT_GENERATION);
+this(topics, null, Collections.emptyList(), DEFAULT_GENERATION, 
Optional.empty());
 }
 
 public List topics() {
@@ -138,6 +140,10 @@ public interface ConsumerPartitionAssignor {
 return ownedPartitions;
 }
 
+public Optional rackId() {
+return rackId;
+}
+
 public void setGroupInstanceId(Optional groupInstanceId) {
 this.groupInstanceId = groupInstanceId;
 }
@@ -158,6 +164,7 @@ public interface ConsumerPartitionAssignor {
 ", ownedPartitions=" + ownedPartitions +
 ", groupInstanceId=" + 
groupInstanceId.map(String::toString).orElse("null") +
 ", generationId=" + generationId.orElse(-1) +
+", rackId=" + (rackId.orElse("null")) +
 ")";
 }
 }
diff --git 
a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java 
b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java
index f07846945a7..cf85798f82b 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java
@@ -790,7 +790,8 @@ public class KafkaConsumer implements Consumer {
 enableAutoCommit,
 
config.getInt(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG),
 this.interceptors,
-
config.getBoolean(ConsumerConfig.THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED));
+
config.getBoolean(ConsumerConfig.THROW_ON_FETCH_STABLE_OFFSET_UNS

[kafka] branch trunk updated: KAFKA-14352: Rack-aware consumer partition assignment protocol changes (KIP-881) (#12954)

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

rsivaram 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 d23ce20bdfb KAFKA-14352: Rack-aware consumer partition assignment 
protocol changes (KIP-881) (#12954)
d23ce20bdfb is described below

commit d23ce20bdfbe5a9598523961cb7cf747ce4f52ef
Author: Rajini Sivaram 
AuthorDate: Wed Dec 7 11:41:21 2022 +

KAFKA-14352: Rack-aware consumer partition assignment protocol changes 
(KIP-881) (#12954)

Reviewers: David Jacot 
---
 checkstyle/suppressions.xml|  2 +-
 .../consumer/ConsumerPartitionAssignor.java| 15 +++--
 .../kafka/clients/consumer/KafkaConsumer.java  |  3 +-
 .../consumer/internals/ConsumerCoordinator.java|  8 ++-
 .../consumer/internals/ConsumerProtocol.java   |  5 +-
 .../common/message/ConsumerProtocolAssignment.json |  3 +-
 .../message/ConsumerProtocolSubscription.json  |  6 +-
 .../consumer/CooperativeStickyAssignorTest.java|  8 +--
 .../kafka/clients/consumer/KafkaConsumerTest.java  |  3 +-
 .../kafka/clients/consumer/StickyAssignorTest.java |  8 +--
 .../internals/ConsumerCoordinatorTest.java | 67 +++---
 .../consumer/internals/ConsumerProtocolTest.java   | 23 ++--
 .../kafka/api/PlaintextConsumerTest.scala  | 17 ++
 13 files changed, 136 insertions(+), 32 deletions(-)

diff --git a/checkstyle/suppressions.xml b/checkstyle/suppressions.xml
index ac0accc17fa..cea6a193790 100644
--- a/checkstyle/suppressions.xml
+++ b/checkstyle/suppressions.xml
@@ -66,7 +66,7 @@
 
 
+  files="(KafkaConsumer|ConsumerCoordinator).java"/>
 
  topics;
 private final ByteBuffer userData;
 private final List ownedPartitions;
+private final Optional rackId;
 private Optional groupInstanceId;
 private final Optional generationId;
 
-public Subscription(List topics, ByteBuffer userData, 
List ownedPartitions, int generationId) {
+public Subscription(List topics, ByteBuffer userData, 
List ownedPartitions, int generationId, Optional 
rackId) {
 this.topics = topics;
 this.userData = userData;
 this.ownedPartitions = ownedPartitions;
 this.groupInstanceId = Optional.empty();
 this.generationId = generationId < 0 ? Optional.empty() : 
Optional.of(generationId);
+this.rackId = rackId;
 }
 
 public Subscription(List topics, ByteBuffer userData, 
List ownedPartitions) {
-this(topics, userData, ownedPartitions, DEFAULT_GENERATION);
+this(topics, userData, ownedPartitions, DEFAULT_GENERATION, 
Optional.empty());
 }
 
 public Subscription(List topics, ByteBuffer userData) {
-this(topics, userData, Collections.emptyList(), 
DEFAULT_GENERATION);
+this(topics, userData, Collections.emptyList(), 
DEFAULT_GENERATION, Optional.empty());
 }
 
 public Subscription(List topics) {
-this(topics, null, Collections.emptyList(), DEFAULT_GENERATION);
+this(topics, null, Collections.emptyList(), DEFAULT_GENERATION, 
Optional.empty());
 }
 
 public List topics() {
@@ -138,6 +140,10 @@ public interface ConsumerPartitionAssignor {
 return ownedPartitions;
 }
 
+public Optional rackId() {
+return rackId;
+}
+
 public void setGroupInstanceId(Optional groupInstanceId) {
 this.groupInstanceId = groupInstanceId;
 }
@@ -158,6 +164,7 @@ public interface ConsumerPartitionAssignor {
 ", ownedPartitions=" + ownedPartitions +
 ", groupInstanceId=" + 
groupInstanceId.map(String::toString).orElse("null") +
 ", generationId=" + generationId.orElse(-1) +
+", rackId=" + (rackId.orElse("null")) +
 ")";
 }
 }
diff --git 
a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java 
b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java
index f07846945a7..cf85798f82b 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java
@@ -790,7 +790,8 @@ public class KafkaConsumer implements Consumer {
 enableAutoCommit,
 
config.getInt(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG),
 this.interceptors,
-
config.getBoolean(ConsumerConfig.THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED));
+
config.getBoolean(ConsumerConfig.THROW_ON_FETCH_STABLE_OFFSET_UNS

[kafka] branch trunk updated: KAFKA-13559: Fix issue where responses intermittently takes 300+ ms to respond, even when the server is idle. (#12416)

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

rsivaram 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 d529d86aa4b KAFKA-13559: Fix issue where responses intermittently 
takes 300+ ms to respond, even when the server is idle. (#12416)
d529d86aa4b is described below

commit d529d86aa4be533d1251cfc0b4c0fb57c69ace72
Author: Badai Aqrandista 
AuthorDate: Mon Aug 15 21:34:03 2022 +1000

KAFKA-13559: Fix issue where responses intermittently takes 300+ ms to 
respond, even when the server is idle. (#12416)

Ensures that SSL buffered data is processed by server immediately on the 
next poll when channel is unmuted after processing previous request. Poll 
timeout is reset to zero for this case to avoid 300ms delay in poll() if no new 
data arrives on the sockets.

Reviewers: David Mao , Ismael Juma , 
Rajini Sivaram 
---
 .../org/apache/kafka/common/network/Selector.java  |  1 +
 .../unit/kafka/network/SocketServerTest.scala  | 44 +-
 2 files changed, 43 insertions(+), 2 deletions(-)

diff --git 
a/clients/src/main/java/org/apache/kafka/common/network/Selector.java 
b/clients/src/main/java/org/apache/kafka/common/network/Selector.java
index bd1175a8ee0..2e581187625 100644
--- a/clients/src/main/java/org/apache/kafka/common/network/Selector.java
+++ b/clients/src/main/java/org/apache/kafka/common/network/Selector.java
@@ -757,6 +757,7 @@ public class Selector implements Selectable, AutoCloseable {
 explicitlyMutedChannels.remove(channel);
 if (channel.hasBytesBuffered()) {
 keysWithBufferedRead.add(channel.selectionKey());
+madeReadProgressLastPoll = true;
 }
 }
 }
diff --git a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala 
b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala
index 98f92d61ff2..801a2d83cab 100644
--- a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala
+++ b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala
@@ -1878,6 +1878,44 @@ class SocketServerTest {
 }, false)
   }
 
+  /**
+   * Test to ensure "Selector.poll()" does not block at "select(timeout)" when 
there is no data in the socket but there
+   * is data in the buffer. This only happens when SSL protocol is used.
+   */
+  @Test
+  def testLatencyWithBufferedDataAndNoSocketData(): Unit = {
+shutdownServerAndMetrics(server)
+
+props ++= sslServerProps
+val testableServer = new TestableSocketServer(KafkaConfig.fromProps(props))
+testableServer.enableRequestProcessing(Map.empty)
+val testableSelector = testableServer.testableSelector
+val proxyServer = new ProxyServer(testableServer)
+val selectTimeoutMs = 5000
+// set pollTimeoutOverride to "selectTimeoutMs" to ensure poll() timeout 
is distinct and can be identified
+testableSelector.pollTimeoutOverride = Some(selectTimeoutMs)
+
+try {
+  // initiate SSL connection by sending 1 request via socket, then send 2 
requests directly into the netReadBuffer
+  val (sslSocket, req1) = makeSocketWithBufferedRequests(testableServer, 
testableSelector, proxyServer)
+
+  // force all data to be transferred to the kafka broker by closing the 
client connection to proxy server
+  sslSocket.close()
+  TestUtils.waitUntilTrue(() => proxyServer.clientConnSocket.isClosed, 
"proxyServer.clientConnSocket is still not closed after 6 ms", 6)
+
+  // process the request and send the response
+  processRequest(testableServer.dataPlaneRequestChannel, req1)
+
+  // process the requests in the netReadBuffer, this should not block
+  val req2 = receiveRequest(testableServer.dataPlaneRequestChannel)
+  processRequest(testableServer.dataPlaneRequestChannel, req2)
+
+} finally {
+  proxyServer.close()
+  shutdownServerAndMetrics(testableServer)
+}
+  }
+
   private def sslServerProps: Properties = {
 val trustStoreFile = File.createTempFile("truststore", ".jks")
 val sslProps = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, 
interBrokerSecurityProtocol = Some(SecurityProtocol.SSL),
@@ -2044,10 +2082,12 @@ class SocketServerTest {
 }
 
 def testableSelector: TestableSelector =
-  
dataPlaneAcceptors.get(endpoint).processors(0).selector.asInstanceOf[TestableSelector]
+  testableProcessor.selector.asInstanceOf[TestableSelector]
 
-def testableProcessor: TestableProcessor =
+def testableProcessor: TestableProcessor = {
+  val endpoint = this.config.dataPlaneListeners.head
   
dataPlaneAcceptors.get(endpoint).processors(0).asInstanceOf[TestableProcessor]
+}
 
 def waitForChannelClose(connectionId: String, locallyClosed: Boolean): 
Unit = {
   val selector = testableSelector



[kafka] branch 3.3 updated: MINOR: Fix options for old-style Admin.listConsumerGroupOffsets (#12406)

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

rsivaram 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 610780668e MINOR: Fix options for old-style 
Admin.listConsumerGroupOffsets (#12406)
610780668e is described below

commit 610780668efa7c1e8d1be193985eb6e4d971fa0a
Author: Rajini Sivaram 
AuthorDate: Fri Jul 15 09:21:35 2022 +0100

MINOR: Fix options for old-style Admin.listConsumerGroupOffsets (#12406)

Reviewers: David Jacot 
---
 .../java/org/apache/kafka/clients/admin/Admin.java |  7 +--
 .../kafka/clients/admin/KafkaAdminClientTest.java  | 52 +++---
 2 files changed, 40 insertions(+), 19 deletions(-)

diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java 
b/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
index 0698d29702..1d469a6643 100644
--- a/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
+++ b/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
@@ -921,12 +921,13 @@ public interface Admin extends AutoCloseable {
  * @return The ListGroupOffsetsResult
  */
 default ListConsumerGroupOffsetsResult listConsumerGroupOffsets(String 
groupId, ListConsumerGroupOffsetsOptions options) {
-ListConsumerGroupOffsetsOptions listOptions = new 
ListConsumerGroupOffsetsOptions()
-.requireStable(options.requireStable());
 @SuppressWarnings("deprecation")
 ListConsumerGroupOffsetsSpec groupSpec = new 
ListConsumerGroupOffsetsSpec()
 .topicPartitions(options.topicPartitions());
-return listConsumerGroupOffsets(Collections.singletonMap(groupId, 
groupSpec), listOptions);
+
+// We can use the provided options with the batched API, which uses 
topic partitions from
+// the group spec and ignores any topic partitions set in the options.
+return listConsumerGroupOffsets(Collections.singletonMap(groupId, 
groupSpec), options);
 }
 
 /**
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 3d285a45f7..de57813679 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
@@ -131,6 +131,8 @@ import 
org.apache.kafka.common.message.OffsetDeleteResponseData.OffsetDeleteResp
 import 
org.apache.kafka.common.message.OffsetDeleteResponseData.OffsetDeleteResponseTopic;
 import 
org.apache.kafka.common.message.OffsetDeleteResponseData.OffsetDeleteResponseTopicCollection;
 import org.apache.kafka.common.message.OffsetFetchRequestData;
+import 
org.apache.kafka.common.message.OffsetFetchRequestData.OffsetFetchRequestGroup;
+import 
org.apache.kafka.common.message.OffsetFetchRequestData.OffsetFetchRequestTopics;
 import org.apache.kafka.common.message.UnregisterBrokerResponseData;
 import org.apache.kafka.common.message.WriteTxnMarkersResponseData;
 import org.apache.kafka.common.protocol.ApiKeys;
@@ -3075,7 +3077,17 @@ public class KafkaAdminClientTest {
 }
 
 @Test
-public void testListConsumerGroupOffsetsOptions() throws Exception {
+public void testListConsumerGroupOffsetsOptionsWithUnbatchedApi() throws 
Exception {
+verifyListConsumerGroupOffsetsOptions(false);
+}
+
+@Test
+public void testListConsumerGroupOffsetsOptionsWithBatchedApi() throws 
Exception {
+verifyListConsumerGroupOffsetsOptions(true);
+}
+
+@SuppressWarnings("deprecation")
+private void verifyListConsumerGroupOffsetsOptions(boolean batchedApi) 
throws Exception {
 final Cluster cluster = mockCluster(3, 0);
 final Time time = new MockTime();
 
@@ -3085,24 +3097,32 @@ public class KafkaAdminClientTest {
 
 
env.kafkaClient().prepareResponse(prepareFindCoordinatorResponse(Errors.NONE, 
env.cluster().controller()));
 
-final TopicPartition tp1 = new TopicPartition("A", 0);
+final List partitions = 
Collections.singletonList(new TopicPartition("A", 0));
 final ListConsumerGroupOffsetsOptions options = new 
ListConsumerGroupOffsetsOptions()
-.requireStable(true);
-final ListConsumerGroupOffsetsSpec groupSpec = new 
ListConsumerGroupOffsetsSpec()
-.topicPartitions(Collections.singletonList(tp1));
-
env.adminClient().listConsumerGroupOffsets(Collections.singletonMap(GROUP_ID, 
groupSpec), options);
+.requireStable(true)
+.timeoutMs(300);
+if (batchedApi) {
+final ListConsumerGroupOffsetsSpec groupSpec = new 
ListConsumerGroupOffsetsSpec()
+.topicPartitions(partit

[kafka] branch trunk updated: MINOR: Fix options for old-style Admin.listConsumerGroupOffsets (#12406)

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

rsivaram 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 ddbc030036 MINOR: Fix options for old-style 
Admin.listConsumerGroupOffsets (#12406)
ddbc030036 is described below

commit ddbc0300365dd1d9a2fb2c73faef8c4cbec0b316
Author: Rajini Sivaram 
AuthorDate: Fri Jul 15 09:21:35 2022 +0100

MINOR: Fix options for old-style Admin.listConsumerGroupOffsets (#12406)

Reviewers: David Jacot 
---
 .../java/org/apache/kafka/clients/admin/Admin.java |  7 +--
 .../kafka/clients/admin/KafkaAdminClientTest.java  | 52 +++---
 2 files changed, 40 insertions(+), 19 deletions(-)

diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java 
b/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
index 0698d29702..1d469a6643 100644
--- a/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
+++ b/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
@@ -921,12 +921,13 @@ public interface Admin extends AutoCloseable {
  * @return The ListGroupOffsetsResult
  */
 default ListConsumerGroupOffsetsResult listConsumerGroupOffsets(String 
groupId, ListConsumerGroupOffsetsOptions options) {
-ListConsumerGroupOffsetsOptions listOptions = new 
ListConsumerGroupOffsetsOptions()
-.requireStable(options.requireStable());
 @SuppressWarnings("deprecation")
 ListConsumerGroupOffsetsSpec groupSpec = new 
ListConsumerGroupOffsetsSpec()
 .topicPartitions(options.topicPartitions());
-return listConsumerGroupOffsets(Collections.singletonMap(groupId, 
groupSpec), listOptions);
+
+// We can use the provided options with the batched API, which uses 
topic partitions from
+// the group spec and ignores any topic partitions set in the options.
+return listConsumerGroupOffsets(Collections.singletonMap(groupId, 
groupSpec), options);
 }
 
 /**
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 3d285a45f7..de57813679 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
@@ -131,6 +131,8 @@ import 
org.apache.kafka.common.message.OffsetDeleteResponseData.OffsetDeleteResp
 import 
org.apache.kafka.common.message.OffsetDeleteResponseData.OffsetDeleteResponseTopic;
 import 
org.apache.kafka.common.message.OffsetDeleteResponseData.OffsetDeleteResponseTopicCollection;
 import org.apache.kafka.common.message.OffsetFetchRequestData;
+import 
org.apache.kafka.common.message.OffsetFetchRequestData.OffsetFetchRequestGroup;
+import 
org.apache.kafka.common.message.OffsetFetchRequestData.OffsetFetchRequestTopics;
 import org.apache.kafka.common.message.UnregisterBrokerResponseData;
 import org.apache.kafka.common.message.WriteTxnMarkersResponseData;
 import org.apache.kafka.common.protocol.ApiKeys;
@@ -3075,7 +3077,17 @@ public class KafkaAdminClientTest {
 }
 
 @Test
-public void testListConsumerGroupOffsetsOptions() throws Exception {
+public void testListConsumerGroupOffsetsOptionsWithUnbatchedApi() throws 
Exception {
+verifyListConsumerGroupOffsetsOptions(false);
+}
+
+@Test
+public void testListConsumerGroupOffsetsOptionsWithBatchedApi() throws 
Exception {
+verifyListConsumerGroupOffsetsOptions(true);
+}
+
+@SuppressWarnings("deprecation")
+private void verifyListConsumerGroupOffsetsOptions(boolean batchedApi) 
throws Exception {
 final Cluster cluster = mockCluster(3, 0);
 final Time time = new MockTime();
 
@@ -3085,24 +3097,32 @@ public class KafkaAdminClientTest {
 
 
env.kafkaClient().prepareResponse(prepareFindCoordinatorResponse(Errors.NONE, 
env.cluster().controller()));
 
-final TopicPartition tp1 = new TopicPartition("A", 0);
+final List partitions = 
Collections.singletonList(new TopicPartition("A", 0));
 final ListConsumerGroupOffsetsOptions options = new 
ListConsumerGroupOffsetsOptions()
-.requireStable(true);
-final ListConsumerGroupOffsetsSpec groupSpec = new 
ListConsumerGroupOffsetsSpec()
-.topicPartitions(Collections.singletonList(tp1));
-
env.adminClient().listConsumerGroupOffsets(Collections.singletonMap(GROUP_ID, 
groupSpec), options);
+.requireStable(true)
+.timeoutMs(300);
+if (batchedApi) {
+final ListConsumerGroupOffsetsSpec groupSpec = new 
ListConsumerGroupOffsetsSpec()
+.topicPartitions(partit

[kafka] branch 3.3 updated: KAFKA-13043: Implement Admin APIs for offsetFetch batching (#10964)

2022-07-14 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

rsivaram 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 66000787c1 KAFKA-13043: Implement Admin APIs for offsetFetch batching 
(#10964)
66000787c1 is described below

commit 66000787c1146c6d08d88b9c564f5a000608f013
Author: Sanjana Kaundinya 
AuthorDate: Thu Jul 14 05:47:34 2022 -0700

KAFKA-13043: Implement Admin APIs for offsetFetch batching (#10964)

This implements the AdminAPI portion of KIP-709: 
https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=173084258. The 
request/response protocol changes were implemented in 3.0.0. A new batched API 
has been introduced to list consumer offsets for different groups. For brokers 
older than 3.0.0, separate requests are sent for each group.

Co-authored-by: Rajini Sivaram 
Co-authored-by: David Jacot 

Reviewers: David Jacot ,  Rajini Sivaram 

---
 .../java/org/apache/kafka/clients/admin/Admin.java |  36 ++-
 .../kafka/clients/admin/KafkaAdminClient.java  |  11 +-
 .../admin/ListConsumerGroupOffsetsOptions.java |  14 +-
 .../admin/ListConsumerGroupOffsetsResult.java  |  56 +++-
 .../admin/ListConsumerGroupOffsetsSpec.java|  79 ++
 .../clients/admin/internals/AdminApiDriver.java|   3 +-
 .../admin/internals/CoordinatorStrategy.java   |   4 +
 .../internals/ListConsumerGroupOffsetsHandler.java | 128 +
 .../kafka/common/requests/OffsetFetchResponse.java |  10 +-
 .../kafka/clients/admin/AdminClientTestUtils.java  |  12 +-
 .../kafka/clients/admin/KafkaAdminClientTest.java  | 206 --
 .../kafka/clients/admin/MockAdminClient.java   |  16 +-
 .../ListConsumerGroupOffsetsHandlerTest.java   | 308 +++--
 .../kafka/clients/consumer/KafkaConsumerTest.java  |   6 +-
 .../internals/ConsumerCoordinatorTest.java |  26 +-
 .../scala/kafka/admin/ConsumerGroupCommand.scala   |   8 +-
 .../kafka/admin/ConsumerGroupServiceTest.scala |  22 +-
 .../scala/unit/kafka/server/RequestQuotaTest.scala |   2 +-
 .../processor/internals/StoreChangelogReader.java  |  12 +-
 .../internals/StoreChangelogReaderTest.java|  11 +-
 20 files changed, 813 insertions(+), 157 deletions(-)

diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java 
b/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
index fdacc09db8..0698d29702 100644
--- a/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
+++ b/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
@@ -36,6 +36,7 @@ import org.apache.kafka.common.requests.LeaveGroupResponse;
 
 import java.time.Duration;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.Optional;
@@ -919,12 +920,20 @@ public interface Admin extends AutoCloseable {
  * @param options The options to use when listing the consumer group 
offsets.
  * @return The ListGroupOffsetsResult
  */
-ListConsumerGroupOffsetsResult listConsumerGroupOffsets(String groupId, 
ListConsumerGroupOffsetsOptions options);
+default ListConsumerGroupOffsetsResult listConsumerGroupOffsets(String 
groupId, ListConsumerGroupOffsetsOptions options) {
+ListConsumerGroupOffsetsOptions listOptions = new 
ListConsumerGroupOffsetsOptions()
+.requireStable(options.requireStable());
+@SuppressWarnings("deprecation")
+ListConsumerGroupOffsetsSpec groupSpec = new 
ListConsumerGroupOffsetsSpec()
+.topicPartitions(options.topicPartitions());
+return listConsumerGroupOffsets(Collections.singletonMap(groupId, 
groupSpec), listOptions);
+}
 
 /**
  * List the consumer group offsets available in the cluster with the 
default options.
  * 
- * This is a convenience method for {@link 
#listConsumerGroupOffsets(String, ListConsumerGroupOffsetsOptions)} with 
default options.
+ * This is a convenience method for {@link #listConsumerGroupOffsets(Map, 
ListConsumerGroupOffsetsOptions)}
+ * to list offsets of all partitions of one group with default options.
  *
  * @return The ListGroupOffsetsResult.
  */
@@ -932,6 +941,29 @@ public interface Admin extends AutoCloseable {
 return listConsumerGroupOffsets(groupId, new 
ListConsumerGroupOffsetsOptions());
 }
 
+/**
+ * List the consumer group offsets available in the cluster for the 
specified consumer groups.
+ *
+ * @param groupSpecs Map of consumer group ids to a spec that specifies 
the topic partitions of the group to list offsets for.
+ *
+ * @param options The options to use when listing the consumer group 
offsets.
+ * @return The ListConsumerGroupOffsetsResult
+ */
+ListConsumerGroupOffsetsResult listConsumerGroupOffsets(Map 

[kafka] branch trunk updated: KAFKA-13043: Implement Admin APIs for offsetFetch batching (#10964)

2022-07-14 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

rsivaram 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 beac86f049 KAFKA-13043: Implement Admin APIs for offsetFetch batching 
(#10964)
beac86f049 is described below

commit beac86f049385932309158c1cb49c8657e53f45f
Author: Sanjana Kaundinya 
AuthorDate: Thu Jul 14 05:47:34 2022 -0700

KAFKA-13043: Implement Admin APIs for offsetFetch batching (#10964)

This implements the AdminAPI portion of KIP-709: 
https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=173084258. The 
request/response protocol changes were implemented in 3.0.0. A new batched API 
has been introduced to list consumer offsets for different groups. For brokers 
older than 3.0.0, separate requests are sent for each group.

Co-authored-by: Rajini Sivaram 
Co-authored-by: David Jacot 

Reviewers: David Jacot ,  Rajini Sivaram 

---
 .../java/org/apache/kafka/clients/admin/Admin.java |  36 ++-
 .../kafka/clients/admin/KafkaAdminClient.java  |  11 +-
 .../admin/ListConsumerGroupOffsetsOptions.java |  14 +-
 .../admin/ListConsumerGroupOffsetsResult.java  |  56 +++-
 .../admin/ListConsumerGroupOffsetsSpec.java|  79 ++
 .../clients/admin/internals/AdminApiDriver.java|   3 +-
 .../admin/internals/CoordinatorStrategy.java   |   4 +
 .../internals/ListConsumerGroupOffsetsHandler.java | 128 +
 .../kafka/common/requests/OffsetFetchResponse.java |  10 +-
 .../kafka/clients/admin/AdminClientTestUtils.java  |  12 +-
 .../kafka/clients/admin/KafkaAdminClientTest.java  | 206 --
 .../kafka/clients/admin/MockAdminClient.java   |  16 +-
 .../ListConsumerGroupOffsetsHandlerTest.java   | 308 +++--
 .../kafka/clients/consumer/KafkaConsumerTest.java  |   6 +-
 .../internals/ConsumerCoordinatorTest.java |  26 +-
 .../scala/kafka/admin/ConsumerGroupCommand.scala   |   8 +-
 .../kafka/admin/ConsumerGroupServiceTest.scala |  22 +-
 .../scala/unit/kafka/server/RequestQuotaTest.scala |   2 +-
 .../processor/internals/StoreChangelogReader.java  |  12 +-
 .../internals/StoreChangelogReaderTest.java|  11 +-
 20 files changed, 813 insertions(+), 157 deletions(-)

diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java 
b/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
index fdacc09db8..0698d29702 100644
--- a/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
+++ b/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
@@ -36,6 +36,7 @@ import org.apache.kafka.common.requests.LeaveGroupResponse;
 
 import java.time.Duration;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.Optional;
@@ -919,12 +920,20 @@ public interface Admin extends AutoCloseable {
  * @param options The options to use when listing the consumer group 
offsets.
  * @return The ListGroupOffsetsResult
  */
-ListConsumerGroupOffsetsResult listConsumerGroupOffsets(String groupId, 
ListConsumerGroupOffsetsOptions options);
+default ListConsumerGroupOffsetsResult listConsumerGroupOffsets(String 
groupId, ListConsumerGroupOffsetsOptions options) {
+ListConsumerGroupOffsetsOptions listOptions = new 
ListConsumerGroupOffsetsOptions()
+.requireStable(options.requireStable());
+@SuppressWarnings("deprecation")
+ListConsumerGroupOffsetsSpec groupSpec = new 
ListConsumerGroupOffsetsSpec()
+.topicPartitions(options.topicPartitions());
+return listConsumerGroupOffsets(Collections.singletonMap(groupId, 
groupSpec), listOptions);
+}
 
 /**
  * List the consumer group offsets available in the cluster with the 
default options.
  * 
- * This is a convenience method for {@link 
#listConsumerGroupOffsets(String, ListConsumerGroupOffsetsOptions)} with 
default options.
+ * This is a convenience method for {@link #listConsumerGroupOffsets(Map, 
ListConsumerGroupOffsetsOptions)}
+ * to list offsets of all partitions of one group with default options.
  *
  * @return The ListGroupOffsetsResult.
  */
@@ -932,6 +941,29 @@ public interface Admin extends AutoCloseable {
 return listConsumerGroupOffsets(groupId, new 
ListConsumerGroupOffsetsOptions());
 }
 
+/**
+ * List the consumer group offsets available in the cluster for the 
specified consumer groups.
+ *
+ * @param groupSpecs Map of consumer group ids to a spec that specifies 
the topic partitions of the group to list offsets for.
+ *
+ * @param options The options to use when listing the consumer group 
offsets.
+ * @return The ListConsumerGroupOffsetsResult
+ */
+ListConsumerGroupOffsetsResult listConsumerGroupOffsets(Map 

[kafka] branch trunk updated: KAFKA-13879: Reconnect exponential backoff is ineffective in some cases (#12131)

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

rsivaram 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 eeb1e702eb KAFKA-13879: Reconnect exponential backoff is ineffective 
in some cases (#12131)
eeb1e702eb is described below

commit eeb1e702eb7a43d88f11458f739672e2b7aa4871
Author: chern 
AuthorDate: Tue May 10 03:36:42 2022 -0700

KAFKA-13879: Reconnect exponential backoff is ineffective in some cases 
(#12131)

When a client connects to a SSL listener using PLAINTEXT security protocol, 
after the TCP connection is setup, the client considers the channel setup is 
complete. In reality the channel setup is not complete yet. The client then 
resets reconnect exponential backoff and issues API version request. Since the 
broker expects SSL handshake, the API version request will cause the connection 
to disconnect. Client reconnects without exponential backoff since it has been 
reset.

This commit removes the reset of reconnect exponential backoff when sending 
API version request. In the good case where the channel setup is complete, 
reconnect exponential backoff will be reset when the node becomes ready, which 
is after getting the API version response. Inter-broker clients which do not 
send API version request and go directly to ready state continue to reset 
backoff before any  successful requests.

Reviewers: Rajini Sivaram 
---
 .../kafka/clients/ClusterConnectionStates.java |  1 -
 .../kafka/clients/ClusterConnectionStatesTest.java | 38 ++
 .../apache/kafka/clients/NetworkClientTest.java| 31 ++
 3 files changed, 48 insertions(+), 22 deletions(-)

diff --git 
a/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java 
b/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java
index 95efdbeae4..f4d9092258 100644
--- 
a/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java
+++ 
b/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java
@@ -246,7 +246,6 @@ final class ClusterConnectionStates {
 public void checkingApiVersions(String id) {
 NodeConnectionState nodeState = nodeState(id);
 nodeState.state = ConnectionState.CHECKING_API_VERSIONS;
-resetReconnectBackoff(nodeState);
 resetConnectionSetupTimeout(nodeState);
 connectingNodes.remove(id);
 }
diff --git 
a/clients/src/test/java/org/apache/kafka/clients/ClusterConnectionStatesTest.java
 
b/clients/src/test/java/org/apache/kafka/clients/ClusterConnectionStatesTest.java
index 72cc123921..96fe89ca11 100644
--- 
a/clients/src/test/java/org/apache/kafka/clients/ClusterConnectionStatesTest.java
+++ 
b/clients/src/test/java/org/apache/kafka/clients/ClusterConnectionStatesTest.java
@@ -231,20 +231,8 @@ public class ClusterConnectionStatesTest {
 
 @Test
 public void testExponentialReconnectBackoff() {
-double reconnectBackoffMaxExp = Math.log(reconnectBackoffMax / 
(double) Math.max(reconnectBackoffMs, 1))
-/ Math.log(reconnectBackoffExpBase);
-
-// Run through 10 disconnects and check that reconnect backoff value 
is within expected range for every attempt
-for (int i = 0; i < 10; i++) {
-connectionStates.connecting(nodeId1, time.milliseconds(), 
"localhost");
-connectionStates.disconnected(nodeId1, time.milliseconds());
-// Calculate expected backoff value without jitter
-long expectedBackoff = 
Math.round(Math.pow(reconnectBackoffExpBase, Math.min(i, 
reconnectBackoffMaxExp))
-* reconnectBackoffMs);
-long currentBackoff = connectionStates.connectionDelay(nodeId1, 
time.milliseconds());
-assertEquals(expectedBackoff, currentBackoff, 
reconnectBackoffJitter * expectedBackoff);
-time.sleep(connectionStates.connectionDelay(nodeId1, 
time.milliseconds()) + 1);
-}
+verifyReconnectExponentialBackoff(false);
+verifyReconnectExponentialBackoff(true);
 }
 
 @Test
@@ -426,4 +414,26 @@ public class ClusterConnectionStatesTest {
 this.connectionStates = new 
ClusterConnectionStates(reconnectBackoffMs, reconnectBackoffMax,
 connectionSetupTimeoutMs, connectionSetupTimeoutMaxMs, new 
LogContext(), this.multipleIPHostResolver);
 }
+
+private void verifyReconnectExponentialBackoff(boolean 
enterCheckingApiVersionState) {
+double reconnectBackoffMaxExp = Math.log(reconnectBackoffMax / 
(double) Math.max(reconnectBackoffMs, 1))
+/ Math.log(reconnectBackoffExpBase);
+
+connectionStates.remove(nodeId1);
+// Run through 10 disconnects and check that reconnect backoff value 
is within expected range for every attempt
+for (int i =

[kafka] branch trunk updated (8e205b5 -> bb60eb8)

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

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


from 8e205b5  KAFKA-13719: Fix connector restart cause duplicate tasks 
(#11869)
 add bb60eb8  MINOR: Increase wait in ZooKeeperClientTest (#11973)

No new revisions were added by this update.

Summary of changes:
 core/src/test/scala/unit/kafka/zookeeper/ZooKeeperClientTest.scala | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)


[kafka] branch trunk updated: MINOR: greatly improve test runtime by unblocking purgatory and quota manager threads (#11653)

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

rsivaram 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 0b9a8ba  MINOR: greatly improve test runtime by unblocking purgatory 
and quota manager threads (#11653)
0b9a8ba is described below

commit 0b9a8bac36f16b5397e9ec3a0441758e4b60a384
Author: Lucas Bradstreet 
AuthorDate: Thu Jan 6 01:22:19 2022 -0800

MINOR: greatly improve test runtime by unblocking purgatory and quota 
manager threads (#11653)

Reviewers: Rajini Sivaram 
---
 core/src/main/scala/kafka/server/ClientQuotaManager.scala | 12 +++-
 core/src/main/scala/kafka/server/DelayedOperation.scala   | 11 +--
 2 files changed, 20 insertions(+), 3 deletions(-)

diff --git a/core/src/main/scala/kafka/server/ClientQuotaManager.scala 
b/core/src/main/scala/kafka/server/ClientQuotaManager.scala
index 5860cfc..7334519 100644
--- a/core/src/main/scala/kafka/server/ClientQuotaManager.scala
+++ b/core/src/main/scala/kafka/server/ClientQuotaManager.scala
@@ -562,8 +562,18 @@ class ClientQuotaManager(private val config: 
ClientQuotaManagerConfig,
   quotaMetricTags.asJava)
   }
 
+  def initiateShutdown(): Unit = {
+throttledChannelReaper.initiateShutdown()
+// improve shutdown time by waking up any ShutdownableThread(s) blocked on 
poll by sending a no-op
+delayQueue.add(new ThrottledChannel(time, 0, new ThrottleCallback {
+  override def startThrottling(): Unit = {}
+  override def endThrottling(): Unit = {}
+}))
+  }
+
   def shutdown(): Unit = {
-throttledChannelReaper.shutdown()
+initiateShutdown()
+throttledChannelReaper.awaitShutdown()
   }
 
   class DefaultQuotaCallback extends ClientQuotaCallback {
diff --git a/core/src/main/scala/kafka/server/DelayedOperation.scala 
b/core/src/main/scala/kafka/server/DelayedOperation.scala
index 251dd28..1151e65 100644
--- a/core/src/main/scala/kafka/server/DelayedOperation.scala
+++ b/core/src/main/scala/kafka/server/DelayedOperation.scala
@@ -328,8 +328,15 @@ final class DelayedOperationPurgatory[T <: 
DelayedOperation](purgatoryName: Stri
* Shutdown the expire reaper thread
*/
   def shutdown(): Unit = {
-if (reaperEnabled)
-  expirationReaper.shutdown()
+if (reaperEnabled) {
+  expirationReaper.initiateShutdown()
+  // improve shutdown time by waking up any ShutdownableThread(s) blocked 
on poll by sending a no-op
+  timeoutTimer.add(new TimerTask {
+override val delayMs: Long = 0
+override def run(): Unit = {}
+  })
+  expirationReaper.awaitShutdown()
+}
 timeoutTimer.shutdown()
 removeMetric("PurgatorySize", metricsTags)
 removeMetric("NumDelayedOperations", metricsTags)


[kafka] branch trunk updated (b4602e8 -> 0e150a4)

2021-12-15 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

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


from b4602e8  MINOR: Timeout waitForBlock in connect BlockingConnectorTest 
(#11595)
 add 0e150a4  MINOR: Reset java.security.auth.login.config in ZK-tests to 
avoid config reload affecting subsequent tests (#11602)

No new revisions were added by this update.

Summary of changes:
 core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala | 1 +
 1 file changed, 1 insertion(+)


[kafka] branch trunk updated (acd1f9c -> 065fba9)

2021-12-14 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

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


from acd1f9c  KAFKA-13522: add position tracking and bounding to IQv2 
(#11581)
 add 065fba9  KAFKA-13539: Improve propagation and processing of SSL 
handshake failures (#11597)

No new revisions were added by this update.

Summary of changes:
 .../org/apache/kafka/common/network/Selector.java  |  2 +-
 .../kafka/common/network/SslTransportLayer.java| 59 ++
 .../kafka/common/network/NetworkTestUtils.java |  8 ++-
 .../common/network/SslTransportLayerTest.java  | 18 ++-
 4 files changed, 74 insertions(+), 13 deletions(-)


[kafka] branch 3.0 updated: KAFKA-13461: Don't re-initialize ZK client session after auth failure if connection still alive (#11563)

2021-12-02 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/3.0 by this push:
 new ac923b0  KAFKA-13461: Don't re-initialize ZK client session after auth 
failure if connection still alive (#11563)
ac923b0 is described below

commit ac923b0611817c9400e98cac5e0e19c18833de56
Author: Rajini Sivaram 
AuthorDate: Thu Dec 2 22:10:37 2021 +

KAFKA-13461: Don't re-initialize ZK client session after auth failure if 
connection still alive (#11563)

If JAAS configuration does not contain a Client section for ZK clients, an 
auth failure event is generated. If this occurs after the connection is setup 
in the controller, we schedule reinitialize(), which causes controller to 
resign. In the case where SASL is not mandatory and the connection is alive, 
controller maintains the current session and doesn't register its watchers, 
leaving it in a bad state.

Reviewers: Jun Rao 
---
 core/src/main/scala/kafka/zookeeper/ZooKeeperClient.scala |  4 ++--
 .../test/scala/unit/kafka/zookeeper/ZooKeeperClientTest.scala | 11 ++-
 2 files changed, 12 insertions(+), 3 deletions(-)

diff --git a/core/src/main/scala/kafka/zookeeper/ZooKeeperClient.scala 
b/core/src/main/scala/kafka/zookeeper/ZooKeeperClient.scala
index 091b401..bc634a8 100755
--- a/core/src/main/scala/kafka/zookeeper/ZooKeeperClient.scala
+++ b/core/src/main/scala/kafka/zookeeper/ZooKeeperClient.scala
@@ -433,14 +433,14 @@ class ZooKeeperClient(connectString: String,
 isConnectedOrExpiredCondition.signalAll()
   }
   if (state == KeeperState.AuthFailed) {
-error("Auth failed.")
+error(s"Auth failed, initialized=$isFirstConnectionEstablished 
connectionState=$connectionState")
 stateChangeHandlers.values.foreach(_.onAuthFailure())
 
 // If this is during initial startup, we fail fast. Otherwise, 
schedule retry.
 val initialized = inLock(isConnectedOrExpiredLock) {
   isFirstConnectionEstablished
 }
-if (initialized)
+if (initialized && !connectionState.isAlive)
   scheduleReinitialize("auth-failed", "Reinitializing due to auth 
failure.", RetryBackoffMs)
   } else if (state == KeeperState.Expired) {
 scheduleReinitialize("session-expired", "Session expired.", 
delayMs = 0L)
diff --git a/core/src/test/scala/unit/kafka/zookeeper/ZooKeeperClientTest.scala 
b/core/src/test/scala/unit/kafka/zookeeper/ZooKeeperClientTest.scala
index a0eb1ea..37954e6 100644
--- a/core/src/test/scala/unit/kafka/zookeeper/ZooKeeperClientTest.scala
+++ b/core/src/test/scala/unit/kafka/zookeeper/ZooKeeperClientTest.scala
@@ -648,9 +648,18 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness {
 }
 
 zooKeeperClient.close()
-zooKeeperClient = newZooKeeperClient()
+@volatile var connectionStateOverride: Option[States] = None
+zooKeeperClient = new ZooKeeperClient(zkConnect, zkSessionTimeout, 
zkConnectionTimeout,
+  zkMaxInFlightRequests, time, "testMetricGroup", "testMetricType", new 
ZKClientConfig, "ZooKeeperClientTest") {
+  override def connectionState: States = 
connectionStateOverride.getOrElse(super.connectionState)
+}
 zooKeeperClient.registerStateChangeHandler(changeHandler)
 
+connectionStateOverride = Some(States.CONNECTED)
+zooKeeperClient.ZooKeeperClientWatcher.process(new 
WatchedEvent(EventType.None, KeeperState.AuthFailed, null))
+assertFalse(sessionInitializedCountDownLatch.await(10, 
TimeUnit.MILLISECONDS), "Unexpected session initialization when connection is 
alive")
+
+connectionStateOverride = Some(States.AUTH_FAILED)
 zooKeeperClient.ZooKeeperClientWatcher.process(new 
WatchedEvent(EventType.None, KeeperState.AuthFailed, null))
 assertTrue(sessionInitializedCountDownLatch.await(5, TimeUnit.SECONDS), 
"Failed to receive session initializing notification")
   }


[kafka] branch 3.1 updated: KAFKA-13461: Don't re-initialize ZK client session after auth failure if connection still alive (#11563)

2021-12-02 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/3.1 by this push:
 new de73bea  KAFKA-13461: Don't re-initialize ZK client session after auth 
failure if connection still alive (#11563)
de73bea is described below

commit de73bea58ee6afb466eef4b375f8533aca97494f
Author: Rajini Sivaram 
AuthorDate: Thu Dec 2 22:10:37 2021 +

KAFKA-13461: Don't re-initialize ZK client session after auth failure if 
connection still alive (#11563)

If JAAS configuration does not contain a Client section for ZK clients, an 
auth failure event is generated. If this occurs after the connection is setup 
in the controller, we schedule reinitialize(), which causes controller to 
resign. In the case where SASL is not mandatory and the connection is alive, 
controller maintains the current session and doesn't register its watchers, 
leaving it in a bad state.

Reviewers: Jun Rao 
---
 core/src/main/scala/kafka/zookeeper/ZooKeeperClient.scala |  4 ++--
 .../test/scala/unit/kafka/zookeeper/ZooKeeperClientTest.scala | 11 ++-
 2 files changed, 12 insertions(+), 3 deletions(-)

diff --git a/core/src/main/scala/kafka/zookeeper/ZooKeeperClient.scala 
b/core/src/main/scala/kafka/zookeeper/ZooKeeperClient.scala
index 091b401..bc634a8 100755
--- a/core/src/main/scala/kafka/zookeeper/ZooKeeperClient.scala
+++ b/core/src/main/scala/kafka/zookeeper/ZooKeeperClient.scala
@@ -433,14 +433,14 @@ class ZooKeeperClient(connectString: String,
 isConnectedOrExpiredCondition.signalAll()
   }
   if (state == KeeperState.AuthFailed) {
-error("Auth failed.")
+error(s"Auth failed, initialized=$isFirstConnectionEstablished 
connectionState=$connectionState")
 stateChangeHandlers.values.foreach(_.onAuthFailure())
 
 // If this is during initial startup, we fail fast. Otherwise, 
schedule retry.
 val initialized = inLock(isConnectedOrExpiredLock) {
   isFirstConnectionEstablished
 }
-if (initialized)
+if (initialized && !connectionState.isAlive)
   scheduleReinitialize("auth-failed", "Reinitializing due to auth 
failure.", RetryBackoffMs)
   } else if (state == KeeperState.Expired) {
 scheduleReinitialize("session-expired", "Session expired.", 
delayMs = 0L)
diff --git a/core/src/test/scala/unit/kafka/zookeeper/ZooKeeperClientTest.scala 
b/core/src/test/scala/unit/kafka/zookeeper/ZooKeeperClientTest.scala
index 649f3c5..5af2ba8 100644
--- a/core/src/test/scala/unit/kafka/zookeeper/ZooKeeperClientTest.scala
+++ b/core/src/test/scala/unit/kafka/zookeeper/ZooKeeperClientTest.scala
@@ -649,9 +649,18 @@ class ZooKeeperClientTest extends QuorumTestHarness {
 }
 
 zooKeeperClient.close()
-zooKeeperClient = newZooKeeperClient()
+@volatile var connectionStateOverride: Option[States] = None
+zooKeeperClient = new ZooKeeperClient(zkConnect, zkSessionTimeout, 
zkConnectionTimeout,
+  zkMaxInFlightRequests, time, "testMetricGroup", "testMetricType", new 
ZKClientConfig, "ZooKeeperClientTest") {
+  override def connectionState: States = 
connectionStateOverride.getOrElse(super.connectionState)
+}
 zooKeeperClient.registerStateChangeHandler(changeHandler)
 
+connectionStateOverride = Some(States.CONNECTED)
+zooKeeperClient.ZooKeeperClientWatcher.process(new 
WatchedEvent(EventType.None, KeeperState.AuthFailed, null))
+assertFalse(sessionInitializedCountDownLatch.await(10, 
TimeUnit.MILLISECONDS), "Unexpected session initialization when connection is 
alive")
+
+connectionStateOverride = Some(States.AUTH_FAILED)
 zooKeeperClient.ZooKeeperClientWatcher.process(new 
WatchedEvent(EventType.None, KeeperState.AuthFailed, null))
 assertTrue(sessionInitializedCountDownLatch.await(5, TimeUnit.SECONDS), 
"Failed to receive session initializing notification")
   }


[kafka] branch trunk updated (62f73c3 -> da56146)

2021-12-02 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

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


from 62f73c3  KAFKA-13498: track position in remaining state stores (#11541)
 add da56146  KAFKA-13461: Don't re-initialize ZK client session after auth 
failure if connection still alive (#11563)

No new revisions were added by this update.

Summary of changes:
 core/src/main/scala/kafka/zookeeper/ZooKeeperClient.scala |  4 ++--
 .../test/scala/unit/kafka/zookeeper/ZooKeeperClientTest.scala | 11 ++-
 2 files changed, 12 insertions(+), 3 deletions(-)


[kafka] branch 2.7 updated: KAFKA-13277; Fix size calculation for tagged string fields in message generator (#11308)

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

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


The following commit(s) were added to refs/heads/2.7 by this push:
 new d4e2273  KAFKA-13277; Fix size calculation for tagged string fields in 
message generator (#11308)
d4e2273 is described below

commit d4e227379c9096499788719fdbf2d1a5105d8ee9
Author: Rajini Sivaram 
AuthorDate: Tue Sep 7 21:02:45 2021 +0100

KAFKA-13277; Fix size calculation for tagged string fields in message 
generator (#11308)

Reviewers: Colin P. McCabe 
---
 .../org/apache/kafka/common/message/MessageTest.java | 16 
 .../org/apache/kafka/message/MessageDataGenerator.java   |  2 +-
 2 files changed, 17 insertions(+), 1 deletion(-)

diff --git 
a/clients/src/test/java/org/apache/kafka/common/message/MessageTest.java 
b/clients/src/test/java/org/apache/kafka/common/message/MessageTest.java
index 94e6bc2..9cf18cf 100644
--- a/clients/src/test/java/org/apache/kafka/common/message/MessageTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/message/MessageTest.java
@@ -1068,6 +1068,22 @@ public final class MessageTest {
 verifyWriteSucceeds((short) 6, createTopics);
 }
 
+@Test
+public void testLongTaggedString() throws Exception {
+char[] chars = new char[1024];
+Arrays.fill(chars, 'a');
+String longString = new String(chars);
+SimpleExampleMessageData message = new SimpleExampleMessageData()
+.setMyString(longString);
+ObjectSerializationCache cache = new ObjectSerializationCache();
+short version = 1;
+int size = message.size(cache, version);
+ByteBuffer buf = ByteBuffer.allocate(size);
+ByteBufferAccessor byteBufferAccessor = new ByteBufferAccessor(buf);
+message.write(byteBufferAccessor, cache, version);
+assertEquals(size, buf.position());
+}
+
 private void verifyWriteRaisesNpe(short version, Message message) {
 ObjectSerializationCache cache = new ObjectSerializationCache();
 assertThrows(NullPointerException.class, () -> {
diff --git 
a/generator/src/main/java/org/apache/kafka/message/MessageDataGenerator.java 
b/generator/src/main/java/org/apache/kafka/message/MessageDataGenerator.java
index 9dcaa76..54f2364 100644
--- a/generator/src/main/java/org/apache/kafka/message/MessageDataGenerator.java
+++ b/generator/src/main/java/org/apache/kafka/message/MessageDataGenerator.java
@@ -1530,7 +1530,7 @@ public final class MessageDataGenerator implements 
MessageClassGenerator {
 buffer.printf("int _stringPrefixSize = " +
 
"ByteUtils.sizeOfUnsignedVarint(_stringBytes.length + 1);%n");
 buffer.printf("_size += _stringBytes.length + 
_stringPrefixSize + " +
-
"ByteUtils.sizeOfUnsignedVarint(_stringPrefixSize);%n");
+
"ByteUtils.sizeOfUnsignedVarint(_stringPrefixSize + _stringBytes.length);%n");
 } else {
 buffer.printf("_size += _stringBytes.length + 
" +
 
"ByteUtils.sizeOfUnsignedVarint(_stringBytes.length + 1);%n");


[kafka] branch 2.8 updated: KAFKA-13277; Fix size calculation for tagged string fields in message generator (#11308)

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

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


The following commit(s) were added to refs/heads/2.8 by this push:
 new df0d57a  KAFKA-13277; Fix size calculation for tagged string fields in 
message generator (#11308)
df0d57a is described below

commit df0d57a63ec2de3311fd07e08b255f42eccdc94b
Author: Rajini Sivaram 
AuthorDate: Tue Sep 7 21:02:45 2021 +0100

KAFKA-13277; Fix size calculation for tagged string fields in message 
generator (#11308)

Reviewers: Colin P. McCabe 
---
 .../org/apache/kafka/common/message/MessageTest.java | 16 
 .../org/apache/kafka/message/MessageDataGenerator.java   |  3 ++-
 2 files changed, 18 insertions(+), 1 deletion(-)

diff --git 
a/clients/src/test/java/org/apache/kafka/common/message/MessageTest.java 
b/clients/src/test/java/org/apache/kafka/common/message/MessageTest.java
index 5dc379e..0fcab84 100644
--- a/clients/src/test/java/org/apache/kafka/common/message/MessageTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/message/MessageTest.java
@@ -941,6 +941,22 @@ public final class MessageTest {
 verifyWriteSucceeds((short) 6, createTopics);
 }
 
+@Test
+public void testLongTaggedString() throws Exception {
+char[] chars = new char[1024];
+Arrays.fill(chars, 'a');
+String longString = new String(chars);
+SimpleExampleMessageData message = new SimpleExampleMessageData()
+.setMyString(longString);
+ObjectSerializationCache cache = new ObjectSerializationCache();
+short version = 1;
+int size = message.size(cache, version);
+ByteBuffer buf = ByteBuffer.allocate(size);
+ByteBufferAccessor byteBufferAccessor = new ByteBufferAccessor(buf);
+message.write(byteBufferAccessor, cache, version);
+assertEquals(size, buf.position());
+}
+
 private void verifyWriteRaisesNpe(short version, Message message) {
 ObjectSerializationCache cache = new ObjectSerializationCache();
 assertThrows(NullPointerException.class, () -> {
diff --git 
a/generator/src/main/java/org/apache/kafka/message/MessageDataGenerator.java 
b/generator/src/main/java/org/apache/kafka/message/MessageDataGenerator.java
index 81b0c1e..5e591c5 100644
--- a/generator/src/main/java/org/apache/kafka/message/MessageDataGenerator.java
+++ b/generator/src/main/java/org/apache/kafka/message/MessageDataGenerator.java
@@ -1157,7 +1157,8 @@ public final class MessageDataGenerator implements 
MessageClassGenerator {
 buffer.printf("int _stringPrefixSize = " +
 
"ByteUtils.sizeOfUnsignedVarint(_stringBytes.length + 1);%n");
 
buffer.printf("_size.addBytes(_stringBytes.length + _stringPrefixSize + " +
-
"ByteUtils.sizeOfUnsignedVarint(_stringPrefixSize));%n");
+
"ByteUtils.sizeOfUnsignedVarint(_stringPrefixSize + _stringBytes.length));%n");
+
 } else {
 
buffer.printf("_size.addBytes(_stringBytes.length + " +
 
"ByteUtils.sizeOfUnsignedVarint(_stringBytes.length + 1));%n");


[kafka] branch 3.0 updated: KAFKA-13277; Fix size calculation for tagged string fields in message generator (#11308)

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

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


The following commit(s) were added to refs/heads/3.0 by this push:
 new b149f91  KAFKA-13277; Fix size calculation for tagged string fields in 
message generator (#11308)
b149f91 is described below

commit b149f9141853eda53ad68730e74caed626711284
Author: Rajini Sivaram 
AuthorDate: Tue Sep 7 21:02:45 2021 +0100

KAFKA-13277; Fix size calculation for tagged string fields in message 
generator (#11308)

Reviewers: Colin P. McCabe 
---
 .../org/apache/kafka/common/message/MessageTest.java | 16 
 .../org/apache/kafka/message/MessageDataGenerator.java   |  3 ++-
 2 files changed, 18 insertions(+), 1 deletion(-)

diff --git 
a/clients/src/test/java/org/apache/kafka/common/message/MessageTest.java 
b/clients/src/test/java/org/apache/kafka/common/message/MessageTest.java
index aa00c24..3fcd007 100644
--- a/clients/src/test/java/org/apache/kafka/common/message/MessageTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/message/MessageTest.java
@@ -1161,6 +1161,22 @@ public final class MessageTest {
 verifyWriteSucceeds((short) 6, createTopics);
 }
 
+@Test
+public void testLongTaggedString() throws Exception {
+char[] chars = new char[1024];
+Arrays.fill(chars, 'a');
+String longString = new String(chars);
+SimpleExampleMessageData message = new SimpleExampleMessageData()
+.setMyString(longString);
+ObjectSerializationCache cache = new ObjectSerializationCache();
+short version = 1;
+int size = message.size(cache, version);
+ByteBuffer buf = ByteBuffer.allocate(size);
+ByteBufferAccessor byteBufferAccessor = new ByteBufferAccessor(buf);
+message.write(byteBufferAccessor, cache, version);
+assertEquals(size, buf.position());
+}
+
 private void verifyWriteRaisesNpe(short version, Message message) {
 ObjectSerializationCache cache = new ObjectSerializationCache();
 assertThrows(NullPointerException.class, () -> {
diff --git 
a/generator/src/main/java/org/apache/kafka/message/MessageDataGenerator.java 
b/generator/src/main/java/org/apache/kafka/message/MessageDataGenerator.java
index b3c243e..b9923ee 100644
--- a/generator/src/main/java/org/apache/kafka/message/MessageDataGenerator.java
+++ b/generator/src/main/java/org/apache/kafka/message/MessageDataGenerator.java
@@ -1156,7 +1156,8 @@ public final class MessageDataGenerator implements 
MessageClassGenerator {
 buffer.printf("int _stringPrefixSize = " +
 
"ByteUtils.sizeOfUnsignedVarint(_stringBytes.length + 1);%n");
 
buffer.printf("_size.addBytes(_stringBytes.length + _stringPrefixSize + " +
-
"ByteUtils.sizeOfUnsignedVarint(_stringPrefixSize));%n");
+
"ByteUtils.sizeOfUnsignedVarint(_stringPrefixSize + _stringBytes.length));%n");
+
 } else {
 
buffer.printf("_size.addBytes(_stringBytes.length + " +
 
"ByteUtils.sizeOfUnsignedVarint(_stringBytes.length + 1));%n");


[kafka] branch trunk updated: KAFKA-10774; Admin API for Describe topic using topic IDs (#9769)

2021-08-28 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

rsivaram 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 1d22b0d  KAFKA-10774; Admin API for Describe topic using topic IDs 
(#9769)
1d22b0d is described below

commit 1d22b0d70686aef5689b775ea2ea7610a37f3e8c
Author: dengziming 
AuthorDate: Sat Aug 28 16:00:36 2021 +0800

KAFKA-10774; Admin API for Describe topic using topic IDs (#9769)

Reviewers: Justine Olshan , Chia-Ping Tsai 
, Satish Duggana , Rajini Sivaram 

---
 .../java/org/apache/kafka/clients/admin/Admin.java |  44 +--
 .../kafka/clients/admin/DeleteTopicsResult.java|   2 +-
 .../kafka/clients/admin/DescribeTopicsResult.java  |  97 +--
 .../kafka/clients/admin/KafkaAdminClient.java  | 127 
 .../kafka/clients/admin/TopicDescription.java  |  12 +-
 .../apache/kafka/clients/admin/TopicListing.java   |  28 -
 .../main/java/org/apache/kafka/common/Cluster.java |   8 ++
 .../kafka/common/requests/MetadataRequest.java |  40 ++-
 .../kafka/common/requests/MetadataResponse.java|  19 +++
 .../resources/common/message/MetadataRequest.json  |   3 +-
 .../resources/common/message/MetadataResponse.json |   5 +-
 .../kafka/clients/admin/AdminClientTestUtils.java  |   7 +-
 .../kafka/clients/admin/KafkaAdminClientTest.java  |   4 +-
 .../kafka/clients/admin/MockAdminClient.java   | 100 +---
 .../ClientAuthenticationFailureTest.java   |   2 +-
 .../connect/mirror/MirrorSourceConnector.java  |   2 +-
 .../org/apache/kafka/connect/util/TopicAdmin.java  |   2 +-
 .../apache/kafka/connect/util/TopicAdminTest.java  |   2 +-
 .../util/clusters/EmbeddedKafkaCluster.java|   2 +-
 .../scala/kafka/admin/ConsumerGroupCommand.scala   |   4 +-
 .../kafka/admin/ReassignPartitionsCommand.scala|   4 +-
 core/src/main/scala/kafka/admin/TopicCommand.scala | 133 +++--
 core/src/main/scala/kafka/server/KafkaApis.scala   |  36 +-
 .../main/scala/kafka/server/MetadataCache.scala|   4 +
 .../kafka/server/metadata/KRaftMetadataCache.scala |   4 +
 .../kafka/server/metadata/ZkMetadataCache.scala|   8 ++
 .../kafka/tools/ReplicaVerificationTool.scala  |   2 +-
 .../kafka/api/BaseAdminIntegrationTest.scala   |   4 +-
 .../api/DescribeAuthorizedOperationsTest.scala |   6 +-
 .../kafka/api/EndToEndAuthorizationTest.scala  |   8 +-
 .../kafka/api/PlaintextAdminIntegrationTest.scala  |  24 +++-
 .../SaslClientsWithInvalidCredentialsTest.scala|   2 +-
 .../server/DynamicBrokerReconfigurationTest.scala  |   2 +-
 .../kafka/server/KRaftClusterTest.scala|   2 +-
 .../MetadataRequestBetweenDifferentIbpTest.scala   |  96 +++
 .../scala/unit/kafka/admin/DeleteTopicTest.scala   |   2 +-
 .../kafka/admin/TopicCommandIntegrationTest.scala  |  18 +--
 .../scala/unit/kafka/server/KafkaApisTest.scala| 111 +
 .../test/scala/unit/kafka/utils/TestUtils.scala|   8 +-
 .../TopicBasedRemoteLogMetadataManager.java|   2 +-
 .../processor/internals/InternalTopicManager.java  |   4 +-
 .../KStreamRepartitionIntegrationTest.java |   2 +-
 ...bleJoinTopologyOptimizationIntegrationTest.java |   2 +-
 .../internals/InternalTopicManagerTest.java|   8 +-
 .../kafka/tools/ClientCompatibilityTest.java   |   2 +-
 .../apache/kafka/tools/TransactionsCommand.java|   2 +-
 .../kafka/tools/TransactionsCommandTest.java   |   2 +-
 .../apache/kafka/trogdor/common/WorkerUtils.java   |   4 +-
 .../kafka/trogdor/common/WorkerUtilsTest.java  |   6 +-
 49 files changed, 846 insertions(+), 172 deletions(-)

diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java 
b/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
index 54d103b..4b6fe49 100644
--- a/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
+++ b/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
@@ -17,14 +17,6 @@
 
 package org.apache.kafka.clients.admin;
 
-import java.time.Duration;
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
-import java.util.Optional;
-import java.util.Properties;
-import java.util.Set;
-
 import org.apache.kafka.clients.consumer.OffsetAndMetadata;
 import org.apache.kafka.common.ElectionType;
 import org.apache.kafka.common.KafkaFuture;
@@ -42,6 +34,14 @@ import org.apache.kafka.common.quota.ClientQuotaAlteration;
 import org.apache.kafka.common.quota.ClientQuotaFilter;
 import org.apache.kafka.common.requests.LeaveGroupResponse;
 
+import java.time.Duration;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+
 /**
  * The administrative client for Kafka, which supports managing and inspecting 
topics, brokers

[kafka] branch 3.0 updated: KAFKA-13207: Skip truncation on fetch response with diverging epoch if partition removed from fetcher (#11221)

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

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


The following commit(s) were added to refs/heads/3.0 by this push:
 new 4d1efc9  KAFKA-13207: Skip truncation on fetch response with diverging 
epoch if partition removed from fetcher (#11221)
4d1efc9 is described below

commit 4d1efc90af02a090e6313f86577cfbb4714597a5
Author: Rajini Sivaram 
AuthorDate: Tue Aug 17 17:12:51 2021 +0100

KAFKA-13207: Skip truncation on fetch response with diverging epoch if 
partition removed from fetcher (#11221)

AbstractFetcherThread#truncateOnFetchResponse is used with IBP 2.7 and 
above to truncate partitions based on diverging epoch returned in fetch 
responses. Truncation should only be performed for partitions that are still 
owned by the fetcher and this check should be done while holding 
partitionMapLock to ensure that any partitions removed from the fetcher thread 
are not truncated. Truncation will be performed by any new fetcher that owns 
the partition when it restarts fetching.

Reviewers: David Jacot , Jason Gustafson 

---
 .../scala/kafka/server/AbstractFetcherThread.scala | 38 +++
 .../kafka/server/AbstractFetcherThreadTest.scala   | 43 ++
 2 files changed, 66 insertions(+), 15 deletions(-)

diff --git a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala 
b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala
index 6118c5f..7b315be 100755
--- a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala
+++ b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala
@@ -229,7 +229,8 @@ abstract class AbstractFetcherThread(name: String,
 }
   }
 
-  protected def truncateOnFetchResponse(epochEndOffsets: Map[TopicPartition, 
EpochEndOffset]): Unit = {
+  // Visibility for unit tests
+  protected[server] def truncateOnFetchResponse(epochEndOffsets: 
Map[TopicPartition, EpochEndOffset]): Unit = {
 inLock(partitionMapLock) {
   val ResultWithPartitions(fetchOffsets, partitionsWithError) = 
maybeTruncateToEpochEndOffsets(epochEndOffsets, Map.empty)
   handlePartitionsWithErrors(partitionsWithError, 
"truncateOnFetchResponse")
@@ -262,21 +263,28 @@ abstract class AbstractFetcherThread(name: String,
 val partitionsWithError = mutable.HashSet.empty[TopicPartition]
 
 fetchedEpochs.forKeyValue { (tp, leaderEpochOffset) =>
-  Errors.forCode(leaderEpochOffset.errorCode) match {
-case Errors.NONE =>
-  val offsetTruncationState = getOffsetTruncationState(tp, 
leaderEpochOffset)
-  if (doTruncate(tp, offsetTruncationState))
-fetchOffsets.put(tp, offsetTruncationState)
-
-case Errors.FENCED_LEADER_EPOCH =>
-  val currentLeaderEpoch = latestEpochsForPartitions.get(tp)
-.map(epochEndOffset => 
Int.box(epochEndOffset.currentLeaderEpoch)).asJava
-  if (onPartitionFenced(tp, currentLeaderEpoch))
+  if (partitionStates.contains(tp)) {
+Errors.forCode(leaderEpochOffset.errorCode) match {
+  case Errors.NONE =>
+val offsetTruncationState = getOffsetTruncationState(tp, 
leaderEpochOffset)
+if (doTruncate(tp, offsetTruncationState))
+  fetchOffsets.put(tp, offsetTruncationState)
+
+  case Errors.FENCED_LEADER_EPOCH =>
+val currentLeaderEpoch = latestEpochsForPartitions.get(tp)
+  .map(epochEndOffset => 
Int.box(epochEndOffset.currentLeaderEpoch)).asJava
+if (onPartitionFenced(tp, currentLeaderEpoch))
+  partitionsWithError += tp
+
+  case error =>
+info(s"Retrying leaderEpoch request for partition $tp as the 
leader reported an error: $error")
 partitionsWithError += tp
-
-case error =>
-  info(s"Retrying leaderEpoch request for partition $tp as the leader 
reported an error: $error")
-  partitionsWithError += tp
+}
+  } else {
+// Partitions may have been removed from the fetcher while the thread 
was waiting for fetch
+// response. Removed partitions are filtered out while holding 
`partitionMapLock` to ensure that we
+// don't update state for any partition that may have already been 
migrated to another thread.
+trace(s"Ignoring epoch offsets for partition $tp since it has been 
removed from this fetcher thread.")
   }
 }
 
diff --git 
a/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala 
b/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala
index 394897f..738c1ea 100644
--- a/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala
+++ b/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala
@@ -39,6 +39,7 @@ import 
org.apache.kafka.common.requests.OffsetsForLeaderEpochRespons

[kafka] branch 2.8 updated: KAFKA-13207: Skip truncation on fetch response with diverging epoch if partition removed from fetcher (#11221)

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

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


The following commit(s) were added to refs/heads/2.8 by this push:
 new 9618c61  KAFKA-13207: Skip truncation on fetch response with diverging 
epoch if partition removed from fetcher (#11221)
9618c61 is described below

commit 9618c61a2488fd8a1ecaf140c27f27e58a0efdf3
Author: Rajini Sivaram 
AuthorDate: Tue Aug 17 17:12:51 2021 +0100

KAFKA-13207: Skip truncation on fetch response with diverging epoch if 
partition removed from fetcher (#11221)

AbstractFetcherThread#truncateOnFetchResponse is used with IBP 2.7 and 
above to truncate partitions based on diverging epoch returned in fetch 
responses. Truncation should only be performed for partitions that are still 
owned by the fetcher and this check should be done while holding 
partitionMapLock to ensure that any partitions removed from the fetcher thread 
are not truncated. Truncation will be performed by any new fetcher that owns 
the partition when it restarts fetching.

Reviewers: David Jacot , Jason Gustafson 

---
 .../scala/kafka/server/AbstractFetcherThread.scala | 38 +++
 .../kafka/server/AbstractFetcherThreadTest.scala   | 43 ++
 2 files changed, 66 insertions(+), 15 deletions(-)

diff --git a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala 
b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala
index 5e716e2..492efa9 100755
--- a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala
+++ b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala
@@ -233,7 +233,8 @@ abstract class AbstractFetcherThread(name: String,
 }
   }
 
-  protected def truncateOnFetchResponse(epochEndOffsets: Map[TopicPartition, 
EpochEndOffset]): Unit = {
+  // Visibility for unit tests
+  protected[server] def truncateOnFetchResponse(epochEndOffsets: 
Map[TopicPartition, EpochEndOffset]): Unit = {
 inLock(partitionMapLock) {
   val ResultWithPartitions(fetchOffsets, partitionsWithError) = 
maybeTruncateToEpochEndOffsets(epochEndOffsets, Map.empty)
   handlePartitionsWithErrors(partitionsWithError, 
"truncateOnFetchResponse")
@@ -266,21 +267,28 @@ abstract class AbstractFetcherThread(name: String,
 val partitionsWithError = mutable.HashSet.empty[TopicPartition]
 
 fetchedEpochs.forKeyValue { (tp, leaderEpochOffset) =>
-  Errors.forCode(leaderEpochOffset.errorCode) match {
-case Errors.NONE =>
-  val offsetTruncationState = getOffsetTruncationState(tp, 
leaderEpochOffset)
-  if (doTruncate(tp, offsetTruncationState))
-fetchOffsets.put(tp, offsetTruncationState)
-
-case Errors.FENCED_LEADER_EPOCH =>
-  val currentLeaderEpoch = latestEpochsForPartitions.get(tp)
-.map(epochEndOffset => 
Int.box(epochEndOffset.currentLeaderEpoch)).asJava
-  if (onPartitionFenced(tp, currentLeaderEpoch))
+  if (partitionStates.contains(tp)) {
+Errors.forCode(leaderEpochOffset.errorCode) match {
+  case Errors.NONE =>
+val offsetTruncationState = getOffsetTruncationState(tp, 
leaderEpochOffset)
+if (doTruncate(tp, offsetTruncationState))
+  fetchOffsets.put(tp, offsetTruncationState)
+
+  case Errors.FENCED_LEADER_EPOCH =>
+val currentLeaderEpoch = latestEpochsForPartitions.get(tp)
+  .map(epochEndOffset => 
Int.box(epochEndOffset.currentLeaderEpoch)).asJava
+if (onPartitionFenced(tp, currentLeaderEpoch))
+  partitionsWithError += tp
+
+  case error =>
+info(s"Retrying leaderEpoch request for partition $tp as the 
leader reported an error: $error")
 partitionsWithError += tp
-
-case error =>
-  info(s"Retrying leaderEpoch request for partition $tp as the leader 
reported an error: $error")
-  partitionsWithError += tp
+}
+  } else {
+// Partitions may have been removed from the fetcher while the thread 
was waiting for fetch
+// response. Removed partitions are filtered out while holding 
`partitionMapLock` to ensure that we
+// don't update state for any partition that may have already been 
migrated to another thread.
+trace(s"Ignoring epoch offsets for partition $tp since it has been 
removed from this fetcher thread.")
   }
 }
 
diff --git 
a/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala 
b/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala
index 9b9fe8e..3b6fa41 100644
--- a/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala
+++ b/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala
@@ -40,6 +40,7 @@ import 
org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.{UNDE

[kafka] branch trunk updated: KAFKA-13207: Skip truncation on fetch response with diverging epoch if partition removed from fetcher (#11221)

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

rsivaram 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 bbf1ee7  KAFKA-13207: Skip truncation on fetch response with diverging 
epoch if partition removed from fetcher (#11221)
bbf1ee7 is described below

commit bbf1ee74d719494ed5a5cac9dc54b7093171707c
Author: Rajini Sivaram 
AuthorDate: Tue Aug 17 17:12:51 2021 +0100

KAFKA-13207: Skip truncation on fetch response with diverging epoch if 
partition removed from fetcher (#11221)

AbstractFetcherThread#truncateOnFetchResponse is used with IBP 2.7 and 
above to truncate partitions based on diverging epoch returned in fetch 
responses. Truncation should only be performed for partitions that are still 
owned by the fetcher and this check should be done while holding 
partitionMapLock to ensure that any partitions removed from the fetcher thread 
are not truncated. Truncation will be performed by any new fetcher that owns 
the partition when it restarts fetching.

Reviewers: David Jacot , Jason Gustafson 

---
 .../scala/kafka/server/AbstractFetcherThread.scala | 40 
 .../kafka/server/AbstractFetcherThreadTest.scala   | 43 ++
 2 files changed, 67 insertions(+), 16 deletions(-)

diff --git a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala 
b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala
index 1de1835..6de1588 100755
--- a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala
+++ b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala
@@ -229,7 +229,8 @@ abstract class AbstractFetcherThread(name: String,
 }
   }
 
-  protected def truncateOnFetchResponse(epochEndOffsets: Map[TopicPartition, 
EpochEndOffset]): Unit = {
+  // Visibility for unit tests
+  protected[server] def truncateOnFetchResponse(epochEndOffsets: 
Map[TopicPartition, EpochEndOffset]): Unit = {
 inLock(partitionMapLock) {
   val ResultWithPartitions(fetchOffsets, partitionsWithError) = 
maybeTruncateToEpochEndOffsets(epochEndOffsets, Map.empty)
   handlePartitionsWithErrors(partitionsWithError, 
"truncateOnFetchResponse")
@@ -262,22 +263,29 @@ abstract class AbstractFetcherThread(name: String,
 val partitionsWithError = mutable.HashSet.empty[TopicPartition]
 
 fetchedEpochs.forKeyValue { (tp, leaderEpochOffset) =>
-  Errors.forCode(leaderEpochOffset.errorCode) match {
-case Errors.NONE =>
-  val offsetTruncationState = getOffsetTruncationState(tp, 
leaderEpochOffset)
-  info(s"Truncating partition $tp with $offsetTruncationState due to 
leader epoch and offset $leaderEpochOffset")
-  if (doTruncate(tp, offsetTruncationState))
-fetchOffsets.put(tp, offsetTruncationState)
-
-case Errors.FENCED_LEADER_EPOCH =>
-  val currentLeaderEpoch = latestEpochsForPartitions.get(tp)
-.map(epochEndOffset => 
Int.box(epochEndOffset.currentLeaderEpoch)).asJava
-  if (onPartitionFenced(tp, currentLeaderEpoch))
+  if (partitionStates.contains(tp)) {
+Errors.forCode(leaderEpochOffset.errorCode) match {
+  case Errors.NONE =>
+val offsetTruncationState = getOffsetTruncationState(tp, 
leaderEpochOffset)
+info(s"Truncating partition $tp with $offsetTruncationState due to 
leader epoch and offset $leaderEpochOffset")
+if (doTruncate(tp, offsetTruncationState))
+  fetchOffsets.put(tp, offsetTruncationState)
+
+  case Errors.FENCED_LEADER_EPOCH =>
+val currentLeaderEpoch = latestEpochsForPartitions.get(tp)
+  .map(epochEndOffset => 
Int.box(epochEndOffset.currentLeaderEpoch)).asJava
+if (onPartitionFenced(tp, currentLeaderEpoch))
+  partitionsWithError += tp
+
+  case error =>
+info(s"Retrying leaderEpoch request for partition $tp as the 
leader reported an error: $error")
 partitionsWithError += tp
-
-case error =>
-  info(s"Retrying leaderEpoch request for partition $tp as the leader 
reported an error: $error")
-  partitionsWithError += tp
+}
+  } else {
+// Partitions may have been removed from the fetcher while the thread 
was waiting for fetch
+// response. Removed partitions are filtered out while holding 
`partitionMapLock` to ensure that we
+// don't update state for any partition that may have already been 
migrated to another thread.
+trace(s"Ignoring epoch offsets for partition $tp since it has been 
removed from this fetcher thread.")
   }
 }
 
diff --git 
a/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala 
b/core/src/test/scala/unit/kafka/server/AbstractFetcherThr

[kafka] branch 2.7 updated: KAFKA-13141; Skip follower fetch offset update in leader if diverging epoch is present (#11136)

2021-07-28 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/2.7 by this push:
 new c69d5f3  KAFKA-13141; Skip follower fetch offset update in leader if 
diverging epoch is present (#11136)
c69d5f3 is described below

commit c69d5f30f9ecac7e0074b21d9170de4837be6067
Author: Rajini Sivaram 
AuthorDate: Wed Jul 28 17:27:26 2021 +0100

KAFKA-13141; Skip follower fetch offset update in leader if diverging epoch 
is present (#11136)

Reviewers: Jason Gustafson 
---
 .../main/scala/kafka/server/ReplicaManager.scala   |  7 ++-
 .../unit/kafka/server/ReplicaManagerTest.scala | 22 ++
 2 files changed, 28 insertions(+), 1 deletion(-)

diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala 
b/core/src/main/scala/kafka/server/ReplicaManager.scala
index 1b819d3..7d40110 100644
--- a/core/src/main/scala/kafka/server/ReplicaManager.scala
+++ b/core/src/main/scala/kafka/server/ReplicaManager.scala
@@ -1709,7 +1709,8 @@ class ReplicaManager(val config: KafkaConfig,
* records in fetch response. Log start/end offset and high watermark may 
change not only due to
* this fetch request, e.g., rolling new log segment and removing old log 
segment may move log
* start offset further than the last offset in the fetched records. The 
followers will get the
-   * updated leader's state in the next fetch response.
+   * updated leader's state in the next fetch response. If follower has a 
diverging epoch or if read
+   * fails with any error, follower fetch state is not updated.
*/
   private def updateFollowerFetchState(followerId: Int,
readResults: Seq[(TopicPartition, 
LogReadResult)]): Seq[(TopicPartition, LogReadResult)] = {
@@ -1718,6 +1719,10 @@ class ReplicaManager(val config: KafkaConfig,
 debug(s"Skipping update of fetch state for follower $followerId since 
the " +
   s"log read returned error ${readResult.error}")
 readResult
+  } else if (readResult.divergingEpoch.nonEmpty) {
+debug(s"Skipping update of fetch state for follower $followerId since 
the " +
+  s"log read returned diverging epoch ${readResult.divergingEpoch}")
+readResult
   } else {
 nonOfflinePartition(topicPartition) match {
   case Some(partition) =>
diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala 
b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
index 6a0f94c..127aff5 100644
--- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
@@ -671,6 +671,28 @@ class ReplicaManagerTest {
   assertEquals(0L, followerReplica.logStartOffset)
   assertEquals(0L, followerReplica.logEndOffset)
 
+  // Next we receive an invalid request with a higher fetch offset, but a 
diverging epoch.
+  // We expect that the replica state does not get updated.
+  val divergingFetchPartitionData = new FetchRequest.PartitionData(3L, 0L, 
maxFetchBytes,
+Optional.of(leaderEpoch), Optional.of(leaderEpoch - 1))
+
+  replicaManager.fetchMessages(
+timeout = 0L,
+replicaId = 1,
+fetchMinBytes = 1,
+fetchMaxBytes = maxFetchBytes,
+hardMaxBytesLimit = false,
+fetchInfos = Seq(tp -> divergingFetchPartitionData),
+quota = UnboundedQuota,
+isolationLevel = IsolationLevel.READ_UNCOMMITTED,
+responseCallback = callback,
+clientMetadata = None
+  )
+
+  assertTrue(successfulFetch.isDefined)
+  assertEquals(0L, followerReplica.logStartOffset)
+  assertEquals(0L, followerReplica.logEndOffset)
+
 } finally {
   replicaManager.shutdown(checkpointHW = false)
 }


[kafka] branch 2.8 updated: KAFKA-13141; Skip follower fetch offset update in leader if diverging epoch is present (#11136)

2021-07-28 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/2.8 by this push:
 new 7e4a315  KAFKA-13141; Skip follower fetch offset update in leader if 
diverging epoch is present (#11136)
7e4a315 is described below

commit 7e4a315963a3d635756183e4c509dcfd82d8359d
Author: Rajini Sivaram 
AuthorDate: Wed Jul 28 17:27:26 2021 +0100

KAFKA-13141; Skip follower fetch offset update in leader if diverging epoch 
is present (#11136)

Reviewers: Jason Gustafson 
---
 .../main/scala/kafka/server/ReplicaManager.scala   |  7 ++-
 .../unit/kafka/server/ReplicaManagerTest.scala | 22 ++
 2 files changed, 28 insertions(+), 1 deletion(-)

diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala 
b/core/src/main/scala/kafka/server/ReplicaManager.scala
index 6e4fc5c..b25516c 100644
--- a/core/src/main/scala/kafka/server/ReplicaManager.scala
+++ b/core/src/main/scala/kafka/server/ReplicaManager.scala
@@ -1753,7 +1753,8 @@ class ReplicaManager(val config: KafkaConfig,
* records in fetch response. Log start/end offset and high watermark may 
change not only due to
* this fetch request, e.g., rolling new log segment and removing old log 
segment may move log
* start offset further than the last offset in the fetched records. The 
followers will get the
-   * updated leader's state in the next fetch response.
+   * updated leader's state in the next fetch response. If follower has a 
diverging epoch or if read
+   * fails with any error, follower fetch state is not updated.
*/
   private def updateFollowerFetchState(followerId: Int,
readResults: Seq[(TopicPartition, 
LogReadResult)]): Seq[(TopicPartition, LogReadResult)] = {
@@ -1762,6 +1763,10 @@ class ReplicaManager(val config: KafkaConfig,
 debug(s"Skipping update of fetch state for follower $followerId since 
the " +
   s"log read returned error ${readResult.error}")
 readResult
+  } else if (readResult.divergingEpoch.nonEmpty) {
+debug(s"Skipping update of fetch state for follower $followerId since 
the " +
+  s"log read returned diverging epoch ${readResult.divergingEpoch}")
+readResult
   } else {
 onlinePartition(topicPartition) match {
   case Some(partition) =>
diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala 
b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
index 9b289e5..881430e 100644
--- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
@@ -683,6 +683,28 @@ class ReplicaManagerTest {
   assertEquals(0L, followerReplica.logStartOffset)
   assertEquals(0L, followerReplica.logEndOffset)
 
+  // Next we receive an invalid request with a higher fetch offset, but a 
diverging epoch.
+  // We expect that the replica state does not get updated.
+  val divergingFetchPartitionData = new FetchRequest.PartitionData(3L, 0L, 
maxFetchBytes,
+Optional.of(leaderEpoch), Optional.of(leaderEpoch - 1))
+
+  replicaManager.fetchMessages(
+timeout = 0L,
+replicaId = 1,
+fetchMinBytes = 1,
+fetchMaxBytes = maxFetchBytes,
+hardMaxBytesLimit = false,
+fetchInfos = Seq(tp -> divergingFetchPartitionData),
+quota = UnboundedQuota,
+isolationLevel = IsolationLevel.READ_UNCOMMITTED,
+responseCallback = callback,
+clientMetadata = None
+  )
+
+  assertTrue(successfulFetch.isDefined)
+  assertEquals(0L, followerReplica.logStartOffset)
+  assertEquals(0L, followerReplica.logEndOffset)
+
 } finally {
   replicaManager.shutdown(checkpointHW = false)
 }


[kafka] branch 3.0 updated: KAFKA-13141; Skip follower fetch offset update in leader if diverging epoch is present (#11136)

2021-07-28 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/3.0 by this push:
 new 047608f  KAFKA-13141; Skip follower fetch offset update in leader if 
diverging epoch is present (#11136)
047608f is described below

commit 047608fe5e6631139d890ce1ca045052daa0e43c
Author: Rajini Sivaram 
AuthorDate: Wed Jul 28 17:27:26 2021 +0100

KAFKA-13141; Skip follower fetch offset update in leader if diverging epoch 
is present (#11136)

Reviewers: Jason Gustafson 
---
 .../main/scala/kafka/server/ReplicaManager.scala   |  7 ++-
 .../unit/kafka/server/ReplicaManagerTest.scala | 22 ++
 2 files changed, 28 insertions(+), 1 deletion(-)

diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala 
b/core/src/main/scala/kafka/server/ReplicaManager.scala
index 6837d81..f03571c 100644
--- a/core/src/main/scala/kafka/server/ReplicaManager.scala
+++ b/core/src/main/scala/kafka/server/ReplicaManager.scala
@@ -1750,7 +1750,8 @@ class ReplicaManager(val config: KafkaConfig,
* records in fetch response. Log start/end offset and high watermark may 
change not only due to
* this fetch request, e.g., rolling new log segment and removing old log 
segment may move log
* start offset further than the last offset in the fetched records. The 
followers will get the
-   * updated leader's state in the next fetch response.
+   * updated leader's state in the next fetch response. If follower has a 
diverging epoch or if read
+   * fails with any error, follower fetch state is not updated.
*/
   private def updateFollowerFetchState(followerId: Int,
readResults: Seq[(TopicPartition, 
LogReadResult)]): Seq[(TopicPartition, LogReadResult)] = {
@@ -1759,6 +1760,10 @@ class ReplicaManager(val config: KafkaConfig,
 debug(s"Skipping update of fetch state for follower $followerId since 
the " +
   s"log read returned error ${readResult.error}")
 readResult
+  } else if (readResult.divergingEpoch.nonEmpty) {
+debug(s"Skipping update of fetch state for follower $followerId since 
the " +
+  s"log read returned diverging epoch ${readResult.divergingEpoch}")
+readResult
   } else {
 onlinePartition(topicPartition) match {
   case Some(partition) =>
diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala 
b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
index 5e2563e..fd6ba75 100644
--- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
@@ -698,6 +698,28 @@ class ReplicaManagerTest {
   assertEquals(0L, followerReplica.logStartOffset)
   assertEquals(0L, followerReplica.logEndOffset)
 
+  // Next we receive an invalid request with a higher fetch offset, but a 
diverging epoch.
+  // We expect that the replica state does not get updated.
+  val divergingFetchPartitionData = new FetchRequest.PartitionData(3L, 0L, 
maxFetchBytes,
+Optional.of(leaderEpoch), Optional.of(leaderEpoch - 1))
+
+  replicaManager.fetchMessages(
+timeout = 0L,
+replicaId = 1,
+fetchMinBytes = 1,
+fetchMaxBytes = maxFetchBytes,
+hardMaxBytesLimit = false,
+fetchInfos = Seq(tp -> divergingFetchPartitionData),
+quota = UnboundedQuota,
+isolationLevel = IsolationLevel.READ_UNCOMMITTED,
+responseCallback = callback,
+clientMetadata = None
+  )
+
+  assertTrue(successfulFetch.isDefined)
+  assertEquals(0L, followerReplica.logStartOffset)
+  assertEquals(0L, followerReplica.logEndOffset)
+
 } finally {
   replicaManager.shutdown(checkpointHW = false)
 }


[kafka] branch trunk updated: KAFKA-13141; Skip follower fetch offset update in leader if diverging epoch is present (#11136)

2021-07-28 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

rsivaram 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 fe0fe68  KAFKA-13141; Skip follower fetch offset update in leader if 
diverging epoch is present (#11136)
fe0fe68 is described below

commit fe0fe686e92d019ac2b5c8407ab2cbb55ae069e1
Author: Rajini Sivaram 
AuthorDate: Wed Jul 28 17:27:26 2021 +0100

KAFKA-13141; Skip follower fetch offset update in leader if diverging epoch 
is present (#11136)

Reviewers: Jason Gustafson 
---
 .../main/scala/kafka/server/ReplicaManager.scala   |  7 ++-
 .../unit/kafka/server/ReplicaManagerTest.scala | 23 ++
 2 files changed, 29 insertions(+), 1 deletion(-)

diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala 
b/core/src/main/scala/kafka/server/ReplicaManager.scala
index 949ff47..a9c99a9 100644
--- a/core/src/main/scala/kafka/server/ReplicaManager.scala
+++ b/core/src/main/scala/kafka/server/ReplicaManager.scala
@@ -1769,7 +1769,8 @@ class ReplicaManager(val config: KafkaConfig,
* records in fetch response. Log start/end offset and high watermark may 
change not only due to
* this fetch request, e.g., rolling new log segment and removing old log 
segment may move log
* start offset further than the last offset in the fetched records. The 
followers will get the
-   * updated leader's state in the next fetch response.
+   * updated leader's state in the next fetch response. If follower has a 
diverging epoch or if read
+   * fails with any error, follower fetch state is not updated.
*/
   private def updateFollowerFetchState(followerId: Int,
readResults: Seq[(TopicPartition, 
LogReadResult)]): Seq[(TopicPartition, LogReadResult)] = {
@@ -1778,6 +1779,10 @@ class ReplicaManager(val config: KafkaConfig,
 debug(s"Skipping update of fetch state for follower $followerId since 
the " +
   s"log read returned error ${readResult.error}")
 readResult
+  } else if (readResult.divergingEpoch.nonEmpty) {
+debug(s"Skipping update of fetch state for follower $followerId since 
the " +
+  s"log read returned diverging epoch ${readResult.divergingEpoch}")
+readResult
   } else {
 onlinePartition(topicPartition) match {
   case Some(partition) =>
diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala 
b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
index 1d875ae..d50aa7b 100644
--- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
@@ -703,6 +703,29 @@ class ReplicaManagerTest {
   assertEquals(0L, followerReplica.logStartOffset)
   assertEquals(0L, followerReplica.logEndOffset)
 
+  // Next we receive an invalid request with a higher fetch offset, but a 
diverging epoch.
+  // We expect that the replica state does not get updated.
+  val divergingFetchPartitionData = new FetchRequest.PartitionData(3L, 0L, 
maxFetchBytes,
+Optional.of(leaderEpoch), Optional.of(leaderEpoch - 1))
+
+  replicaManager.fetchMessages(
+timeout = 0L,
+replicaId = 1,
+fetchMinBytes = 1,
+fetchMaxBytes = maxFetchBytes,
+hardMaxBytesLimit = false,
+fetchInfos = Seq(tp -> divergingFetchPartitionData),
+topicIds = topicIds.asJava,
+quota = UnboundedQuota,
+isolationLevel = IsolationLevel.READ_UNCOMMITTED,
+responseCallback = callback,
+clientMetadata = None
+  )
+
+  assertTrue(successfulFetch.isDefined)
+  assertEquals(0L, followerReplica.logStartOffset)
+  assertEquals(0L, followerReplica.logEndOffset)
+
 } finally {
   replicaManager.shutdown(checkpointHW = false)
 }


[kafka] branch 3.0 updated: KAFKA-13026: Idempotent producer (KAFKA-10619) follow-up testings (#11002)

2021-07-26 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/3.0 by this push:
 new 5fde508  KAFKA-13026: Idempotent producer (KAFKA-10619) follow-up 
testings (#11002)
5fde508 is described below

commit 5fde508731e000f528b7995de5f37602639d84df
Author: Cheng Tan <31675100+d8tlt...@users.noreply.github.com>
AuthorDate: Mon Jul 26 13:45:59 2021 -0700

KAFKA-13026: Idempotent producer (KAFKA-10619) follow-up testings (#11002)

Reviewers: Rajini Sivaram 
---
 .../kafka/clients/producer/ProducerConfig.java |  2 +-
 .../kafka/clients/producer/KafkaProducerTest.java  | 83 ++
 .../sanity_checks/test_verifiable_producer.py  |  8 ++-
 3 files changed, 90 insertions(+), 3 deletions(-)

diff --git 
a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java 
b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java
index 970a70b..0492fbf 100644
--- 
a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java
+++ 
b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java
@@ -286,7 +286,7 @@ public class ProducerConfig extends AbstractConfig {
 Type.STRING,
 "all",
 in("all", "-1", "0", "1"),
-Importance.HIGH,
+Importance.LOW,
 ACKS_DOC)
 .define(COMPRESSION_TYPE_CONFIG, Type.STRING, 
"none", Importance.HIGH, COMPRESSION_TYPE_DOC)
 .define(BATCH_SIZE_CONFIG, Type.INT, 16384, 
atLeast(0), Importance.MEDIUM, BATCH_SIZE_DOC)
diff --git 
a/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java
 
b/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java
index f055e12..2784f19 100644
--- 
a/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java
+++ 
b/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java
@@ -173,6 +173,89 @@ public class KafkaProducerTest {
 }
 
 @Test
+public void testAcksAndIdempotenceForIdempotentProducers() {
+Properties baseProps = new Properties() {{
+setProperty(
+ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:");
+setProperty(
+ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, 
StringSerializer.class.getName());
+setProperty(
+ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, 
StringSerializer.class.getName());
+}};
+
+Properties validProps = new Properties() {{
+putAll(baseProps);
+setProperty(ProducerConfig.ACKS_CONFIG, "0");
+setProperty(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, "false");
+}};
+ProducerConfig config = new ProducerConfig(validProps);
+assertFalse(
+config.getBoolean(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG),
+"idempotence should be overwritten");
+assertEquals(
+"0",
+config.getString(ProducerConfig.ACKS_CONFIG),
+"acks should be overwritten");
+
+Properties validProps2 = new Properties() {{
+putAll(baseProps);
+setProperty(ProducerConfig.TRANSACTIONAL_ID_CONFIG, 
"transactionalId");
+}};
+config = new ProducerConfig(validProps2);
+assertTrue(
+config.getBoolean(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG),
+"idempotence should be set with the default value");
+assertEquals(
+"-1",
+config.getString(ProducerConfig.ACKS_CONFIG),
+"acks should be set with the default value");
+
+Properties validProps3 = new Properties() {{
+putAll(baseProps);
+setProperty(ProducerConfig.ACKS_CONFIG, "all");
+setProperty(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, "false");
+}};
+config = new ProducerConfig(validProps3);
+
assertFalse(config.getBoolean(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG),
+"idempotence should be overwritten");
+assertEquals(
+"-1",
+config.getString(ProducerConfig.ACKS_CONFIG),
+"acks should be overwritten");
+
+Properties invalidProps = new Properties() {{
+putAll(baseProps);
+ 

[kafka] branch trunk updated (c807980 -> 8ed271e)

2021-07-26 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

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


from c807980  MINOR: Update `./gradlew allDepInsight` example in README 
(#11125)
 add 8ed271e  KAFKA-13026: Idempotent producer (KAFKA-10619) follow-up 
testings (#11002)

No new revisions were added by this update.

Summary of changes:
 .../kafka/clients/producer/ProducerConfig.java |  2 +-
 .../kafka/clients/producer/KafkaProducerTest.java  | 83 ++
 .../sanity_checks/test_verifiable_producer.py  |  8 ++-
 3 files changed, 90 insertions(+), 3 deletions(-)


[kafka] branch 3.0 updated: KAFKA-13045: Adding a test for batched offsetFetch requests with one group repeating (#11000)

2021-07-10 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/3.0 by this push:
 new e076d2b  KAFKA-13045: Adding a test for batched offsetFetch requests 
with one group repeating (#11000)
e076d2b is described below

commit e076d2b32bec0db190796b9ce3c9541a8ea0999d
Author: Sanjana Kaundinya 
AuthorDate: Sat Jul 10 00:43:55 2021 -0700

KAFKA-13045: Adding a test for batched offsetFetch requests with one group 
repeating (#11000)

Reviewers: Rajini Sivaram 
---
 .../kafka/api/AuthorizerIntegrationTest.scala  |  58 ++--
 .../unit/kafka/server/OffsetFetchRequestTest.scala | 147 ++---
 2 files changed, 84 insertions(+), 121 deletions(-)

diff --git 
a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala 
b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala
index fd5f12c..36cdc51 100644
--- a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala
+++ b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala
@@ -1422,40 +1422,23 @@ class AuthorizerIntegrationTest extends BaseRequestTest 
{
 val offset = 15L
 val leaderEpoch: Optional[Integer] = Optional.of(1)
 val metadata = "metadata"
-val topicOneOffsets = topic1List.asScala.map {
-  tp => (tp, new OffsetAndMetadata(offset, leaderEpoch, metadata))
-}.toMap.asJava
-val topicOneAndTwoOffsets = topic1And2List.asScala.map {
-  tp => (tp, new OffsetAndMetadata(offset, leaderEpoch, metadata))
-}.toMap.asJava
-val allTopicOffsets = allTopicsList.asScala.map {
-  tp => (tp, new OffsetAndMetadata(offset, leaderEpoch, metadata))
-}.toMap.asJava
 
-// create 5 consumers to commit offsets so we can fetch them later
-
-def commitOffsets(tpList: util.List[TopicPartition],
-  offsets: util.Map[TopicPartition, OffsetAndMetadata]): 
Unit = {
+def commitOffsets(tpList: util.List[TopicPartition]): Unit = {
   val consumer = createConsumer()
   consumer.assign(tpList)
+  val offsets = tpList.asScala.map{
+tp => (tp, new OffsetAndMetadata(offset, leaderEpoch, metadata))
+  }.toMap.asJava
   consumer.commitSync(offsets)
   consumer.close()
 }
 
-consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, groups(0))
-commitOffsets(topic1List, topicOneOffsets)
-
-consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, groups(1))
-commitOffsets(topic1And2List, topicOneAndTwoOffsets)
-
-consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, groups(2))
-commitOffsets(allTopicsList, allTopicOffsets)
-
-consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, groups(3))
-commitOffsets(allTopicsList, allTopicOffsets)
-
-consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, groups(4))
-commitOffsets(allTopicsList, allTopicOffsets)
+// create 5 consumers to commit offsets so we can fetch them later
+val partitionMap = groupToPartitionMap.asScala.map(e => (e._1, 
Option(e._2).getOrElse(allTopicsList)))
+groups.foreach { groupId =>
+  consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, groupId)
+  commitOffsets(partitionMap(groupId))
+}
 
 removeAllClientAcls()
 
@@ -1548,24 +1531,9 @@ class AuthorizerIntegrationTest extends BaseRequestTest {
 // from the offsetFetch response
 addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, 
WildcardHost, DESCRIBE, ALLOW)), topicResources(2))
 offsetFetchResponse = 
connectAndReceive[OffsetFetchResponse](offsetFetchRequest)
-offsetFetchResponse.data().groups().forEach(g =>
-  g.groupId() match {
-case "group1" =>
-  verifyResponse(offsetFetchResponse.groupLevelError(groups(0)), 
offsetFetchResponse
-.partitionDataMap(groups(0)), topic1List)
-case "group2" =>
-  verifyResponse(offsetFetchResponse.groupLevelError(groups(1)), 
offsetFetchResponse
-.partitionDataMap(groups(1)), topic1And2List)
-case "group3" =>
-  verifyResponse(offsetFetchResponse.groupLevelError(groups(2)), 
offsetFetchResponse
-.partitionDataMap(groups(2)), allTopicsList)
-case "group4" =>
-  verifyResponse(offsetFetchResponse.groupLevelError(groups(3)), 
offsetFetchResponse
-.partitionDataMap(groups(3)), allTopicsList)
-case "group5" =>
-  verifyResponse(offsetFetchResponse.groupLevelError(groups(4)), 
offsetFetchResponse
-.partitionDataMap(groups(4)), allTopicsList)
-  })
+offsetFetchResponse.data.groups.asScala.map(_.groupId).foreach( groupId =>
+  verifyResponse(offsetFetchResponse.groupLevelError(groupId), 
offsetFetchResponse.partit

[kafka] branch trunk updated: KAFKA-13045: Adding a test for batched offsetFetch requests with one group repeating (#11000)

2021-07-10 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

rsivaram 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 af282f7  KAFKA-13045: Adding a test for batched offsetFetch requests 
with one group repeating (#11000)
af282f7 is described below

commit af282f76a7fcf15e87071523b2a7dcae9e10524b
Author: Sanjana Kaundinya 
AuthorDate: Sat Jul 10 00:43:55 2021 -0700

KAFKA-13045: Adding a test for batched offsetFetch requests with one group 
repeating (#11000)

Reviewers: Rajini Sivaram 
---
 .../kafka/api/AuthorizerIntegrationTest.scala  |  58 ++--
 .../unit/kafka/server/OffsetFetchRequestTest.scala | 147 ++---
 2 files changed, 84 insertions(+), 121 deletions(-)

diff --git 
a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala 
b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala
index 206ad50..264f6eb 100644
--- a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala
+++ b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala
@@ -1446,40 +1446,23 @@ class AuthorizerIntegrationTest extends BaseRequestTest 
{
 val offset = 15L
 val leaderEpoch: Optional[Integer] = Optional.of(1)
 val metadata = "metadata"
-val topicOneOffsets = topic1List.asScala.map {
-  tp => (tp, new OffsetAndMetadata(offset, leaderEpoch, metadata))
-}.toMap.asJava
-val topicOneAndTwoOffsets = topic1And2List.asScala.map {
-  tp => (tp, new OffsetAndMetadata(offset, leaderEpoch, metadata))
-}.toMap.asJava
-val allTopicOffsets = allTopicsList.asScala.map {
-  tp => (tp, new OffsetAndMetadata(offset, leaderEpoch, metadata))
-}.toMap.asJava
 
-// create 5 consumers to commit offsets so we can fetch them later
-
-def commitOffsets(tpList: util.List[TopicPartition],
-  offsets: util.Map[TopicPartition, OffsetAndMetadata]): 
Unit = {
+def commitOffsets(tpList: util.List[TopicPartition]): Unit = {
   val consumer = createConsumer()
   consumer.assign(tpList)
+  val offsets = tpList.asScala.map{
+tp => (tp, new OffsetAndMetadata(offset, leaderEpoch, metadata))
+  }.toMap.asJava
   consumer.commitSync(offsets)
   consumer.close()
 }
 
-consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, groups(0))
-commitOffsets(topic1List, topicOneOffsets)
-
-consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, groups(1))
-commitOffsets(topic1And2List, topicOneAndTwoOffsets)
-
-consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, groups(2))
-commitOffsets(allTopicsList, allTopicOffsets)
-
-consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, groups(3))
-commitOffsets(allTopicsList, allTopicOffsets)
-
-consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, groups(4))
-commitOffsets(allTopicsList, allTopicOffsets)
+// create 5 consumers to commit offsets so we can fetch them later
+val partitionMap = groupToPartitionMap.asScala.map(e => (e._1, 
Option(e._2).getOrElse(allTopicsList)))
+groups.foreach { groupId =>
+  consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, groupId)
+  commitOffsets(partitionMap(groupId))
+}
 
 removeAllClientAcls()
 
@@ -1572,24 +1555,9 @@ class AuthorizerIntegrationTest extends BaseRequestTest {
 // from the offsetFetch response
 addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, 
WildcardHost, DESCRIBE, ALLOW)), topicResources(2))
 offsetFetchResponse = 
connectAndReceive[OffsetFetchResponse](offsetFetchRequest)
-offsetFetchResponse.data().groups().forEach(g =>
-  g.groupId() match {
-case "group1" =>
-  verifyResponse(offsetFetchResponse.groupLevelError(groups(0)), 
offsetFetchResponse
-.partitionDataMap(groups(0)), topic1List)
-case "group2" =>
-  verifyResponse(offsetFetchResponse.groupLevelError(groups(1)), 
offsetFetchResponse
-.partitionDataMap(groups(1)), topic1And2List)
-case "group3" =>
-  verifyResponse(offsetFetchResponse.groupLevelError(groups(2)), 
offsetFetchResponse
-.partitionDataMap(groups(2)), allTopicsList)
-case "group4" =>
-  verifyResponse(offsetFetchResponse.groupLevelError(groups(3)), 
offsetFetchResponse
-.partitionDataMap(groups(3)), allTopicsList)
-case "group5" =>
-  verifyResponse(offsetFetchResponse.groupLevelError(groups(4)), 
offsetFetchResponse
-.partitionDataMap(groups(4)), allTopicsList)
-  })
+offsetFetchResponse.data.groups.asScala.map(_.groupId).foreach( groupId =>
+  verifyResponse(offsetFetchResponse.groupLevelError(groupId), 
offsetFetchResponse.

[kafka] branch 3.0 updated: KAFKA-12234: Implement request/response for offsetFetch batching (KIP-709) (#10962)

2021-07-07 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/3.0 by this push:
 new 471a5e0  KAFKA-12234: Implement request/response for offsetFetch 
batching (KIP-709) (#10962)
471a5e0 is described below

commit 471a5e0e9c107e94031f368abe7672b142eca476
Author: Sanjana Kaundinya 
AuthorDate: Wed Jul 7 03:55:00 2021 -0700

KAFKA-12234: Implement request/response for offsetFetch batching (KIP-709) 
(#10962)

This implements the request and response portion of KIP-709. It updates the 
OffsetFetch request and response to support fetching offsets for multiple 
consumer groups at a time. If the broker does not support the new OffsetFetch 
version, clients can revert to the previous behaviour and use a request for 
each coordinator.

Reviewers: Rajini Sivaram , Konstantine 
Karantasis 
---
 checkstyle/suppressions.xml|   2 +-
 .../internals/ListConsumerGroupOffsetsHandler.java |  10 +-
 .../consumer/internals/ConsumerCoordinator.java|  26 +-
 .../kafka/common/requests/OffsetFetchRequest.java  | 170 ++-
 .../kafka/common/requests/OffsetFetchResponse.java | 131 +++-
 .../common/message/OffsetFetchRequest.json |  26 +-
 .../common/message/OffsetFetchResponse.json|  49 ++-
 .../ListConsumerGroupOffsetsHandlerTest.java   |   8 +-
 .../apache/kafka/common/message/MessageTest.java   | 242 ++-
 .../common/requests/OffsetFetchRequestTest.java| 194 +---
 .../common/requests/OffsetFetchResponseTest.java   | 331 +
 .../kafka/common/requests/RequestResponseTest.java | 147 +++--
 core/src/main/scala/kafka/server/KafkaApis.scala   | 173 +++
 .../kafka/api/AuthorizerIntegrationTest.scala  | 215 -
 .../unit/kafka/server/OffsetFetchRequestTest.scala | 237 +++
 15 files changed, 1701 insertions(+), 260 deletions(-)

diff --git a/checkstyle/suppressions.xml b/checkstyle/suppressions.xml
index cfa7272..0b1ccb0 100644
--- a/checkstyle/suppressions.xml
+++ b/checkstyle/suppressions.xml
@@ -85,7 +85,7 @@
 
files="clients[\\/]src[\\/](generated|generated-test)[\\/].+.java$"/>
 
 
+files="MessageTest.java|OffsetFetchRequest.java"/>
 
 
  failed = new HashMap<>();
 List unmapped = new ArrayList<>();
 
-if (response.error() != Errors.NONE) {
-handleError(groupId, response.error(), failed, unmapped);
+Errors responseError = response.groupLevelError(groupId.idValue);
+if (responseError != Errors.NONE) {
+handleError(groupId, responseError, failed, unmapped);
 } else {
 final Map groupOffsetsListing = 
new HashMap<>();
-for (Map.Entry 
entry :
-response.responseData().entrySet()) {
+Map 
partitionDataMap =
+response.partitionDataMap(groupId.idValue);
+for (Map.Entry 
entry : partitionDataMap.entrySet()) {
 final TopicPartition topicPartition = entry.getKey();
 OffsetFetchResponse.PartitionData partitionData = 
entry.getValue();
 final Errors error = partitionData.error;
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 39f4520..68cf8a9 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
@@ -1308,29 +1308,31 @@ public final class ConsumerCoordinator extends 
AbstractCoordinator {
 
 @Override
 public void handle(OffsetFetchResponse response, 
RequestFuture> future) {
-if (response.hasError()) {
-Errors error = response.error();
-log.debug("Offset fetch failed: {}", error.message());
+Errors responseError = 
response.groupLevelError(rebalanceConfig.groupId);
+if (responseError != Errors.NONE) {
+log.debug("Offset fetch failed: {}", responseError.message());
 
-if (error == Errors.COORDINATOR_LOAD_IN_PROGRESS) {
+if (responseError == Errors.COORDINATOR_LOAD_IN_PROGRESS) {
 // just retry
-future.raise(error);
-} else if (error == Errors.NOT_COORDINATOR) {
+future.raise(responseError);
+} else if (responseError == Errors.NOT_COORDINATOR) {
 // re-discover the coordinator and retry
-markCoordinatorUnknown(error);
-future.raise(error);
-  

[kafka] branch trunk updated (c671e44 -> e00c0f3)

2021-07-07 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

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


from c671e44  MINOR: Add `KafkaAdminClient.getListOffsetsCalls` benchmark 
(#10955)
 add e00c0f3  KAFKA-12234: Implement request/response for offsetFetch 
batching (KIP-709) (#10962)

No new revisions were added by this update.

Summary of changes:
 checkstyle/suppressions.xml|   2 +-
 .../internals/ListConsumerGroupOffsetsHandler.java |  10 +-
 .../consumer/internals/ConsumerCoordinator.java|  26 +-
 .../kafka/common/requests/OffsetFetchRequest.java  | 170 ++-
 .../kafka/common/requests/OffsetFetchResponse.java | 131 +++-
 .../common/message/OffsetFetchRequest.json |  26 +-
 .../common/message/OffsetFetchResponse.json|  49 ++-
 .../ListConsumerGroupOffsetsHandlerTest.java   |   8 +-
 .../apache/kafka/common/message/MessageTest.java   | 242 ++-
 .../common/requests/OffsetFetchRequestTest.java| 194 +---
 .../common/requests/OffsetFetchResponseTest.java   | 331 +
 .../kafka/common/requests/RequestResponseTest.java | 147 +++--
 core/src/main/scala/kafka/server/KafkaApis.scala   | 173 +++
 .../kafka/api/AuthorizerIntegrationTest.scala  | 215 -
 .../unit/kafka/server/OffsetFetchRequestTest.scala | 237 +++
 15 files changed, 1701 insertions(+), 260 deletions(-)
 create mode 100644 
core/src/test/scala/unit/kafka/server/OffsetFetchRequestTest.scala


[kafka] branch trunk updated (16d224d -> 2b6365c)

2021-07-05 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

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


from 16d224d  MINOR: Delete unused DeleteTopicsWithIdsResult (#10957)
 add 2b6365c  KAFKA-13029; Set appropriate fields for 
FindCoordinatorRequest based on version (#10965)

No new revisions were added by this update.

Summary of changes:
 .../admin/internals/CoordinatorStrategy.java   | 25 +-
 .../consumer/internals/AbstractCoordinator.java| 54 ++
 .../producer/internals/TransactionManager.java | 46 +-
 .../common/requests/FindCoordinatorRequest.java| 20 ++--
 .../common/requests/FindCoordinatorResponse.java   | 16 +++
 .../kafka/clients/admin/KafkaAdminClientTest.java  | 54 +++---
 .../kafka/common/requests/RequestResponseTest.java | 42 +
 7 files changed, 150 insertions(+), 107 deletions(-)


[kafka] branch 2.7 updated: KAFKA-12996; Return OFFSET_OUT_OF_RANGE for fetchOffset < startOffset even for diverging epochs (#10930)

2021-06-29 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/2.7 by this push:
 new 51a1aec  KAFKA-12996; Return OFFSET_OUT_OF_RANGE for fetchOffset < 
startOffset even for diverging epochs (#10930)
51a1aec is described below

commit 51a1aecc0914f116fbef151c2e16db5d517a980b
Author: Rajini Sivaram 
AuthorDate: Tue Jun 29 16:49:36 2021 +0100

KAFKA-12996; Return OFFSET_OUT_OF_RANGE for fetchOffset < startOffset even 
for diverging epochs (#10930)

If fetchOffset < startOffset, we currently throw OffsetOutOfRangeException 
when attempting to read from the log in the regular case. But for diverging 
epochs, we return Errors.NONE with the new leader start offset, hwm etc.. 
ReplicaFetcherThread throws OffsetOutOfRangeException when processing responses 
with Errors.NONE if the leader's offsets in the response are out of range and 
this moves the partition to failed state. The PR adds a check for this case 
when processing fetch requests [...]

Reviewers: Luke Chen , Nikhil Bhatia 
, Guozhang Wang 
---
 core/src/main/scala/kafka/cluster/Partition.scala  | 6 ++
 core/src/test/scala/unit/kafka/cluster/PartitionTest.scala | 9 +
 2 files changed, 15 insertions(+)

diff --git a/core/src/main/scala/kafka/cluster/Partition.scala 
b/core/src/main/scala/kafka/cluster/Partition.scala
index 586afbf..295be26 100755
--- a/core/src/main/scala/kafka/cluster/Partition.scala
+++ b/core/src/main/scala/kafka/cluster/Partition.scala
@@ -1118,6 +1118,12 @@ class Partition(val topicPartition: TopicPartition,
   s"$lastFetchedEpoch from the request")
   }
 
+  // If fetch offset is less than log start, fail with 
OffsetOutOfRangeException, regardless of whether epochs are diverging
+  if (fetchOffset < initialLogStartOffset) {
+throw new OffsetOutOfRangeException(s"Received request for offset 
$fetchOffset for partition $topicPartition, " +
+  s"but we only have log segments in the range $initialLogStartOffset 
to $initialLogEndOffset.")
+  }
+
   if (epochEndOffset.leaderEpoch < fetchEpoch || epochEndOffset.endOffset 
< fetchOffset) {
 val emptyFetchData = FetchDataInfo(
   fetchOffsetMetadata = LogOffsetMetadata(fetchOffset),
diff --git a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala 
b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala
index d7aa33b..9cf9fc7 100644
--- a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala
+++ b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala
@@ -127,6 +127,15 @@ class PartitionTest extends AbstractPartitionTest {
 assertThrows[OffsetOutOfRangeException] {
   read(lastFetchedEpoch = 0, fetchOffset = 0)
 }
+
+// Fetch offset lower than start offset should throw 
OffsetOutOfRangeException
+log.maybeIncrementLogStartOffset(newLogStartOffset = 10, 
ClientRecordDeletion)
+assertThrows[OffsetOutOfRangeException] {
+  read(lastFetchedEpoch = 5, fetchOffset = 6) // diverging
+}
+assertThrows[OffsetOutOfRangeException] {
+  read(lastFetchedEpoch = 3, fetchOffset = 6) // not diverging
+}
   }
 
   @Test


[kafka] branch 2.8 updated: KAFKA-12996; Return OFFSET_OUT_OF_RANGE for fetchOffset < startOffset even for diverging epochs (#10930)

2021-06-29 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/2.8 by this push:
 new fc0a10f  KAFKA-12996; Return OFFSET_OUT_OF_RANGE for fetchOffset < 
startOffset even for diverging epochs (#10930)
fc0a10f is described below

commit fc0a10f1a8cc510f23c43851fe73bb31a5b7f491
Author: Rajini Sivaram 
AuthorDate: Tue Jun 29 16:49:36 2021 +0100

KAFKA-12996; Return OFFSET_OUT_OF_RANGE for fetchOffset < startOffset even 
for diverging epochs (#10930)

If fetchOffset < startOffset, we currently throw OffsetOutOfRangeException 
when attempting to read from the log in the regular case. But for diverging 
epochs, we return Errors.NONE with the new leader start offset, hwm etc.. 
ReplicaFetcherThread throws OffsetOutOfRangeException when processing responses 
with Errors.NONE if the leader's offsets in the response are out of range and 
this moves the partition to failed state. The PR adds a check for this case 
when processing fetch requests [...]

Reviewers: Luke Chen , Nikhil Bhatia 
, Guozhang Wang 
---
 core/src/main/scala/kafka/cluster/Partition.scala  | 6 ++
 core/src/test/scala/unit/kafka/cluster/PartitionTest.scala | 5 +
 2 files changed, 11 insertions(+)

diff --git a/core/src/main/scala/kafka/cluster/Partition.scala 
b/core/src/main/scala/kafka/cluster/Partition.scala
index 917ca50..9b35a00 100755
--- a/core/src/main/scala/kafka/cluster/Partition.scala
+++ b/core/src/main/scala/kafka/cluster/Partition.scala
@@ -1109,6 +1109,12 @@ class Partition(val topicPartition: TopicPartition,
   s"$lastFetchedEpoch from the request")
   }
 
+  // If fetch offset is less than log start, fail with 
OffsetOutOfRangeException, regardless of whether epochs are diverging
+  if (fetchOffset < initialLogStartOffset) {
+throw new OffsetOutOfRangeException(s"Received request for offset 
$fetchOffset for partition $topicPartition, " +
+  s"but we only have log segments in the range $initialLogStartOffset 
to $initialLogEndOffset.")
+  }
+
   if (epochEndOffset.leaderEpoch < fetchEpoch || epochEndOffset.endOffset 
< fetchOffset) {
 val emptyFetchData = FetchDataInfo(
   fetchOffsetMetadata = LogOffsetMetadata(fetchOffset),
diff --git a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala 
b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala
index d569c89..b3369df 100644
--- a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala
+++ b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala
@@ -124,6 +124,11 @@ class PartitionTest extends AbstractPartitionTest {
 assertNoDivergence(read(lastFetchedEpoch = 3, fetchOffset = 5))
 
 assertThrows(classOf[OffsetOutOfRangeException], () => 
read(lastFetchedEpoch = 0, fetchOffset = 0))
+
+// Fetch offset lower than start offset should throw 
OffsetOutOfRangeException
+log.maybeIncrementLogStartOffset(newLogStartOffset = 10, 
ClientRecordDeletion)
+assertThrows(classOf[OffsetOutOfRangeException], () => 
read(lastFetchedEpoch = 5, fetchOffset = 6)) // diverging
+assertThrows(classOf[OffsetOutOfRangeException], () => 
read(lastFetchedEpoch = 3, fetchOffset = 6)) // not diverging
   }
 
   @Test


[kafka] branch 2.7 updated: KAFKA-12948: Remove node from ClusterConnectionStates.connectingNodes when node is removed (#10882)

2021-06-15 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/2.7 by this push:
 new 80b153d  KAFKA-12948: Remove node from 
ClusterConnectionStates.connectingNodes when node is removed (#10882)
80b153d is described below

commit 80b153dbba649d020999335b827e260417a97ae6
Author: Rajini Sivaram 
AuthorDate: Tue Jun 15 09:18:30 2021 +0100

KAFKA-12948: Remove node from ClusterConnectionStates.connectingNodes when 
node is removed (#10882)

NetworkClient.poll() throws IllegalStateException when checking 
isConnectionSetupTimeout if all nodes in 
ClusterConnectionStates.connectingNodes aren't present in 
ClusterConnectionStates.nodeState. This commit ensures that when we remove a 
node from nodeState, we also remove from connectingNodes.

Reviewers: David Jacot 
---
 .../kafka/clients/ClusterConnectionStates.java |  1 +
 .../apache/kafka/clients/NetworkClientTest.java| 33 ++
 2 files changed, 34 insertions(+)

diff --git 
a/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java 
b/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java
index 16bf59ac..7a5dc93 100644
--- 
a/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java
+++ 
b/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java
@@ -389,6 +389,7 @@ final class ClusterConnectionStates {
  */
 public void remove(String id) {
 nodeState.remove(id);
+connectingNodes.remove(id);
 }
 
 /**
diff --git 
a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java 
b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java
index de0e524..6dd2ed7 100644
--- a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java
@@ -1099,6 +1099,39 @@ public class NetworkClientTest {
 assertEquals(2, mockHostResolver.resolutionCount());
 }
 
+@Test
+public void testCloseConnectingNode() {
+Cluster cluster = TestUtils.clusterWith(2);
+Node node0 = cluster.nodeById(0);
+Node node1 = cluster.nodeById(1);
+client.ready(node0, time.milliseconds());
+selector.serverConnectionBlocked(node0.idString());
+client.poll(1, time.milliseconds());
+client.close(node0.idString());
+
+// Poll without any connections should return without exceptions
+client.poll(0, time.milliseconds());
+assertFalse(NetworkClientUtils.isReady(client, node0, 
time.milliseconds()));
+assertFalse(NetworkClientUtils.isReady(client, node1, 
time.milliseconds()));
+
+// Connection to new node should work
+client.ready(node1, time.milliseconds());
+ByteBuffer buffer = 
ApiVersionsResponse.DEFAULT_API_VERSIONS_RESPONSE.serialize(ApiKeys.API_VERSIONS,
 ApiKeys.API_VERSIONS.latestVersion(), 0);
+selector.delayedReceive(new DelayedReceive(node1.idString(), new 
NetworkReceive(node1.idString(), buffer)));
+while (!client.ready(node1, time.milliseconds()))
+client.poll(1, time.milliseconds());
+assertTrue(client.isReady(node1, time.milliseconds()));
+selector.clear();
+
+// New connection to node closed earlier should work
+client.ready(node0, time.milliseconds());
+buffer = 
ApiVersionsResponse.DEFAULT_API_VERSIONS_RESPONSE.serialize(ApiKeys.API_VERSIONS,
 ApiKeys.API_VERSIONS.latestVersion(), 1);
+selector.delayedReceive(new DelayedReceive(node0.idString(), new 
NetworkReceive(node0.idString(), buffer)));
+while (!client.ready(node0, time.milliseconds()))
+client.poll(1, time.milliseconds());
+assertTrue(client.isReady(node0, time.milliseconds()));
+}
+
 private RequestHeader parseHeader(ByteBuffer buffer) {
 buffer.getInt(); // skip size
 return RequestHeader.parse(buffer.slice());


[kafka] branch 2.8 updated: KAFKA-12948: Remove node from ClusterConnectionStates.connectingNodes when node is removed (#10882)

2021-06-15 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/2.8 by this push:
 new 3878165  KAFKA-12948: Remove node from 
ClusterConnectionStates.connectingNodes when node is removed (#10882)
3878165 is described below

commit 3878165a066791bf850030423724b4342da98985
Author: Rajini Sivaram 
AuthorDate: Tue Jun 15 09:18:30 2021 +0100

KAFKA-12948: Remove node from ClusterConnectionStates.connectingNodes when 
node is removed (#10882)

NetworkClient.poll() throws IllegalStateException when checking 
isConnectionSetupTimeout if all nodes in 
ClusterConnectionStates.connectingNodes aren't present in 
ClusterConnectionStates.nodeState. This commit ensures that when we remove a 
node from nodeState, we also remove from connectingNodes.

Reviewers: David Jacot 
---
 .../kafka/clients/ClusterConnectionStates.java |  1 +
 .../apache/kafka/clients/NetworkClientTest.java| 33 ++
 2 files changed, 34 insertions(+)

diff --git 
a/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java 
b/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java
index e00494c..20de2ec 100644
--- 
a/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java
+++ 
b/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java
@@ -389,6 +389,7 @@ final class ClusterConnectionStates {
  */
 public void remove(String id) {
 nodeState.remove(id);
+connectingNodes.remove(id);
 }
 
 /**
diff --git 
a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java 
b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java
index b13f854..dac6424 100644
--- a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java
@@ -1064,6 +1064,39 @@ public class NetworkClientTest {
 assertEquals(2, mockHostResolver.resolutionCount());
 }
 
+@Test
+public void testCloseConnectingNode() {
+Cluster cluster = TestUtils.clusterWith(2);
+Node node0 = cluster.nodeById(0);
+Node node1 = cluster.nodeById(1);
+client.ready(node0, time.milliseconds());
+selector.serverConnectionBlocked(node0.idString());
+client.poll(1, time.milliseconds());
+client.close(node0.idString());
+
+// Poll without any connections should return without exceptions
+client.poll(0, time.milliseconds());
+assertFalse(NetworkClientUtils.isReady(client, node0, 
time.milliseconds()));
+assertFalse(NetworkClientUtils.isReady(client, node1, 
time.milliseconds()));
+
+// Connection to new node should work
+client.ready(node1, time.milliseconds());
+ByteBuffer buffer = 
RequestTestUtils.serializeResponseWithHeader(defaultApiVersionsResponse(), 
ApiKeys.API_VERSIONS.latestVersion(), 0);
+selector.delayedReceive(new DelayedReceive(node1.idString(), new 
NetworkReceive(node1.idString(), buffer)));
+while (!client.ready(node1, time.milliseconds()))
+client.poll(1, time.milliseconds());
+assertTrue(client.isReady(node1, time.milliseconds()));
+selector.clear();
+
+// New connection to node closed earlier should work
+client.ready(node0, time.milliseconds());
+buffer = 
RequestTestUtils.serializeResponseWithHeader(defaultApiVersionsResponse(), 
ApiKeys.API_VERSIONS.latestVersion(), 1);
+selector.delayedReceive(new DelayedReceive(node0.idString(), new 
NetworkReceive(node0.idString(), buffer)));
+while (!client.ready(node0, time.milliseconds()))
+client.poll(1, time.milliseconds());
+assertTrue(client.isReady(node0, time.milliseconds()));
+}
+
 private RequestHeader parseHeader(ByteBuffer buffer) {
 buffer.getInt(); // skip size
 return RequestHeader.parse(buffer.slice());


[kafka] branch trunk updated (01967e4 -> 1e88c75)

2021-06-15 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

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


from 01967e4  KAFKA-12914: StreamSourceNode should return `null` topic name 
for pattern subscription (#10846)
 add 1e88c75  KAFKA-12948: Remove node from 
ClusterConnectionStates.connectingNodes when node is removed (#10882)

No new revisions were added by this update.

Summary of changes:
 .../kafka/clients/ClusterConnectionStates.java |  1 +
 .../apache/kafka/clients/NetworkClientTest.java| 33 ++
 2 files changed, 34 insertions(+)


[kafka] branch trunk updated (4c1efd3 -> 13ba9c9)

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

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


from 4c1efd3  MINOR: Update kafka-topics.sh line command tool upgrade notes 
with removed option (#10806)
 add 13ba9c9  KAFKA-12867: Fix ConsumeBenchWorker exit behavior for 
maxMessages config (#10797)

No new revisions were added by this update.

Summary of changes:
 .../main/java/org/apache/kafka/trogdor/workload/ConsumeBenchWorker.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)


[kafka] branch 2.7 updated: KAFKA-12751: Reset AlterIsr in-flight state for duplicate update requests (#10633)

2021-05-17 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/2.7 by this push:
 new aee9edc  KAFKA-12751: Reset AlterIsr in-flight state for duplicate 
update requests (#10633)
aee9edc is described below

commit aee9edce6a39138d3a5ce87d4511040ccd40fa8f
Author: Rajini Sivaram 
AuthorDate: Mon May 17 19:31:39 2021 +0100

KAFKA-12751: Reset AlterIsr in-flight state for duplicate update requests 
(#10633)

Reviewers: David Arthur 
---
 core/src/main/scala/kafka/cluster/Partition.scala | 7 ++-
 1 file changed, 6 insertions(+), 1 deletion(-)

diff --git a/core/src/main/scala/kafka/cluster/Partition.scala 
b/core/src/main/scala/kafka/cluster/Partition.scala
index 4e06198..586afbf 100755
--- a/core/src/main/scala/kafka/cluster/Partition.scala
+++ b/core/src/main/scala/kafka/cluster/Partition.scala
@@ -1442,10 +1442,15 @@ class Partition(val topicPartition: TopicPartition,
   if (leaderAndIsr.leaderEpoch != leaderEpoch) {
 debug(s"Ignoring ISR from AlterIsr with ${leaderAndIsr} since we 
have a stale leader epoch $leaderEpoch.")
 isrChangeListener.markFailed()
-  } else if (leaderAndIsr.zkVersion <= zkVersion) {
+  } else if (leaderAndIsr.zkVersion < zkVersion) {
 debug(s"Ignoring ISR from AlterIsr with ${leaderAndIsr} since we 
have a newer version $zkVersion.")
 isrChangeListener.markFailed()
   } else {
+// This is one of two states:
+//   1) leaderAndIsr.zkVersion > zkVersion: Controller updated to 
new version with proposedIsrState.
+//   2) leaderAndIsr.zkVersion == zkVersion: No update was 
performed since proposed and actual state are the same.
+// In both cases, we want to move from Pending to Committed state 
to ensure new updates are processed.
+
 isrState = CommittedIsr(leaderAndIsr.isr.toSet)
 zkVersion = leaderAndIsr.zkVersion
 info(s"ISR updated from AlterIsr to ${isrState.isr.mkString(",")} 
and version updated to [$zkVersion]")


[kafka] branch 2.8 updated: KAFKA-12751: Reset AlterIsr in-flight state for duplicate update requests (#10633)

2021-05-17 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/2.8 by this push:
 new aebca9b  KAFKA-12751: Reset AlterIsr in-flight state for duplicate 
update requests (#10633)
aebca9b is described below

commit aebca9b69bec2b5e7d0744cf1acf732d6bc1d9b7
Author: Rajini Sivaram 
AuthorDate: Mon May 17 19:31:39 2021 +0100

KAFKA-12751: Reset AlterIsr in-flight state for duplicate update requests 
(#10633)

Reviewers: David Arthur 
---
 core/src/main/scala/kafka/cluster/Partition.scala  | 7 ++-
 core/src/test/scala/unit/kafka/cluster/PartitionTest.scala | 7 +++
 2 files changed, 13 insertions(+), 1 deletion(-)

diff --git a/core/src/main/scala/kafka/cluster/Partition.scala 
b/core/src/main/scala/kafka/cluster/Partition.scala
index 5442fc8..917ca50 100755
--- a/core/src/main/scala/kafka/cluster/Partition.scala
+++ b/core/src/main/scala/kafka/cluster/Partition.scala
@@ -1412,10 +1412,15 @@ class Partition(val topicPartition: TopicPartition,
   if (leaderAndIsr.leaderEpoch != leaderEpoch) {
 debug(s"Ignoring new ISR ${leaderAndIsr} since we have a stale 
leader epoch $leaderEpoch.")
 isrChangeListener.markFailed()
-  } else if (leaderAndIsr.zkVersion <= zkVersion) {
+  } else if (leaderAndIsr.zkVersion < zkVersion) {
 debug(s"Ignoring new ISR ${leaderAndIsr} since we have a newer 
version $zkVersion.")
 isrChangeListener.markFailed()
   } else {
+// This is one of two states:
+//   1) leaderAndIsr.zkVersion > zkVersion: Controller updated to 
new version with proposedIsrState.
+//   2) leaderAndIsr.zkVersion == zkVersion: No update was 
performed since proposed and actual state are the same.
+// In both cases, we want to move from Pending to Committed state 
to ensure new updates are processed.
+
 isrState = CommittedIsr(leaderAndIsr.isr.toSet)
 zkVersion = leaderAndIsr.zkVersion
 info(s"ISR updated to ${isrState.isr.mkString(",")} and version 
updated to [$zkVersion]")
diff --git a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala 
b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala
index 9e66969..d569c89 100644
--- a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala
+++ b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala
@@ -1612,6 +1612,13 @@ class PartitionTest extends AbstractPartitionTest {
   case _ => fail("Expected a committed ISR following Zk expansion")
 }
 
+// Verify duplicate request. In-flight state should be reset even though 
version hasn't changed.
+doAnswer(_ => (true, 2))
+  .when(kafkaZkClient)
+  .conditionalUpdatePath(anyString(), any(), ArgumentMatchers.eq(2), any())
+partition.expandIsr(follower3)
+TestUtils.waitUntilTrue(() => !partition.isrState.isInflight, "Expected 
ISR state to be committed", 100)
+
 scheduler.shutdown()
   }
 


[kafka] branch trunk updated (55b24ce -> 9eb9b16)

2021-05-17 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

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


from 55b24ce  MINOR: fix system test TestSecurityRollingUpgrade (#10694)
 add 9eb9b16  KAFKA-12751: Reset AlterIsr in-flight state for duplicate 
update requests (#10633)

No new revisions were added by this update.

Summary of changes:
 core/src/main/scala/kafka/cluster/Partition.scala  | 7 ++-
 core/src/test/scala/unit/kafka/cluster/PartitionTest.scala | 7 +++
 2 files changed, 13 insertions(+), 1 deletion(-)


[kafka] 02/02: KAFKA-12730; Avoid duplicate logout if Kerberos login fails (#10611)

2021-05-05 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

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

commit dbce99258ecf9164180551be85319e9f89bd7c8e
Author: Rajini Sivaram 
AuthorDate: Thu Apr 29 14:32:50 2021 +0100

KAFKA-12730; Avoid duplicate logout if Kerberos login fails (#10611)

From Java 9 onwards, LoginContext#logout() throws an NPE if invoked 
multiple times due to https://bugs.openjdk.java.net/browse/JDK-8173069. 
KerberosLogin currently attempts logout followed by login in a background 
refresh thread. If login fails we retry the same sequence. As a result, a 
single login failure prevents subsequent re-login. And clients will never be 
able to authenticate successfully after the first failure, until the process is 
restarted. The commit checks if logout is ne [...]

Reviewers: Manikumar Reddy 
---
 .../common/security/kerberos/KerberosLogin.java| 14 -
 .../kafka/server/GssapiAuthenticationTest.scala| 64 ++
 2 files changed, 63 insertions(+), 15 deletions(-)

diff --git 
a/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosLogin.java
 
b/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosLogin.java
index f39f35c..a91a964 100644
--- 
a/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosLogin.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosLogin.java
@@ -360,17 +360,25 @@ public class KerberosLogin extends AbstractLogin {
 lastLogin = currentElapsedTime();
 //clear up the kerberos state. But the tokens are not cleared! As 
per
 //the Java kerberos login module code, only the kerberos 
credentials
-//are cleared
-logout();
+//are cleared. If previous logout succeeded but login failed, we 
shouldn't
+//logout again since duplicate logout causes NPE from Java 9 
onwards.
+if (subject != null && !subject.getPrincipals().isEmpty()) {
+logout();
+}
 //login and also update the subject field of this instance to
 //have the new credentials (pass it to the LoginContext 
constructor)
 loginContext = new LoginContext(contextName(), subject, null, 
configuration());
 log.info("Initiating re-login for {}", principal);
-loginContext.login();
+login(loginContext);
 }
 }
 
 // Visibility to override for testing
+protected void login(LoginContext loginContext) throws LoginException {
+loginContext.login();
+}
+
+// Visibility to override for testing
 protected void logout() throws LoginException {
 loginContext.logout();
 }
diff --git 
a/core/src/test/scala/integration/kafka/server/GssapiAuthenticationTest.scala 
b/core/src/test/scala/integration/kafka/server/GssapiAuthenticationTest.scala
index f03b146..da14860 100644
--- 
a/core/src/test/scala/integration/kafka/server/GssapiAuthenticationTest.scala
+++ 
b/core/src/test/scala/integration/kafka/server/GssapiAuthenticationTest.scala
@@ -23,6 +23,7 @@ import java.time.Duration
 import java.util.{Collections, Properties}
 import java.util.concurrent.{CountDownLatch, Executors, TimeUnit}
 
+import javax.security.auth.login.LoginContext
 import kafka.api.{Both, IntegrationTestHarness, SaslSetup}
 import kafka.utils.TestUtils
 import org.apache.kafka.clients.CommonClientConfigs
@@ -100,6 +101,29 @@ class GssapiAuthenticationTest extends 
IntegrationTestHarness with SaslSetup {
   }
 
   /**
+   * Verifies that if login fails, subsequent re-login without failures works 
and clients
+   * are able to connect after the second re-login. Verifies that logout is 
performed only once
+   * since duplicate logouts without successful login results in NPE from Java 
9 onwards.
+   */
+  @Test
+  def testLoginFailure(): Unit = {
+val selector = createSelectorWithRelogin()
+try {
+  val login = TestableKerberosLogin.instance
+  assertNotNull(login)
+  login.loginException = Some(new RuntimeException("Test exception to fail 
login"))
+  executor.submit(() => login.reLogin(), 0)
+  executor.submit(() => login.reLogin(), 0)
+
+  verifyRelogin(selector, login)
+  assertEquals(2, login.loginAttempts)
+  assertEquals(1, login.logoutAttempts)
+} finally {
+  selector.close()
+}
+  }
+
+  /**
* Verifies that there are no authentication failures during Kerberos 
re-login. If authentication
* is performed when credentials are unavailable between logout and login, 
we handle it as a
* transient error and not an authentication failure so that clients may 
retry.
@@ -111,23 +135,26 @@ class GssapiAuthenticationTest extends 
IntegrationTestHarness with SaslSetup {
   val login = TestableKerberosLogin.instance
   assertNot

[kafka] 01/02: KAFKA-10727; Handle Kerberos error during re-login as transient failure in clients (#9605)

2021-05-05 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

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

commit 4a6fada98772ba15a5e760a29b892b88eb213422
Author: Rajini Sivaram 
AuthorDate: Mon Nov 23 09:04:16 2020 +

KAFKA-10727; Handle Kerberos error during re-login as transient failure in 
clients (#9605)

We use a background thread for Kerberos to perform re-login before tickets 
expire. The thread performs logout() followed by login(), relying on the Java 
library to clear and then populate credentials in Subject. This leaves a timing 
window where clients fail to authenticate because credentials are not 
available. We cannot introduce any form of locking since authentication is 
performed on the network thread. So this commit treats NO_CRED as a transient 
failure rather than a fatal authe [...]

Reviewers: Ron Dagostino , Manikumar Reddy 

---
 .../kafka/common/network/SaslChannelBuilder.java   |   2 +-
 .../authenticator/SaslClientAuthenticator.java |   8 +-
 .../common/security/kerberos/KerberosError.java|  19 
 .../common/security/kerberos/KerberosLogin.java|   9 +-
 .../kafka/server/GssapiAuthenticationTest.scala| 107 ++---
 5 files changed, 126 insertions(+), 19 deletions(-)

diff --git 
a/clients/src/main/java/org/apache/kafka/common/network/SaslChannelBuilder.java 
b/clients/src/main/java/org/apache/kafka/common/network/SaslChannelBuilder.java
index e9d5dfe..7c81f145 100644
--- 
a/clients/src/main/java/org/apache/kafka/common/network/SaslChannelBuilder.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/network/SaslChannelBuilder.java
@@ -326,7 +326,7 @@ public class SaslChannelBuilder implements ChannelBuilder, 
ListenerReconfigurabl
 }
 }
 
-private Class defaultLoginClass() {
+protected Class defaultLoginClass() {
 if (jaasContexts.containsKey(SaslConfigs.GSSAPI_MECHANISM))
 return KerberosLogin.class;
 if 
(OAuthBearerLoginModule.OAUTHBEARER_MECHANISM.equals(clientSaslMechanism))
diff --git 
a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientAuthenticator.java
 
b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientAuthenticator.java
index 8b32e81..10b9907 100644
--- 
a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientAuthenticator.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientAuthenticator.java
@@ -529,15 +529,17 @@ public class SaslClientAuthenticator implements 
Authenticator {
 " Users must configure FQDN of kafka brokers when 
authenticating using SASL and" +
 " `socketChannel.socket().getInetAddress().getHostName()` 
must match the hostname in `principal/hostname@realm`";
 }
-error += " Kafka Client will go to AUTHENTICATION_FAILED state.";
 //Unwrap the SaslException inside `PrivilegedActionException`
 Throwable cause = e.getCause();
 // Treat transient Kerberos errors as non-fatal SaslExceptions 
that are processed as I/O exceptions
 // and all other failures as fatal SaslAuthenticationException.
-if (kerberosError != null && kerberosError.retriable())
+if ((kerberosError != null && kerberosError.retriable()) || 
(kerberosError == null && KerberosError.isRetriableClientGssException(e))) {
+error += " Kafka Client will retry.";
 throw new SaslException(error, cause);
-else
+} else {
+error += " Kafka Client will go to AUTHENTICATION_FAILED 
state.";
 throw new SaslAuthenticationException(error, cause);
+}
 }
 }
 
diff --git 
a/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosError.java
 
b/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosError.java
index 9c76482..4b8e8e0 100644
--- 
a/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosError.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosError.java
@@ -19,6 +19,7 @@ package org.apache.kafka.common.security.kerberos;
 import org.apache.kafka.common.KafkaException;
 import org.apache.kafka.common.security.authenticator.SaslClientAuthenticator;
 import org.apache.kafka.common.utils.Java;
+import org.ietf.jgss.GSSException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -109,4 +110,22 @@ public enum KerberosError {
 }
 return null;
 }
+
+/**
+ * Returns true if the exception should be handled as a transient failure 
on clients.
+ * We handle GSSException.NO_CRED as retriable on the client-side since 
this may
+ * occur during re-login if a clients attempt

[kafka] branch 2.5 updated (d5e85ef -> dbce992)

2021-05-05 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

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


from d5e85ef  KAFKA-12474: Handle failure to write new session keys 
gracefully (#10396)
 new 4a6fada  KAFKA-10727; Handle Kerberos error during re-login as 
transient failure in clients (#9605)
 new dbce992  KAFKA-12730; Avoid duplicate logout if Kerberos login fails 
(#10611)

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../kafka/common/network/SaslChannelBuilder.java   |   2 +-
 .../authenticator/SaslClientAuthenticator.java |   8 +-
 .../common/security/kerberos/KerberosError.java|  19 +++
 .../common/security/kerberos/KerberosLogin.java|  21 ++-
 .../kafka/server/GssapiAuthenticationTest.scala| 147 +++--
 5 files changed, 176 insertions(+), 21 deletions(-)


[kafka] 02/02: KAFKA-12730; Avoid duplicate logout if Kerberos login fails (#10611)

2021-05-05 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

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

commit a9dad0fc58da860fb0af1a3ccb902fc3fdaf8c04
Author: Rajini Sivaram 
AuthorDate: Thu Apr 29 14:32:50 2021 +0100

KAFKA-12730; Avoid duplicate logout if Kerberos login fails (#10611)

From Java 9 onwards, LoginContext#logout() throws an NPE if invoked 
multiple times due to https://bugs.openjdk.java.net/browse/JDK-8173069. 
KerberosLogin currently attempts logout followed by login in a background 
refresh thread. If login fails we retry the same sequence. As a result, a 
single login failure prevents subsequent re-login. And clients will never be 
able to authenticate successfully after the first failure, until the process is 
restarted. The commit checks if logout is ne [...]

Reviewers: Manikumar Reddy 
---
 .../common/security/kerberos/KerberosLogin.java| 14 -
 .../kafka/server/GssapiAuthenticationTest.scala| 64 ++
 2 files changed, 63 insertions(+), 15 deletions(-)

diff --git 
a/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosLogin.java
 
b/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosLogin.java
index f39f35c..a91a964 100644
--- 
a/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosLogin.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosLogin.java
@@ -360,17 +360,25 @@ public class KerberosLogin extends AbstractLogin {
 lastLogin = currentElapsedTime();
 //clear up the kerberos state. But the tokens are not cleared! As 
per
 //the Java kerberos login module code, only the kerberos 
credentials
-//are cleared
-logout();
+//are cleared. If previous logout succeeded but login failed, we 
shouldn't
+//logout again since duplicate logout causes NPE from Java 9 
onwards.
+if (subject != null && !subject.getPrincipals().isEmpty()) {
+logout();
+}
 //login and also update the subject field of this instance to
 //have the new credentials (pass it to the LoginContext 
constructor)
 loginContext = new LoginContext(contextName(), subject, null, 
configuration());
 log.info("Initiating re-login for {}", principal);
-loginContext.login();
+login(loginContext);
 }
 }
 
 // Visibility to override for testing
+protected void login(LoginContext loginContext) throws LoginException {
+loginContext.login();
+}
+
+// Visibility to override for testing
 protected void logout() throws LoginException {
 loginContext.logout();
 }
diff --git 
a/core/src/test/scala/integration/kafka/server/GssapiAuthenticationTest.scala 
b/core/src/test/scala/integration/kafka/server/GssapiAuthenticationTest.scala
index fa21a94..efb51f1 100644
--- 
a/core/src/test/scala/integration/kafka/server/GssapiAuthenticationTest.scala
+++ 
b/core/src/test/scala/integration/kafka/server/GssapiAuthenticationTest.scala
@@ -23,6 +23,7 @@ import java.time.Duration
 import java.util.{Collections, Properties}
 import java.util.concurrent.{CountDownLatch, Executors, TimeUnit}
 
+import javax.security.auth.login.LoginContext
 import kafka.api.{Both, IntegrationTestHarness, SaslSetup}
 import kafka.utils.TestUtils
 import org.apache.kafka.clients.CommonClientConfigs
@@ -100,6 +101,29 @@ class GssapiAuthenticationTest extends 
IntegrationTestHarness with SaslSetup {
   }
 
   /**
+   * Verifies that if login fails, subsequent re-login without failures works 
and clients
+   * are able to connect after the second re-login. Verifies that logout is 
performed only once
+   * since duplicate logouts without successful login results in NPE from Java 
9 onwards.
+   */
+  @Test
+  def testLoginFailure(): Unit = {
+val selector = createSelectorWithRelogin()
+try {
+  val login = TestableKerberosLogin.instance
+  assertNotNull(login)
+  login.loginException = Some(new RuntimeException("Test exception to fail 
login"))
+  executor.submit(() => login.reLogin(), 0)
+  executor.submit(() => login.reLogin(), 0)
+
+  verifyRelogin(selector, login)
+  assertEquals(2, login.loginAttempts)
+  assertEquals(1, login.logoutAttempts)
+} finally {
+  selector.close()
+}
+  }
+
+  /**
* Verifies that there are no authentication failures during Kerberos 
re-login. If authentication
* is performed when credentials are unavailable between logout and login, 
we handle it as a
* transient error and not an authentication failure so that clients may 
retry.
@@ -111,23 +135,26 @@ class GssapiAuthenticationTest extends 
IntegrationTestHarness with SaslSetup {
   val login = TestableKerberosLogin.instance
   assertNot

[kafka] 01/02: KAFKA-10727; Handle Kerberos error during re-login as transient failure in clients (#9605)

2021-05-05 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

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

commit ed6bfcdd799617ec036993236135c42d40c9f901
Author: Rajini Sivaram 
AuthorDate: Mon Nov 23 09:04:16 2020 +

KAFKA-10727; Handle Kerberos error during re-login as transient failure in 
clients (#9605)

We use a background thread for Kerberos to perform re-login before tickets 
expire. The thread performs logout() followed by login(), relying on the Java 
library to clear and then populate credentials in Subject. This leaves a timing 
window where clients fail to authenticate because credentials are not 
available. We cannot introduce any form of locking since authentication is 
performed on the network thread. So this commit treats NO_CRED as a transient 
failure rather than a fatal authe [...]

Reviewers: Ron Dagostino , Manikumar Reddy 

---
 .../kafka/common/network/SaslChannelBuilder.java   |   2 +-
 .../authenticator/SaslClientAuthenticator.java |   8 +-
 .../common/security/kerberos/KerberosError.java|  19 
 .../common/security/kerberos/KerberosLogin.java|   9 +-
 .../kafka/server/GssapiAuthenticationTest.scala| 107 ++---
 5 files changed, 126 insertions(+), 19 deletions(-)

diff --git 
a/clients/src/main/java/org/apache/kafka/common/network/SaslChannelBuilder.java 
b/clients/src/main/java/org/apache/kafka/common/network/SaslChannelBuilder.java
index a29148a..f01c4ef 100644
--- 
a/clients/src/main/java/org/apache/kafka/common/network/SaslChannelBuilder.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/network/SaslChannelBuilder.java
@@ -327,7 +327,7 @@ public class SaslChannelBuilder implements ChannelBuilder, 
ListenerReconfigurabl
 }
 }
 
-private Class defaultLoginClass() {
+protected Class defaultLoginClass() {
 if (jaasContexts.containsKey(SaslConfigs.GSSAPI_MECHANISM))
 return KerberosLogin.class;
 if 
(OAuthBearerLoginModule.OAUTHBEARER_MECHANISM.equals(clientSaslMechanism))
diff --git 
a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientAuthenticator.java
 
b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientAuthenticator.java
index 8b32e81..10b9907 100644
--- 
a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientAuthenticator.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientAuthenticator.java
@@ -529,15 +529,17 @@ public class SaslClientAuthenticator implements 
Authenticator {
 " Users must configure FQDN of kafka brokers when 
authenticating using SASL and" +
 " `socketChannel.socket().getInetAddress().getHostName()` 
must match the hostname in `principal/hostname@realm`";
 }
-error += " Kafka Client will go to AUTHENTICATION_FAILED state.";
 //Unwrap the SaslException inside `PrivilegedActionException`
 Throwable cause = e.getCause();
 // Treat transient Kerberos errors as non-fatal SaslExceptions 
that are processed as I/O exceptions
 // and all other failures as fatal SaslAuthenticationException.
-if (kerberosError != null && kerberosError.retriable())
+if ((kerberosError != null && kerberosError.retriable()) || 
(kerberosError == null && KerberosError.isRetriableClientGssException(e))) {
+error += " Kafka Client will retry.";
 throw new SaslException(error, cause);
-else
+} else {
+error += " Kafka Client will go to AUTHENTICATION_FAILED 
state.";
 throw new SaslAuthenticationException(error, cause);
+}
 }
 }
 
diff --git 
a/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosError.java
 
b/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosError.java
index 9c76482..4b8e8e0 100644
--- 
a/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosError.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosError.java
@@ -19,6 +19,7 @@ package org.apache.kafka.common.security.kerberos;
 import org.apache.kafka.common.KafkaException;
 import org.apache.kafka.common.security.authenticator.SaslClientAuthenticator;
 import org.apache.kafka.common.utils.Java;
+import org.ietf.jgss.GSSException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -109,4 +110,22 @@ public enum KerberosError {
 }
 return null;
 }
+
+/**
+ * Returns true if the exception should be handled as a transient failure 
on clients.
+ * We handle GSSException.NO_CRED as retriable on the client-side since 
this may
+ * occur during re-login if a clients attempt

[kafka] branch 2.6 updated (f0690ed -> a9dad0f)

2021-05-05 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

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


from f0690ed  Bump to 2.6.3-SNAPSHOT in missing files (#10607)
 new ed6bfcd  KAFKA-10727; Handle Kerberos error during re-login as 
transient failure in clients (#9605)
 new a9dad0f  KAFKA-12730; Avoid duplicate logout if Kerberos login fails 
(#10611)

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../kafka/common/network/SaslChannelBuilder.java   |   2 +-
 .../authenticator/SaslClientAuthenticator.java |   8 +-
 .../common/security/kerberos/KerberosError.java|  19 +++
 .../common/security/kerberos/KerberosLogin.java|  21 ++-
 .../kafka/server/GssapiAuthenticationTest.scala| 147 +++--
 5 files changed, 176 insertions(+), 21 deletions(-)


[kafka] 02/02: KAFKA-12730; Avoid duplicate logout if Kerberos login fails (#10611)

2021-05-05 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

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

commit adaeb0f6469207e7e2eb44bdc14434cd4d2ee6a7
Author: Rajini Sivaram 
AuthorDate: Thu Apr 29 14:32:50 2021 +0100

KAFKA-12730; Avoid duplicate logout if Kerberos login fails (#10611)

From Java 9 onwards, LoginContext#logout() throws an NPE if invoked 
multiple times due to https://bugs.openjdk.java.net/browse/JDK-8173069. 
KerberosLogin currently attempts logout followed by login in a background 
refresh thread. If login fails we retry the same sequence. As a result, a 
single login failure prevents subsequent re-login. And clients will never be 
able to authenticate successfully after the first failure, until the process is 
restarted. The commit checks if logout is ne [...]

Reviewers: Manikumar Reddy 
---
 .../common/security/kerberos/KerberosLogin.java| 14 -
 .../kafka/server/GssapiAuthenticationTest.scala| 64 ++
 2 files changed, 63 insertions(+), 15 deletions(-)

diff --git 
a/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosLogin.java
 
b/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosLogin.java
index 9626da8..3f0a46e 100644
--- 
a/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosLogin.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosLogin.java
@@ -362,17 +362,25 @@ public class KerberosLogin extends AbstractLogin {
 lastLogin = currentElapsedTime();
 //clear up the kerberos state. But the tokens are not cleared! As 
per
 //the Java kerberos login module code, only the kerberos 
credentials
-//are cleared
-logout();
+//are cleared. If previous logout succeeded but login failed, we 
shouldn't
+//logout again since duplicate logout causes NPE from Java 9 
onwards.
+if (subject != null && !subject.getPrincipals().isEmpty()) {
+logout();
+}
 //login and also update the subject field of this instance to
 //have the new credentials (pass it to the LoginContext 
constructor)
 loginContext = new LoginContext(contextName(), subject, null, 
configuration());
 log.info("Initiating re-login for {}", principal);
-loginContext.login();
+login(loginContext);
 }
 }
 
 // Visibility to override for testing
+protected void login(LoginContext loginContext) throws LoginException {
+loginContext.login();
+}
+
+// Visibility to override for testing
 protected void logout() throws LoginException {
 loginContext.logout();
 }
diff --git 
a/core/src/test/scala/integration/kafka/server/GssapiAuthenticationTest.scala 
b/core/src/test/scala/integration/kafka/server/GssapiAuthenticationTest.scala
index fa21a94..efb51f1 100644
--- 
a/core/src/test/scala/integration/kafka/server/GssapiAuthenticationTest.scala
+++ 
b/core/src/test/scala/integration/kafka/server/GssapiAuthenticationTest.scala
@@ -23,6 +23,7 @@ import java.time.Duration
 import java.util.{Collections, Properties}
 import java.util.concurrent.{CountDownLatch, Executors, TimeUnit}
 
+import javax.security.auth.login.LoginContext
 import kafka.api.{Both, IntegrationTestHarness, SaslSetup}
 import kafka.utils.TestUtils
 import org.apache.kafka.clients.CommonClientConfigs
@@ -100,6 +101,29 @@ class GssapiAuthenticationTest extends 
IntegrationTestHarness with SaslSetup {
   }
 
   /**
+   * Verifies that if login fails, subsequent re-login without failures works 
and clients
+   * are able to connect after the second re-login. Verifies that logout is 
performed only once
+   * since duplicate logouts without successful login results in NPE from Java 
9 onwards.
+   */
+  @Test
+  def testLoginFailure(): Unit = {
+val selector = createSelectorWithRelogin()
+try {
+  val login = TestableKerberosLogin.instance
+  assertNotNull(login)
+  login.loginException = Some(new RuntimeException("Test exception to fail 
login"))
+  executor.submit(() => login.reLogin(), 0)
+  executor.submit(() => login.reLogin(), 0)
+
+  verifyRelogin(selector, login)
+  assertEquals(2, login.loginAttempts)
+  assertEquals(1, login.logoutAttempts)
+} finally {
+  selector.close()
+}
+  }
+
+  /**
* Verifies that there are no authentication failures during Kerberos 
re-login. If authentication
* is performed when credentials are unavailable between logout and login, 
we handle it as a
* transient error and not an authentication failure so that clients may 
retry.
@@ -111,23 +135,26 @@ class GssapiAuthenticationTest extends 
IntegrationTestHarness with SaslSetup {
   val login = TestableKerberosLogin.instance
   assertNot

[kafka] 01/02: KAFKA-10727; Handle Kerberos error during re-login as transient failure in clients (#9605)

2021-05-05 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

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

commit 6e230782c54a47231d38e3643323d6d5d2cad311
Author: Rajini Sivaram 
AuthorDate: Mon Nov 23 09:04:16 2020 +

KAFKA-10727; Handle Kerberos error during re-login as transient failure in 
clients (#9605)

We use a background thread for Kerberos to perform re-login before tickets 
expire. The thread performs logout() followed by login(), relying on the Java 
library to clear and then populate credentials in Subject. This leaves a timing 
window where clients fail to authenticate because credentials are not 
available. We cannot introduce any form of locking since authentication is 
performed on the network thread. So this commit treats NO_CRED as a transient 
failure rather than a fatal authe [...]

Reviewers: Ron Dagostino , Manikumar Reddy 

---
 .../kafka/common/network/SaslChannelBuilder.java   |   2 +-
 .../authenticator/SaslClientAuthenticator.java |   8 +-
 .../common/security/kerberos/KerberosError.java|  19 
 .../common/security/kerberos/KerberosLogin.java|   9 +-
 .../kafka/server/GssapiAuthenticationTest.scala| 107 ++---
 5 files changed, 126 insertions(+), 19 deletions(-)

diff --git 
a/clients/src/main/java/org/apache/kafka/common/network/SaslChannelBuilder.java 
b/clients/src/main/java/org/apache/kafka/common/network/SaslChannelBuilder.java
index a29148a..f01c4ef 100644
--- 
a/clients/src/main/java/org/apache/kafka/common/network/SaslChannelBuilder.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/network/SaslChannelBuilder.java
@@ -327,7 +327,7 @@ public class SaslChannelBuilder implements ChannelBuilder, 
ListenerReconfigurabl
 }
 }
 
-private Class defaultLoginClass() {
+protected Class defaultLoginClass() {
 if (jaasContexts.containsKey(SaslConfigs.GSSAPI_MECHANISM))
 return KerberosLogin.class;
 if 
(OAuthBearerLoginModule.OAUTHBEARER_MECHANISM.equals(clientSaslMechanism))
diff --git 
a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientAuthenticator.java
 
b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientAuthenticator.java
index bba1c43..fb30c87 100644
--- 
a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientAuthenticator.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientAuthenticator.java
@@ -533,15 +533,17 @@ public class SaslClientAuthenticator implements 
Authenticator {
 " Users must configure FQDN of kafka brokers when 
authenticating using SASL and" +
 " `socketChannel.socket().getInetAddress().getHostName()` 
must match the hostname in `principal/hostname@realm`";
 }
-error += " Kafka Client will go to AUTHENTICATION_FAILED state.";
 //Unwrap the SaslException inside `PrivilegedActionException`
 Throwable cause = e.getCause();
 // Treat transient Kerberos errors as non-fatal SaslExceptions 
that are processed as I/O exceptions
 // and all other failures as fatal SaslAuthenticationException.
-if (kerberosError != null && kerberosError.retriable())
+if ((kerberosError != null && kerberosError.retriable()) || 
(kerberosError == null && KerberosError.isRetriableClientGssException(e))) {
+error += " Kafka Client will retry.";
 throw new SaslException(error, cause);
-else
+} else {
+error += " Kafka Client will go to AUTHENTICATION_FAILED 
state.";
 throw new SaslAuthenticationException(error, cause);
+}
 }
 }
 
diff --git 
a/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosError.java
 
b/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosError.java
index 9c76482..4b8e8e0 100644
--- 
a/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosError.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosError.java
@@ -19,6 +19,7 @@ package org.apache.kafka.common.security.kerberos;
 import org.apache.kafka.common.KafkaException;
 import org.apache.kafka.common.security.authenticator.SaslClientAuthenticator;
 import org.apache.kafka.common.utils.Java;
+import org.ietf.jgss.GSSException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -109,4 +110,22 @@ public enum KerberosError {
 }
 return null;
 }
+
+/**
+ * Returns true if the exception should be handled as a transient failure 
on clients.
+ * We handle GSSException.NO_CRED as retriable on the client-side since 
this may
+ * occur during re-login if a clients attempt

[kafka] branch 2.7 updated (f98db1e -> adaeb0f)

2021-05-05 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

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


from f98db1e  MINOR: Bump latest 2.6 version to 2.6.2 (#10582)
 new 6e23078  KAFKA-10727; Handle Kerberos error during re-login as 
transient failure in clients (#9605)
 new adaeb0f  KAFKA-12730; Avoid duplicate logout if Kerberos login fails 
(#10611)

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../kafka/common/network/SaslChannelBuilder.java   |   2 +-
 .../authenticator/SaslClientAuthenticator.java |   8 +-
 .../common/security/kerberos/KerberosError.java|  19 +++
 .../common/security/kerberos/KerberosLogin.java|  21 ++-
 .../kafka/server/GssapiAuthenticationTest.scala| 147 +++--
 5 files changed, 176 insertions(+), 21 deletions(-)


[kafka] branch 2.8 updated: KAFKA-12730; Avoid duplicate logout if Kerberos login fails (#10611)

2021-05-05 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/2.8 by this push:
 new f732aaa  KAFKA-12730; Avoid duplicate logout if Kerberos login fails 
(#10611)
f732aaa is described below

commit f732aaae7fd4a1f43731184b9c3c1a2e433d393a
Author: Rajini Sivaram 
AuthorDate: Thu Apr 29 14:32:50 2021 +0100

KAFKA-12730; Avoid duplicate logout if Kerberos login fails (#10611)

From Java 9 onwards, LoginContext#logout() throws an NPE if invoked 
multiple times due to https://bugs.openjdk.java.net/browse/JDK-8173069. 
KerberosLogin currently attempts logout followed by login in a background 
refresh thread. If login fails we retry the same sequence. As a result, a 
single login failure prevents subsequent re-login. And clients will never be 
able to authenticate successfully after the first failure, until the process is 
restarted. The commit checks if logout is ne [...]

Reviewers: Manikumar Reddy 
---
 .../common/security/kerberos/KerberosLogin.java| 14 -
 .../kafka/server/GssapiAuthenticationTest.scala| 64 ++
 2 files changed, 63 insertions(+), 15 deletions(-)

diff --git 
a/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosLogin.java
 
b/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosLogin.java
index 9626da8..3f0a46e 100644
--- 
a/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosLogin.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosLogin.java
@@ -362,17 +362,25 @@ public class KerberosLogin extends AbstractLogin {
 lastLogin = currentElapsedTime();
 //clear up the kerberos state. But the tokens are not cleared! As 
per
 //the Java kerberos login module code, only the kerberos 
credentials
-//are cleared
-logout();
+//are cleared. If previous logout succeeded but login failed, we 
shouldn't
+//logout again since duplicate logout causes NPE from Java 9 
onwards.
+if (subject != null && !subject.getPrincipals().isEmpty()) {
+logout();
+}
 //login and also update the subject field of this instance to
 //have the new credentials (pass it to the LoginContext 
constructor)
 loginContext = new LoginContext(contextName(), subject, null, 
configuration());
 log.info("Initiating re-login for {}", principal);
-loginContext.login();
+login(loginContext);
 }
 }
 
 // Visibility to override for testing
+protected void login(LoginContext loginContext) throws LoginException {
+loginContext.login();
+}
+
+// Visibility to override for testing
 protected void logout() throws LoginException {
 loginContext.logout();
 }
diff --git 
a/core/src/test/scala/integration/kafka/server/GssapiAuthenticationTest.scala 
b/core/src/test/scala/integration/kafka/server/GssapiAuthenticationTest.scala
index 90454bb..5c75507 100644
--- 
a/core/src/test/scala/integration/kafka/server/GssapiAuthenticationTest.scala
+++ 
b/core/src/test/scala/integration/kafka/server/GssapiAuthenticationTest.scala
@@ -23,6 +23,7 @@ import java.time.Duration
 import java.util.{Collections, Properties}
 import java.util.concurrent.{CountDownLatch, Executors, TimeUnit}
 
+import javax.security.auth.login.LoginContext
 import kafka.api.{Both, IntegrationTestHarness, SaslSetup}
 import kafka.utils.TestUtils
 import org.apache.kafka.clients.CommonClientConfigs
@@ -102,6 +103,29 @@ class GssapiAuthenticationTest extends 
IntegrationTestHarness with SaslSetup {
   }
 
   /**
+   * Verifies that if login fails, subsequent re-login without failures works 
and clients
+   * are able to connect after the second re-login. Verifies that logout is 
performed only once
+   * since duplicate logouts without successful login results in NPE from Java 
9 onwards.
+   */
+  @Test
+  def testLoginFailure(): Unit = {
+val selector = createSelectorWithRelogin()
+try {
+  val login = TestableKerberosLogin.instance
+  assertNotNull(login)
+  login.loginException = Some(new RuntimeException("Test exception to fail 
login"))
+  executor.submit(() => login.reLogin(), 0)
+  executor.submit(() => login.reLogin(), 0)
+
+  verifyRelogin(selector, login)
+  assertEquals(2, login.loginAttempts)
+  assertEquals(1, login.logoutAttempts)
+} finally {
+  selector.close()
+}
+  }
+
+  /**
* Verifies that there are no authentication failures during Kerberos 
re-login. If authentication
* is performed when credentials are unavailable between logout and login, 
we handle it as a
* transient error and not an authentication failure so that clients may 

[kafka] branch trunk updated (f9de25f -> eaa14a8)

2021-04-29 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

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


from f9de25f  MINOR: rename wrong topic id variable name and description 
(#10598)
 add eaa14a8  KAFKA-12730; Avoid duplicate logout if Kerberos login fails 
(#10611)

No new revisions were added by this update.

Summary of changes:
 .../common/security/kerberos/KerberosLogin.java| 14 -
 .../kafka/server/GssapiAuthenticationTest.scala| 64 ++
 2 files changed, 63 insertions(+), 15 deletions(-)


[kafka] branch trunk updated: KAFKA-12479: Batch partition offset requests in ConsumerGroupCommand (#10371)

2021-03-23 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

rsivaram 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 e3f60c2  KAFKA-12479: Batch partition offset requests in 
ConsumerGroupCommand (#10371)
e3f60c2 is described below

commit e3f60c254c66d7021d3e0b61968a59a70e00cb39
Author: Rajini Sivaram 
AuthorDate: Tue Mar 23 09:56:56 2021 +

KAFKA-12479: Batch partition offset requests in ConsumerGroupCommand 
(#10371)

Reviewers: David Jacot , Chia-Ping Tsai 
, Ismael Juma 
---
 .../kafka/clients/admin/AdminClientTestUtils.java  |  13 ++
 .../scala/kafka/admin/ConsumerGroupCommand.scala   |  65 ++
 .../kafka/admin/ConsumerGroupServiceTest.scala | 133 +
 3 files changed, 186 insertions(+), 25 deletions(-)

diff --git 
a/clients/src/test/java/org/apache/kafka/clients/admin/AdminClientTestUtils.java
 
b/clients/src/test/java/org/apache/kafka/clients/admin/AdminClientTestUtils.java
index b0f0055..a64dab6 100644
--- 
a/clients/src/test/java/org/apache/kafka/clients/admin/AdminClientTestUtils.java
+++ 
b/clients/src/test/java/org/apache/kafka/clients/admin/AdminClientTestUtils.java
@@ -18,7 +18,11 @@ package org.apache.kafka.clients.admin;
 
 import java.util.Collections;
 import java.util.Map;
+import java.util.stream.Collectors;
+
 import 
org.apache.kafka.clients.admin.CreateTopicsResult.TopicMetadataAndConfig;
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.common.KafkaFuture;
 import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.internals.KafkaFutureImpl;
 
@@ -89,4 +93,13 @@ public class AdminClientTestUtils {
 future.complete(description);
 return new DescribeTopicsResult(Collections.singletonMap(topic, 
future));
 }
+
+public static DescribeTopicsResult describeTopicsResult(Map topicDescriptions) {
+return new DescribeTopicsResult(topicDescriptions.entrySet().stream()
+.collect(Collectors.toMap(Map.Entry::getKey, e -> 
KafkaFuture.completedFuture(e.getValue();
+}
+
+public static ListConsumerGroupOffsetsResult 
listConsumerGroupOffsetsResult(Map offsets) {
+return new 
ListConsumerGroupOffsetsResult(KafkaFuture.completedFuture(offsets));
+}
 }
diff --git a/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala 
b/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala
index e948f6f..2c0dc8c 100755
--- a/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala
+++ b/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala
@@ -459,13 +459,7 @@ object ConsumerGroupCommand extends Logging {
   val partitionLevelResult = mutable.Map[TopicPartition, Throwable]()
 
   val (topicWithPartitions, topicWithoutPartitions) = 
topics.partition(_.contains(":"))
-
-  val knownPartitions = topicWithPartitions.flatMap { topicArg =>
-val split = topicArg.split(":")
-split(1).split(",").map { partition =>
-  new TopicPartition(split(0), partition.toInt)
-}
-  }
+  val knownPartitions = 
topicWithPartitions.flatMap(parseTopicsWithPartitions)
 
   // Get the partitions of topics that the user did not explicitly specify 
the partitions
   val describeTopicsResult = adminClient.describeTopics(
@@ -580,18 +574,20 @@ object ConsumerGroupCommand extends Logging {
 partitionOffsets, Some(s"${consumerSummary.consumerId}"), 
Some(s"${consumerSummary.host}"),
 Some(s"${consumerSummary.clientId}"))
 }
-val rowsWithoutConsumer = committedOffsets.filter { case (tp, _) =>
-  !assignedTopicPartitions.contains(tp)
-}.flatMap { case (topicPartition, offset) =>
+
+val unassignedPartitions = committedOffsets.filterNot { case (tp, _) 
=> assignedTopicPartitions.contains(tp) }
+val rowsWithoutConsumer = if (unassignedPartitions.nonEmpty) {
   collectConsumerAssignment(
 groupId,
 Option(consumerGroup.coordinator),
-Seq(topicPartition),
-Map(topicPartition -> Some(offset.offset)),
+unassignedPartitions.keySet.toSeq,
+unassignedPartitions.map { case (tp, offset) => tp -> 
Some(offset.offset) },
 Some(MISSING_COLUMN_VALUE),
 Some(MISSING_COLUMN_VALUE),
 Some(MISSING_COLUMN_VALUE)).toSeq
-}
+} else
+  Seq.empty
+
 groupId -> (Some(state.toString), Some(rowsWithConsumer ++ 
rowsWithoutConsumer))
   }).toMap
 
@@ -696,7 +692,8 @@ object ConsumerGroupCommand extends Logging {
   adminClient.close()
 }
 
-private def createAdminClient(configOverrides: Map[String, String]): Admin 
= {
+// Visibility for testing
+protected def createAdminClient

[kafka] branch trunk updated (bf63990 -> d80a87f)

2021-03-16 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

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


from bf63990  KAFKA-12427: Don't update connection idle time for muted 
connections (#10267)
 add d80a87f  KAFKA-12330; FetchSessionCache may cause starvation for 
partitions when FetchResponse is full (#10318)

No new revisions were added by this update.

Summary of changes:
 .../src/main/scala/kafka/server/FetchSession.scala |  2 +-
 .../scala/unit/kafka/server/FetchSessionTest.scala | 98 +-
 2 files changed, 98 insertions(+), 2 deletions(-)



[kafka] branch trunk updated (eee0045 -> bf63990)

2021-03-16 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

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


from eee0045  MINOR; Various code cleanups (#10319)
 add bf63990  KAFKA-12427: Don't update connection idle time for muted 
connections (#10267)

No new revisions were added by this update.

Summary of changes:
 .../org/apache/kafka/common/network/Selector.java  |  6 +++-
 .../unit/kafka/network/SocketServerTest.scala  | 34 --
 2 files changed, 37 insertions(+), 3 deletions(-)



[kafka] branch 2.8 updated: KAFKA-12254: Ensure MM2 creates topics with source topic configs (#10217)

2021-03-02 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/2.8 by this push:
 new 169ec72  KAFKA-12254: Ensure MM2 creates topics with source topic 
configs (#10217)
169ec72 is described below

commit 169ec72c782644806a118ae53b864728842c5b57
Author: Dhruvil Shah 
AuthorDate: Mon Mar 1 01:30:30 2021 -0800

KAFKA-12254: Ensure MM2 creates topics with source topic configs (#10217)

MM2 creates new topics on the destination cluster with default 
configurations. It has an async periodic task to refresh topic configurations 
from the source to destination. However, this opens up a window where the 
destination cluster has data produced to it with default configurations. In the 
worst case, this could cause data loss if the destination topic is created 
without the right cleanup.policy. This commit fixes the above issue by ensuring 
that the right configurations are suppl [...]

Reviewers: Rajini Sivaram 
---
 .../connect/mirror/MirrorSourceConnector.java  | 90 +-
 .../connect/mirror/MirrorSourceConnectorTest.java  | 31 ++--
 2 files changed, 92 insertions(+), 29 deletions(-)

diff --git 
a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceConnector.java
 
b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceConnector.java
index 3e7f0c7..7b844c8 100644
--- 
a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceConnector.java
+++ 
b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceConnector.java
@@ -49,6 +49,7 @@ import java.util.Set;
 import java.util.HashSet;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.function.Function;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
 import java.util.concurrent.ExecutionException;
@@ -306,40 +307,82 @@ public class MirrorSourceConnector extends 
SourceConnector {
 
MirrorUtils.createSinglePartitionCompactedTopic(config.offsetSyncsTopic(), 
config.offsetSyncsTopicReplicationFactor(), config.sourceAdminConfig());
 }
 
-// visible for testing
-void computeAndCreateTopicPartitions()
-throws InterruptedException, ExecutionException {
-Map partitionCounts = knownSourceTopicPartitions.stream()
-.collect(Collectors.groupingBy(TopicPartition::topic, 
Collectors.counting())).entrySet().stream()
-.collect(Collectors.toMap(x -> formatRemoteTopic(x.getKey()), 
Entry::getValue));
-Set knownTargetTopics = toTopics(knownTargetTopicPartitions);
-List newTopics = partitionCounts.entrySet().stream()
-.filter(x -> !knownTargetTopics.contains(x.getKey()))
-.map(x -> new NewTopic(x.getKey(), x.getValue().intValue(), 
(short) replicationFactor))
-.collect(Collectors.toList());
-Map newPartitions = 
partitionCounts.entrySet().stream()
-.filter(x -> knownTargetTopics.contains(x.getKey()))
-.collect(Collectors.toMap(Entry::getKey, x -> 
NewPartitions.increaseTo(x.getValue().intValue(;
-createTopicPartitions(partitionCounts, newTopics, newPartitions);
+void computeAndCreateTopicPartitions() throws ExecutionException, 
InterruptedException {
+// get source and target topics with respective partition counts
+Map sourceTopicToPartitionCounts = 
knownSourceTopicPartitions.stream()
+.collect(Collectors.groupingBy(TopicPartition::topic, 
Collectors.counting())).entrySet().stream()
+.collect(Collectors.toMap(Entry::getKey, Entry::getValue));
+Map targetTopicToPartitionCounts = 
knownTargetTopicPartitions.stream()
+.collect(Collectors.groupingBy(TopicPartition::topic, 
Collectors.counting())).entrySet().stream()
+.collect(Collectors.toMap(Entry::getKey, Entry::getValue));
+
+Set knownSourceTopics = sourceTopicToPartitionCounts.keySet();
+Set knownTargetTopics = targetTopicToPartitionCounts.keySet();
+Map sourceToRemoteTopics = knownSourceTopics.stream()
+.collect(Collectors.toMap(Function.identity(), sourceTopic -> 
formatRemoteTopic(sourceTopic)));
+
+// compute existing and new source topics
+Map> partitionedSourceTopics = 
knownSourceTopics.stream()
+.collect(Collectors.partitioningBy(sourceTopic -> 
knownTargetTopics.contains(sourceToRemoteTopics.get(sourceTopic)),
+Collectors.toSet()));
+Set existingSourceTopics = partitionedSourceTopics.get(true);
+Set newSourceTopics = partitionedSourceTopics.get(false);
+
+// create new topics
+if (!newSourceTopics.isEmpty())
+createNewTopics(newSourceTopics, sourceTopicToPartitionCounts);
+
+//

[kafka] branch trunk updated: KAFKA-12254: Ensure MM2 creates topics with source topic configs (#10217)

2021-03-01 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

rsivaram 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 cc088c5  KAFKA-12254: Ensure MM2 creates topics with source topic 
configs (#10217)
cc088c5 is described below

commit cc088c5abe367989e833e444171dd546370c356f
Author: Dhruvil Shah 
AuthorDate: Mon Mar 1 01:30:30 2021 -0800

KAFKA-12254: Ensure MM2 creates topics with source topic configs (#10217)

MM2 creates new topics on the destination cluster with default 
configurations. It has an async periodic task to refresh topic configurations 
from the source to destination. However, this opens up a window where the 
destination cluster has data produced to it with default configurations. In the 
worst case, this could cause data loss if the destination topic is created 
without the right cleanup.policy. This commit fixes the above issue by ensuring 
that the right configurations are suppl [...]

Reviewers: Rajini Sivaram 
---
 .../connect/mirror/MirrorSourceConnector.java  | 90 +-
 .../connect/mirror/MirrorSourceConnectorTest.java  | 31 ++--
 2 files changed, 92 insertions(+), 29 deletions(-)

diff --git 
a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceConnector.java
 
b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceConnector.java
index 3e7f0c7..7b844c8 100644
--- 
a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceConnector.java
+++ 
b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceConnector.java
@@ -49,6 +49,7 @@ import java.util.Set;
 import java.util.HashSet;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.function.Function;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
 import java.util.concurrent.ExecutionException;
@@ -306,40 +307,82 @@ public class MirrorSourceConnector extends 
SourceConnector {
 
MirrorUtils.createSinglePartitionCompactedTopic(config.offsetSyncsTopic(), 
config.offsetSyncsTopicReplicationFactor(), config.sourceAdminConfig());
 }
 
-// visible for testing
-void computeAndCreateTopicPartitions()
-throws InterruptedException, ExecutionException {
-Map partitionCounts = knownSourceTopicPartitions.stream()
-.collect(Collectors.groupingBy(TopicPartition::topic, 
Collectors.counting())).entrySet().stream()
-.collect(Collectors.toMap(x -> formatRemoteTopic(x.getKey()), 
Entry::getValue));
-Set knownTargetTopics = toTopics(knownTargetTopicPartitions);
-List newTopics = partitionCounts.entrySet().stream()
-.filter(x -> !knownTargetTopics.contains(x.getKey()))
-.map(x -> new NewTopic(x.getKey(), x.getValue().intValue(), 
(short) replicationFactor))
-.collect(Collectors.toList());
-Map newPartitions = 
partitionCounts.entrySet().stream()
-.filter(x -> knownTargetTopics.contains(x.getKey()))
-.collect(Collectors.toMap(Entry::getKey, x -> 
NewPartitions.increaseTo(x.getValue().intValue(;
-createTopicPartitions(partitionCounts, newTopics, newPartitions);
+void computeAndCreateTopicPartitions() throws ExecutionException, 
InterruptedException {
+// get source and target topics with respective partition counts
+Map sourceTopicToPartitionCounts = 
knownSourceTopicPartitions.stream()
+.collect(Collectors.groupingBy(TopicPartition::topic, 
Collectors.counting())).entrySet().stream()
+.collect(Collectors.toMap(Entry::getKey, Entry::getValue));
+Map targetTopicToPartitionCounts = 
knownTargetTopicPartitions.stream()
+.collect(Collectors.groupingBy(TopicPartition::topic, 
Collectors.counting())).entrySet().stream()
+.collect(Collectors.toMap(Entry::getKey, Entry::getValue));
+
+Set knownSourceTopics = sourceTopicToPartitionCounts.keySet();
+Set knownTargetTopics = targetTopicToPartitionCounts.keySet();
+Map sourceToRemoteTopics = knownSourceTopics.stream()
+.collect(Collectors.toMap(Function.identity(), sourceTopic -> 
formatRemoteTopic(sourceTopic)));
+
+// compute existing and new source topics
+Map> partitionedSourceTopics = 
knownSourceTopics.stream()
+.collect(Collectors.partitioningBy(sourceTopic -> 
knownTargetTopics.contains(sourceToRemoteTopics.get(sourceTopic)),
+Collectors.toSet()));
+Set existingSourceTopics = partitionedSourceTopics.get(true);
+Set newSourceTopics = partitionedSourceTopics.get(false);
+
+// create new topics
+if (!newSourceTopics.isEmpty())
+createNewTopics(newSourceTopics, sourceTopicToPartitionCounts);
+
+//

[kafka] branch 2.7 updated: KAFKA-10710; MM2 - Create herders only if source->target.enabled=true and heartbeats are disabled (#9589)

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

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


The following commit(s) were added to refs/heads/2.7 by this push:
 new 1c7c93b  KAFKA-10710; MM2 - Create herders only if 
source->target.enabled=true and heartbeats are disabled (#9589)
1c7c93b is described below

commit 1c7c93ba37308f4c3c3282b455b5263c0e455f09
Author: Julien Chanaud 
AuthorDate: Thu Jan 28 23:52:51 2021 +0100

KAFKA-10710; MM2 - Create herders only if source->target.enabled=true and 
heartbeats are disabled (#9589)

By default Mirror Maker 2 creates herders for all the possible combinations 
even if the "links" are not enabled.

This is because the beats are emitted from the "opposite" herder.
If there is a replication flow from A to B and heartbeats are required, 2 
herders are needed :

- A->B for the MirrorSourceConnector
- B->A for the MirrorHeartbeatConnector

The MirrorHeartbeatConnector on B->A emits beats into topic heartbeats on 
cluster A.
The MirrorSourceConnector on A->B then replicates whichever topic is 
configured as well as heartbeats.

In cases with multiple clusters (10 and more), this leads to an incredible 
amount of connections, file descriptors and configuration topics created in 
every target clusters that are not necessary.

With this code change, we will leverage the top level property 
"emit.heartbeats.enabled" which defaults to "true".
We skip creating the A->B herder whenever 
A->B.emit.heartbeats.enabled=false (defaults to true) and A->B.enabled=false 
(defaults to false).

Existing users will not see any change and if they depend on these 
"opposites" herders for their monitoring, it will still work.
New users with more complex use case can change this property and fine tune 
their heartbeat generation.

Reviewers: Ryanne Dolan ,  Sanjana Kaundinya 
, Jason Gustafson 
---
 .../kafka/connect/mirror/MirrorMakerConfig.java| 23 ++-
 .../connect/mirror/MirrorMakerConfigTest.java  | 75 ++
 2 files changed, 96 insertions(+), 2 deletions(-)

diff --git 
a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorMakerConfig.java
 
b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorMakerConfig.java
index 059ab78..b5c361c 100644
--- 
a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorMakerConfig.java
+++ 
b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorMakerConfig.java
@@ -87,11 +87,30 @@ public class MirrorMakerConfig extends AbstractConfig {
 public List clusterPairs() {
 List pairs = new ArrayList<>();
 Set clusters = clusters();
+Map originalStrings = originalsStrings();
+boolean globalHeartbeatsEnabled = 
MirrorConnectorConfig.EMIT_HEARTBEATS_ENABLED_DEFAULT;
+if 
(originalStrings.containsKey(MirrorConnectorConfig.EMIT_HEARTBEATS_ENABLED)) {
+globalHeartbeatsEnabled = 
Boolean.valueOf(originalStrings.get(MirrorConnectorConfig.EMIT_HEARTBEATS_ENABLED));
+}
+
 for (String source : clusters) {
 for (String target : clusters) {
-SourceAndTarget sourceAndTarget = new SourceAndTarget(source, 
target);
 if (!source.equals(target)) {
-pairs.add(sourceAndTarget);
+String clusterPairConfigPrefix = source + "->" + target + 
".";
+boolean clusterPairEnabled = 
Boolean.valueOf(originalStrings.getOrDefault(clusterPairConfigPrefix + 
"enabled", "false"));
+boolean clusterPairHeartbeatsEnabled = 
globalHeartbeatsEnabled;
+if (originalStrings.containsKey(clusterPairConfigPrefix + 
MirrorConnectorConfig.EMIT_HEARTBEATS_ENABLED)) {
+clusterPairHeartbeatsEnabled = 
Boolean.valueOf(originalStrings.get(clusterPairConfigPrefix + 
MirrorConnectorConfig.EMIT_HEARTBEATS_ENABLED));
+}
+
+// By default, all source->target Herder combinations are 
created even if `x->y.enabled=false`
+// Unless `emit.heartbeats.enabled=false` or 
`x->y.emit.heartbeats.enabled=false`
+// Reason for this behavior: for a given replication flow 
A->B with heartbeats, 2 herders are required :
+// B->A for the MirrorHeartbeatConnector (emits heartbeats 
into A for monitoring replication health)
+// A->B for the MirrorSourceConnector (actual replication 
flow)
+if (clusterPairEnabled || clusterPairHeartbeatsEnabled) {
+pairs.add(new SourceAndTarget(source, target));
+}
 }
   

[kafka] branch 2.8 updated: MINOR: Remove unused LeaderAndIsrResponse.partitions() since it has been replaced with partitionErrors() (#10127)

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

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


The following commit(s) were added to refs/heads/2.8 by this push:
 new 1392382  MINOR: Remove unused LeaderAndIsrResponse.partitions() since 
it has been replaced with partitionErrors() (#10127)
1392382 is described below

commit 1392382a54f0b4f134d3dd7134628825b831bad1
Author: Rajini Sivaram 
AuthorDate: Mon Feb 15 12:21:01 2021 +

MINOR: Remove unused LeaderAndIsrResponse.partitions() since it has been 
replaced with partitionErrors() (#10127)

Reviewers: David Jacot 
---
 .../org/apache/kafka/common/requests/LeaderAndIsrResponse.java | 10 --
 1 file changed, 10 deletions(-)

diff --git 
a/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrResponse.java
 
b/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrResponse.java
index 60ab3d5..490983f 100644
--- 
a/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrResponse.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrResponse.java
@@ -20,11 +20,9 @@ import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.Uuid;
 import org.apache.kafka.common.message.LeaderAndIsrResponseData;
 import 
org.apache.kafka.common.message.LeaderAndIsrResponseData.LeaderAndIsrTopicError;
-import 
org.apache.kafka.common.message.LeaderAndIsrResponseData.LeaderAndIsrPartitionError;
 import org.apache.kafka.common.protocol.ApiKeys;
 import org.apache.kafka.common.protocol.ByteBufferAccessor;
 import org.apache.kafka.common.protocol.Errors;
-import org.apache.kafka.common.utils.FlattenedIterator;
 
 import java.nio.ByteBuffer;
 import java.util.Collections;
@@ -53,14 +51,6 @@ public class LeaderAndIsrResponse extends AbstractResponse {
 return this.data.topics();
 }
 
-public Iterable partitions() {
-if (version < 5) {
-return data.partitionErrors();
-}
-return () -> new FlattenedIterator<>(data.topics().iterator(),
-topic -> topic.partitionErrors().iterator());
-}
-
 public Errors error() {
 return Errors.forCode(data.errorCode());
 }



[kafka] branch trunk updated: MINOR: Remove unused LeaderAndIsrResponse.partitions() since it has been replaced with partitionErrors() (#10127)

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

rsivaram 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 987aafe  MINOR: Remove unused LeaderAndIsrResponse.partitions() since 
it has been replaced with partitionErrors() (#10127)
987aafe is described below

commit 987aafeddf69645b6089306dac8f9f32ac7cd9b3
Author: Rajini Sivaram 
AuthorDate: Mon Feb 15 12:21:01 2021 +

MINOR: Remove unused LeaderAndIsrResponse.partitions() since it has been 
replaced with partitionErrors() (#10127)

Reviewers: David Jacot 
---
 .../org/apache/kafka/common/requests/LeaderAndIsrResponse.java | 10 --
 1 file changed, 10 deletions(-)

diff --git 
a/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrResponse.java
 
b/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrResponse.java
index 60ab3d5..490983f 100644
--- 
a/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrResponse.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrResponse.java
@@ -20,11 +20,9 @@ import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.Uuid;
 import org.apache.kafka.common.message.LeaderAndIsrResponseData;
 import 
org.apache.kafka.common.message.LeaderAndIsrResponseData.LeaderAndIsrTopicError;
-import 
org.apache.kafka.common.message.LeaderAndIsrResponseData.LeaderAndIsrPartitionError;
 import org.apache.kafka.common.protocol.ApiKeys;
 import org.apache.kafka.common.protocol.ByteBufferAccessor;
 import org.apache.kafka.common.protocol.Errors;
-import org.apache.kafka.common.utils.FlattenedIterator;
 
 import java.nio.ByteBuffer;
 import java.util.Collections;
@@ -53,14 +51,6 @@ public class LeaderAndIsrResponse extends AbstractResponse {
 return this.data.topics();
 }
 
-public Iterable partitions() {
-if (version < 5) {
-return data.partitionErrors();
-}
-return () -> new FlattenedIterator<>(data.topics().iterator(),
-topic -> topic.partitionErrors().iterator());
-}
-
 public Errors error() {
 return Errors.forCode(data.errorCode());
 }



[kafka] branch 2.8 updated: MINOR: Revert AdminClient changes for DeleteTopics (#10121)

2021-02-12 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/2.8 by this push:
 new f4e475c  MINOR: Revert AdminClient changes for DeleteTopics (#10121)
f4e475c is described below

commit f4e475c2cdacc727567faa89d7ff11b932c6627b
Author: Justine Olshan 
AuthorDate: Fri Feb 12 15:26:02 2021 -0500

MINOR: Revert AdminClient changes for DeleteTopics (#10121)

This PR removes the AdminClient changes pertaining to deleteTopicsWithIds 
and DeleteTopicsWithIdsResult in 2.8 branch.

Reviewers: Chia-Ping Tsai , Rajini Sivaram 

---
 .../java/org/apache/kafka/clients/admin/Admin.java |  36 -
 .../clients/admin/DeleteTopicsWithIdsResult.java   |  54 
 .../kafka/clients/admin/KafkaAdminClient.java  | 102 --
 .../kafka/clients/admin/KafkaAdminClientTest.java  | 152 -
 .../kafka/clients/admin/MockAdminClient.java   |  31 -
 .../kafka/api/PlaintextAdminIntegrationTest.scala  |  19 ---
 6 files changed, 394 deletions(-)

diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java 
b/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
index d732cd2..e3f3e48 100644
--- a/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
+++ b/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
@@ -33,7 +33,6 @@ import org.apache.kafka.common.KafkaFuture;
 import org.apache.kafka.common.Metric;
 import org.apache.kafka.common.MetricName;
 import org.apache.kafka.common.TopicPartition;
-import org.apache.kafka.common.Uuid;
 import org.apache.kafka.common.TopicPartitionReplica;
 import org.apache.kafka.common.acl.AclBinding;
 import org.apache.kafka.common.acl.AclBindingFilter;
@@ -252,41 +251,6 @@ public interface Admin extends AutoCloseable {
  * @return The DeleteTopicsResult.
  */
 DeleteTopicsResult deleteTopics(Collection topics, 
DeleteTopicsOptions options);
-
-/**
- * This is a convenience method for {@link 
#deleteTopicsWithIds(Collection, DeleteTopicsOptions)}
- * with default options. See the overload for more details.
- * 
- * This operation is supported by brokers with version 2.8.0 or higher.
- *
- * @param topics The topic IDs for the topics to delete.
- * @return The DeleteTopicsWithIdsResult.
- */
-default DeleteTopicsWithIdsResult deleteTopicsWithIds(Collection 
topics) {
-return deleteTopicsWithIds(topics, new DeleteTopicsOptions());
-}
-
-/**
- * Delete a batch of topics.
- * 
- * This operation is not transactional so it may succeed for some topics 
while fail for others.
- * 
- * It may take several seconds after the {@link DeleteTopicsWithIdsResult} 
returns
- * success for all the brokers to become aware that the topics are gone.
- * During this time, {@link #listTopics()} and {@link 
#describeTopics(Collection)}
- * may continue to return information about the deleted topics.
- * 
- * If delete.topic.enable is false on the brokers, deleteTopicsWithIds 
will mark
- * the topics for deletion, but not actually delete them. The futures will
- * return successfully in this case.
- * 
- * This operation is supported by brokers with version 2.8.0 or higher.
- *
- * @param topics  The topic IDs for the topics to delete.
- * @param options The options to use when deleting the topics.
- * @return The DeleteTopicsWithIdsResult.
- */
-DeleteTopicsWithIdsResult deleteTopicsWithIds(Collection topics, 
DeleteTopicsOptions options);
 
 /**
  * List the topics available in the cluster with the default options.
diff --git 
a/clients/src/main/java/org/apache/kafka/clients/admin/DeleteTopicsWithIdsResult.java
 
b/clients/src/main/java/org/apache/kafka/clients/admin/DeleteTopicsWithIdsResult.java
deleted file mode 100644
index eeb9119..000
--- 
a/clients/src/main/java/org/apache/kafka/clients/admin/DeleteTopicsWithIdsResult.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- *http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the Lice

[kafka] branch 2.1 updated: KAFKA-12193: Re-resolve IPs after a client disconnects (#9902) (#10067) (#10108)

2021-02-11 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/2.1 by this push:
 new 9216968  KAFKA-12193: Re-resolve IPs after a client disconnects 
(#9902) (#10067) (#10108)
9216968 is described below

commit 9216968ef6ab7e5bba7182d61dfaef90513bb352
Author: Bob Barrett 
AuthorDate: Thu Feb 11 07:07:07 2021 -0800

KAFKA-12193: Re-resolve IPs after a client disconnects (#9902) (#10067) 
(#10108)

This patch changes the NetworkClient behavior to resolve the target node's 
hostname after disconnecting from an established connection, rather than 
waiting until the previously-resolved addresses are exhausted. This is to 
handle the scenario when the node's IP addresses have changed during the 
lifetime of the connection, and means that the client does not have to try to 
connect to invalid IP addresses until it has tried each address.

Reviewers: Mickael Maison , Satish Duggana 
, David Jacot 
---
 .../java/org/apache/kafka/clients/ClientUtils.java |  12 +-
 .../kafka/clients/ClusterConnectionStates.java |  27 +++-
 .../apache/kafka/clients/DefaultHostResolver.java  |  29 
 .../org/apache/kafka/clients/HostResolver.java |  26 
 .../org/apache/kafka/clients/NetworkClient.java|  73 +-
 .../kafka/clients/AddressChangeHostResolver.java   |  49 +++
 .../org/apache/kafka/clients/ClientUtilsTest.java  |   7 +-
 .../kafka/clients/ClusterConnectionStatesTest.java |  70 ++---
 .../apache/kafka/clients/NetworkClientTest.java| 157 +
 .../java/org/apache/kafka/test/MockSelector.java   |  11 +-
 10 files changed, 398 insertions(+), 63 deletions(-)

diff --git a/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java 
b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java
index 6c4bd13..dc72433 100644
--- a/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java
+++ b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java
@@ -96,8 +96,9 @@ public final class ClientUtils {
 clientSaslMechanism, true);
 }
 
-static List resolve(String host, ClientDnsLookup 
clientDnsLookup) throws UnknownHostException {
-InetAddress[] addresses = InetAddress.getAllByName(host);
+static List resolve(String host, ClientDnsLookup 
clientDnsLookup,
+ HostResolver hostResolver) throws 
UnknownHostException {
+InetAddress[] addresses = hostResolver.resolve(host);
 if (ClientDnsLookup.USE_ALL_DNS_IPS == clientDnsLookup) {
 return filterPreferredAddresses(addresses);
 } else {
@@ -105,6 +106,13 @@ public final class ClientUtils {
 }
 }
 
+/**
+ * Return a list containing the first address in `allAddresses` and 
subsequent addresses
+ * that are a subtype of the first address.
+ *
+ * The outcome is that all returned addresses are either IPv4 or IPv6 
(InetAddress has two
+ * subclasses: Inet4Address and Inet6Address).
+ */
 static List filterPreferredAddresses(InetAddress[] 
allAddresses) {
 List preferredAddresses = new ArrayList<>();
 Class clazz = null;
diff --git 
a/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java 
b/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java
index e9bd9712..9cad2cb 100644
--- 
a/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java
+++ 
b/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java
@@ -40,13 +40,16 @@ final class ClusterConnectionStates {
 private final double reconnectBackoffMaxExp;
 private final Map nodeState;
 private final Logger log;
+private final HostResolver hostResolver;
 
-public ClusterConnectionStates(long reconnectBackoffMs, long 
reconnectBackoffMaxMs, LogContext logContext) {
+public ClusterConnectionStates(long reconnectBackoffMs, long 
reconnectBackoffMaxMs,
+   LogContext logContext, HostResolver 
hostResolver) {
 this.log = logContext.logger(ClusterConnectionStates.class);
 this.reconnectBackoffInitMs = reconnectBackoffMs;
 this.reconnectBackoffMaxMs = reconnectBackoffMaxMs;
 this.reconnectBackoffMaxExp = Math.log(this.reconnectBackoffMaxMs / 
(double) Math.max(reconnectBackoffMs, 1)) / 
Math.log(RECONNECT_BACKOFF_EXP_BASE);
 this.nodeState = new HashMap<>();
+this.hostResolver = hostResolver;
 }
 
 /**
@@ -130,7 +133,7 @@ final class ClusterConnectionStates {
 // Create a new NodeConnectionState if nodeState does not already 
contain one
 // for the specified id or if the hostname associated with the node id 
changed.
 nodeState.put(id, new NodeConnectionState(ConnectionState.CONN

[kafka] branch 2.2 updated: KAFKA-12193: Re-resolve IPs after a client disconnects (#9902) (#10067) (#10109)

2021-02-11 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/2.2 by this push:
 new 3c715c1  KAFKA-12193: Re-resolve IPs after a client disconnects 
(#9902) (#10067) (#10109)
3c715c1 is described below

commit 3c715c1d261adc8f772a9d67ba8a240bda911a37
Author: Bob Barrett 
AuthorDate: Thu Feb 11 01:53:35 2021 -0800

KAFKA-12193: Re-resolve IPs after a client disconnects (#9902) (#10067) 
(#10109)

This patch changes the NetworkClient behavior to resolve the target node's 
hostname after disconnecting from an established connection, rather than 
waiting until the previously-resolved addresses are exhausted. This is to 
handle the scenario when the node's IP addresses have changed during the 
lifetime of the connection, and means that the client does not have to try to 
connect to invalid IP addresses until it has tried each address.

Reviewers: Mickael Maison , Satish Duggana 
, David Jacot 
---
 .../java/org/apache/kafka/clients/ClientUtils.java |  12 +-
 .../kafka/clients/ClusterConnectionStates.java |  27 +++-
 .../apache/kafka/clients/DefaultHostResolver.java  |  29 
 .../org/apache/kafka/clients/HostResolver.java |  26 
 .../org/apache/kafka/clients/NetworkClient.java|  73 +-
 .../kafka/clients/AddressChangeHostResolver.java   |  49 +++
 .../org/apache/kafka/clients/ClientUtilsTest.java  |   7 +-
 .../kafka/clients/ClusterConnectionStatesTest.java |  70 ++---
 .../apache/kafka/clients/NetworkClientTest.java| 157 +
 .../java/org/apache/kafka/test/MockSelector.java   |  11 +-
 10 files changed, 398 insertions(+), 63 deletions(-)

diff --git a/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java 
b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java
index cdd7513..2aa3ddb 100644
--- a/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java
+++ b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java
@@ -100,8 +100,9 @@ public final class ClientUtils {
 clientSaslMechanism, time, true);
 }
 
-static List resolve(String host, ClientDnsLookup 
clientDnsLookup) throws UnknownHostException {
-InetAddress[] addresses = InetAddress.getAllByName(host);
+static List resolve(String host, ClientDnsLookup 
clientDnsLookup,
+ HostResolver hostResolver) throws 
UnknownHostException {
+InetAddress[] addresses = hostResolver.resolve(host);
 if (ClientDnsLookup.USE_ALL_DNS_IPS == clientDnsLookup) {
 return filterPreferredAddresses(addresses);
 } else {
@@ -109,6 +110,13 @@ public final class ClientUtils {
 }
 }
 
+/**
+ * Return a list containing the first address in `allAddresses` and 
subsequent addresses
+ * that are a subtype of the first address.
+ *
+ * The outcome is that all returned addresses are either IPv4 or IPv6 
(InetAddress has two
+ * subclasses: Inet4Address and Inet6Address).
+ */
 static List filterPreferredAddresses(InetAddress[] 
allAddresses) {
 List preferredAddresses = new ArrayList<>();
 Class clazz = null;
diff --git 
a/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java 
b/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java
index 0d33483..d39712c 100644
--- 
a/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java
+++ 
b/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java
@@ -40,13 +40,16 @@ final class ClusterConnectionStates {
 private final double reconnectBackoffMaxExp;
 private final Map nodeState;
 private final Logger log;
+private final HostResolver hostResolver;
 
-public ClusterConnectionStates(long reconnectBackoffMs, long 
reconnectBackoffMaxMs, LogContext logContext) {
+public ClusterConnectionStates(long reconnectBackoffMs, long 
reconnectBackoffMaxMs,
+   LogContext logContext, HostResolver 
hostResolver) {
 this.log = logContext.logger(ClusterConnectionStates.class);
 this.reconnectBackoffInitMs = reconnectBackoffMs;
 this.reconnectBackoffMaxMs = reconnectBackoffMaxMs;
 this.reconnectBackoffMaxExp = Math.log(this.reconnectBackoffMaxMs / 
(double) Math.max(reconnectBackoffMs, 1)) / 
Math.log(RECONNECT_BACKOFF_EXP_BASE);
 this.nodeState = new HashMap<>();
+this.hostResolver = hostResolver;
 }
 
 /**
@@ -130,7 +133,7 @@ final class ClusterConnectionStates {
 // Create a new NodeConnectionState if nodeState does not already 
contain one
 // for the specified id or if the hostname associated with the node id 
changed.
 nodeState.put(id, new NodeConnectionState(ConnectionState.CONN

[kafka] branch 2.8 updated: Fill in the 2.8 release note for Authorizer (#9865)

2021-02-10 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/2.8 by this push:
 new 8d06681  Fill in the 2.8 release note for Authorizer (#9865)
8d06681 is described below

commit 8d06681cac2301210ee169e6440c41e99eff6f4c
Author: Cheng Tan <31675100+d8tlt...@users.noreply.github.com>
AuthorDate: Wed Feb 10 03:24:13 2021 -0800

Fill in the 2.8 release note for Authorizer (#9865)

Reviewers: Ismael Juma , Rajini Sivaram 

---
 docs/upgrade.html | 8 
 1 file changed, 8 insertions(+)

diff --git a/docs/upgrade.html b/docs/upgrade.html
index 9253433..9b43244 100644
--- a/docs/upgrade.html
+++ b/docs/upgrade.html
@@ -20,6 +20,14 @@
 

[kafka] branch trunk updated (7583e14 -> 19506b6)

2021-02-10 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

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


from 7583e14  MINOR: Remove unused parameters in functions. (#10035)
 add 19506b6  Fill in the 2.8 release note for Authorizer (#9865)

No new revisions were added by this update.

Summary of changes:
 docs/upgrade.html | 8 
 1 file changed, 8 insertions(+)



[kafka] branch 2.3 updated: KAFKA-12193: Re-resolve IPs after a client disconnects (#9902) (#10067)

2021-02-10 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/2.3 by this push:
 new 40867f1  KAFKA-12193: Re-resolve IPs after a client disconnects 
(#9902) (#10067)
40867f1 is described below

commit 40867f15665134525367825a65644d3570cbcc88
Author: Bob Barrett 
AuthorDate: Wed Feb 10 01:43:17 2021 -0800

KAFKA-12193: Re-resolve IPs after a client disconnects (#9902) (#10067)

This patch changes the NetworkClient behavior to resolve the target node's 
hostname after disconnecting from an established connection, rather than 
waiting until the previously-resolved addresses are exhausted. This is to 
handle the scenario when the node's IP addresses have changed during the 
lifetime of the connection, and means that the client does not have to try to 
connect to invalid IP addresses until it has tried each address.

Reviewers: Mickael Maison , Satish Duggana 
, David Jacot 
---
 .../java/org/apache/kafka/clients/ClientUtils.java |  12 +-
 .../kafka/clients/ClusterConnectionStates.java |  27 +++-
 .../apache/kafka/clients/DefaultHostResolver.java  |  29 
 .../org/apache/kafka/clients/HostResolver.java |  26 
 .../org/apache/kafka/clients/NetworkClient.java|  73 +-
 .../kafka/clients/AddressChangeHostResolver.java   |  49 +++
 .../org/apache/kafka/clients/ClientUtilsTest.java  |   7 +-
 .../kafka/clients/ClusterConnectionStatesTest.java |  70 ++---
 .../apache/kafka/clients/NetworkClientTest.java| 157 +
 .../java/org/apache/kafka/test/MockSelector.java   |  11 +-
 10 files changed, 398 insertions(+), 63 deletions(-)

diff --git a/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java 
b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java
index 1d71ea1..8a7d958 100644
--- a/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java
+++ b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java
@@ -100,8 +100,9 @@ public final class ClientUtils {
 clientSaslMechanism, time, true);
 }
 
-static List resolve(String host, ClientDnsLookup 
clientDnsLookup) throws UnknownHostException {
-InetAddress[] addresses = InetAddress.getAllByName(host);
+static List resolve(String host, ClientDnsLookup 
clientDnsLookup,
+ HostResolver hostResolver) throws 
UnknownHostException {
+InetAddress[] addresses = hostResolver.resolve(host);
 if (ClientDnsLookup.USE_ALL_DNS_IPS == clientDnsLookup) {
 return filterPreferredAddresses(addresses);
 } else {
@@ -109,6 +110,13 @@ public final class ClientUtils {
 }
 }
 
+/**
+ * Return a list containing the first address in `allAddresses` and 
subsequent addresses
+ * that are a subtype of the first address.
+ *
+ * The outcome is that all returned addresses are either IPv4 or IPv6 
(InetAddress has two
+ * subclasses: Inet4Address and Inet6Address).
+ */
 static List filterPreferredAddresses(InetAddress[] 
allAddresses) {
 List preferredAddresses = new ArrayList<>();
 Class clazz = null;
diff --git 
a/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java 
b/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java
index ec3465a..83d6f37 100644
--- 
a/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java
+++ 
b/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java
@@ -40,13 +40,16 @@ final class ClusterConnectionStates {
 private final double reconnectBackoffMaxExp;
 private final Map nodeState;
 private final Logger log;
+private final HostResolver hostResolver;
 
-public ClusterConnectionStates(long reconnectBackoffMs, long 
reconnectBackoffMaxMs, LogContext logContext) {
+public ClusterConnectionStates(long reconnectBackoffMs, long 
reconnectBackoffMaxMs,
+   LogContext logContext, HostResolver 
hostResolver) {
 this.log = logContext.logger(ClusterConnectionStates.class);
 this.reconnectBackoffInitMs = reconnectBackoffMs;
 this.reconnectBackoffMaxMs = reconnectBackoffMaxMs;
 this.reconnectBackoffMaxExp = Math.log(this.reconnectBackoffMaxMs / 
(double) Math.max(reconnectBackoffMs, 1)) / 
Math.log(RECONNECT_BACKOFF_EXP_BASE);
 this.nodeState = new HashMap<>();
+this.hostResolver = hostResolver;
 }
 
 /**
@@ -139,7 +142,7 @@ final class ClusterConnectionStates {
 // Create a new NodeConnectionState if nodeState does not already 
contain one
 // for the specified id or if the hostname associated with the node id 
changed.
 nodeState.put(id, new NodeConnectionState(ConnectionState.CONNECTING, 
now,
-this.reconnectB

[kafka] branch 2.4 updated: KAFKA-12193: Re-resolve IPs after a client disconnects (#9902) (#10065)

2021-02-10 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/2.4 by this push:
 new 2cdac40  KAFKA-12193: Re-resolve IPs after a client disconnects 
(#9902) (#10065)
2cdac40 is described below

commit 2cdac40d33d24496b9ff02808491c2ae867a6461
Author: Bob Barrett 
AuthorDate: Wed Feb 10 01:37:57 2021 -0800

KAFKA-12193: Re-resolve IPs after a client disconnects (#9902) (#10065)

This patch changes the NetworkClient behavior to resolve the target node's 
hostname after disconnecting from an established connection, rather than 
waiting until the previously-resolved addresses are exhausted. This is to 
handle the scenario when the node's IP addresses have changed during the 
lifetime of the connection, and means that the client does not have to try to 
connect to invalid IP addresses until it has tried each address.

Reviewers: Mickael Maison , Satish Duggana 
, David Jacot 
---
 .../java/org/apache/kafka/clients/ClientUtils.java |  12 +-
 .../kafka/clients/ClusterConnectionStates.java |  27 +++-
 .../apache/kafka/clients/DefaultHostResolver.java  |  29 
 .../org/apache/kafka/clients/HostResolver.java |  26 
 .../org/apache/kafka/clients/NetworkClient.java|  73 +-
 .../kafka/clients/AddressChangeHostResolver.java   |  49 +++
 .../org/apache/kafka/clients/ClientUtilsTest.java  |   7 +-
 .../kafka/clients/ClusterConnectionStatesTest.java |  70 ++---
 .../apache/kafka/clients/NetworkClientTest.java| 157 +
 .../java/org/apache/kafka/test/MockSelector.java   |  11 +-
 10 files changed, 398 insertions(+), 63 deletions(-)

diff --git a/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java 
b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java
index 1d71ea1..8a7d958 100644
--- a/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java
+++ b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java
@@ -100,8 +100,9 @@ public final class ClientUtils {
 clientSaslMechanism, time, true);
 }
 
-static List resolve(String host, ClientDnsLookup 
clientDnsLookup) throws UnknownHostException {
-InetAddress[] addresses = InetAddress.getAllByName(host);
+static List resolve(String host, ClientDnsLookup 
clientDnsLookup,
+ HostResolver hostResolver) throws 
UnknownHostException {
+InetAddress[] addresses = hostResolver.resolve(host);
 if (ClientDnsLookup.USE_ALL_DNS_IPS == clientDnsLookup) {
 return filterPreferredAddresses(addresses);
 } else {
@@ -109,6 +110,13 @@ public final class ClientUtils {
 }
 }
 
+/**
+ * Return a list containing the first address in `allAddresses` and 
subsequent addresses
+ * that are a subtype of the first address.
+ *
+ * The outcome is that all returned addresses are either IPv4 or IPv6 
(InetAddress has two
+ * subclasses: Inet4Address and Inet6Address).
+ */
 static List filterPreferredAddresses(InetAddress[] 
allAddresses) {
 List preferredAddresses = new ArrayList<>();
 Class clazz = null;
diff --git 
a/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java 
b/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java
index ec3465a..83d6f37 100644
--- 
a/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java
+++ 
b/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java
@@ -40,13 +40,16 @@ final class ClusterConnectionStates {
 private final double reconnectBackoffMaxExp;
 private final Map nodeState;
 private final Logger log;
+private final HostResolver hostResolver;
 
-public ClusterConnectionStates(long reconnectBackoffMs, long 
reconnectBackoffMaxMs, LogContext logContext) {
+public ClusterConnectionStates(long reconnectBackoffMs, long 
reconnectBackoffMaxMs,
+   LogContext logContext, HostResolver 
hostResolver) {
 this.log = logContext.logger(ClusterConnectionStates.class);
 this.reconnectBackoffInitMs = reconnectBackoffMs;
 this.reconnectBackoffMaxMs = reconnectBackoffMaxMs;
 this.reconnectBackoffMaxExp = Math.log(this.reconnectBackoffMaxMs / 
(double) Math.max(reconnectBackoffMs, 1)) / 
Math.log(RECONNECT_BACKOFF_EXP_BASE);
 this.nodeState = new HashMap<>();
+this.hostResolver = hostResolver;
 }
 
 /**
@@ -139,7 +142,7 @@ final class ClusterConnectionStates {
 // Create a new NodeConnectionState if nodeState does not already 
contain one
 // for the specified id or if the hostname associated with the node id 
changed.
 nodeState.put(id, new NodeConnectionState(ConnectionState.CONNECTING, 
now,
-this.reconnectB

[kafka] branch 2.5 updated: KAFKA-12193: Re-resolve IPs after a client disconnects (#9902) (#10064)

2021-02-10 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/2.5 by this push:
 new b86a6d3  KAFKA-12193: Re-resolve IPs after a client disconnects 
(#9902) (#10064)
b86a6d3 is described below

commit b86a6d369674bfaf79c640fb973b4afaca86374d
Author: Bob Barrett 
AuthorDate: Wed Feb 10 01:33:34 2021 -0800

KAFKA-12193: Re-resolve IPs after a client disconnects (#9902) (#10064)

This patch changes the NetworkClient behavior to resolve the target node's 
hostname after disconnecting from an established connection, rather than 
waiting until the previously-resolved addresses are exhausted. This is to 
handle the scenario when the node's IP addresses have changed during the 
lifetime of the connection, and means that the client does not have to try to 
connect to invalid IP addresses until it has tried each address.

Reviewers: Mickael Maison , Satish Duggana 
, David Jacot 
---
 .../java/org/apache/kafka/clients/ClientUtils.java |  12 +-
 .../kafka/clients/ClusterConnectionStates.java |  27 +++-
 .../apache/kafka/clients/DefaultHostResolver.java  |  29 
 .../org/apache/kafka/clients/HostResolver.java |  26 
 .../org/apache/kafka/clients/NetworkClient.java|  73 +-
 .../kafka/clients/AddressChangeHostResolver.java   |  49 +++
 .../org/apache/kafka/clients/ClientUtilsTest.java  |   7 +-
 .../kafka/clients/ClusterConnectionStatesTest.java |  70 ++---
 .../apache/kafka/clients/NetworkClientTest.java| 157 +
 .../java/org/apache/kafka/test/MockSelector.java   |  11 +-
 10 files changed, 398 insertions(+), 63 deletions(-)

diff --git a/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java 
b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java
index bcdac45..401920f 100644
--- a/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java
+++ b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java
@@ -106,8 +106,9 @@ public final class ClientUtils {
 clientSaslMechanism, time, true, logContext);
 }
 
-static List resolve(String host, ClientDnsLookup 
clientDnsLookup) throws UnknownHostException {
-InetAddress[] addresses = InetAddress.getAllByName(host);
+static List resolve(String host, ClientDnsLookup 
clientDnsLookup,
+ HostResolver hostResolver) throws 
UnknownHostException {
+InetAddress[] addresses = hostResolver.resolve(host);
 if (ClientDnsLookup.USE_ALL_DNS_IPS == clientDnsLookup) {
 return filterPreferredAddresses(addresses);
 } else {
@@ -115,6 +116,13 @@ public final class ClientUtils {
 }
 }
 
+/**
+ * Return a list containing the first address in `allAddresses` and 
subsequent addresses
+ * that are a subtype of the first address.
+ *
+ * The outcome is that all returned addresses are either IPv4 or IPv6 
(InetAddress has two
+ * subclasses: Inet4Address and Inet6Address).
+ */
 static List filterPreferredAddresses(InetAddress[] 
allAddresses) {
 List preferredAddresses = new ArrayList<>();
 Class clazz = null;
diff --git 
a/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java 
b/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java
index ec3465a..83d6f37 100644
--- 
a/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java
+++ 
b/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java
@@ -40,13 +40,16 @@ final class ClusterConnectionStates {
 private final double reconnectBackoffMaxExp;
 private final Map nodeState;
 private final Logger log;
+private final HostResolver hostResolver;
 
-public ClusterConnectionStates(long reconnectBackoffMs, long 
reconnectBackoffMaxMs, LogContext logContext) {
+public ClusterConnectionStates(long reconnectBackoffMs, long 
reconnectBackoffMaxMs,
+   LogContext logContext, HostResolver 
hostResolver) {
 this.log = logContext.logger(ClusterConnectionStates.class);
 this.reconnectBackoffInitMs = reconnectBackoffMs;
 this.reconnectBackoffMaxMs = reconnectBackoffMaxMs;
 this.reconnectBackoffMaxExp = Math.log(this.reconnectBackoffMaxMs / 
(double) Math.max(reconnectBackoffMs, 1)) / 
Math.log(RECONNECT_BACKOFF_EXP_BASE);
 this.nodeState = new HashMap<>();
+this.hostResolver = hostResolver;
 }
 
 /**
@@ -139,7 +142,7 @@ final class ClusterConnectionStates {
 // Create a new NodeConnectionState if nodeState does not already 
contain one
 // for the specified id or if the hostname associated with the node id 
changed.
 nodeState.put(id, new NodeConnectionState(ConnectionState.CONN

[kafka] branch trunk updated (821d867 -> 5c562ef)

2021-02-02 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

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


from 821d867  KAFKA-12246: Remove redundant suppression in KafkaAdminClient 
(#9989)
 add 5c562ef  KAFKA-10700 - Support mutual TLS authentication for SASL_SSL 
listeners (KIP-684) (#10007)

No new revisions were added by this update.

Summary of changes:
 .../kafka/common/network/ChannelBuilders.java  |  29 +
 .../kafka/common/network/SaslChannelBuilder.java   |   6 +-
 .../security/auth/SaslAuthenticationContext.java   |  20 
 .../authenticator/SaslServerAuthenticator.java |   7 +-
 .../common/network/SaslChannelBuilderTest.java |   4 +-
 .../authenticator/SaslAuthenticatorTest.java   | 126 -
 .../SaslGssapiSslEndToEndAuthorizationTest.scala   |   5 +
 .../SaslPlainSslEndToEndAuthorizationTest.scala|  27 -
 .../kafka/server/GssapiAuthenticationTest.scala|   2 +-
 .../test/scala/unit/kafka/utils/TestUtils.scala|  22 +++-
 10 files changed, 231 insertions(+), 17 deletions(-)



[kafka] branch trunk updated (d87fc21 -> 4f588f7)

2021-01-29 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

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


from d87fc21  MINOR: Ensure `InterBrokerSendThread` closes `NetworkClient` 
(#)
 add 4f588f7  KAFKA-10764: Add support for returning topic IDs on create, 
supplying topic IDs for delete (#9684)

No new revisions were added by this update.

Summary of changes:
 .../java/org/apache/kafka/clients/admin/Admin.java |  36 +
 .../kafka/clients/admin/CreateTopicsResult.java|  24 +++-
 ...sResult.java => DeleteTopicsWithIdsResult.java} |  13 +-
 .../kafka/clients/admin/KafkaAdminClient.java  | 104 +-
 ...Exception.java => UnknownTopicIdException.java} |   6 +-
 .../org/apache/kafka/common/protocol/Errors.java   |   4 +-
 .../kafka/common/requests/DeleteTopicsRequest.java |  46 +-
 .../common/message/CreateTopicsRequest.json|   4 +-
 .../common/message/CreateTopicsResponse.json   |   5 +-
 .../common/message/DeleteTopicsRequest.json|  11 +-
 .../common/message/DeleteTopicsResponse.json   |   9 +-
 .../kafka/clients/admin/KafkaAdminClientTest.java  | 154 -
 .../kafka/clients/admin/MockAdminClient.java   |  43 +-
 core/src/main/scala/kafka/server/KafkaApis.scala   |  38 +++--
 .../main/scala/kafka/server/ZkAdminManager.scala   |   9 ++
 .../kafka/api/AuthorizerIntegrationTest.scala  | 125 ++---
 .../kafka/api/BaseAdminIntegrationTest.scala   |   7 +
 .../kafka/api/PlaintextAdminIntegrationTest.scala  |  19 +++
 .../kafka/api/SaslSslAdminIntegrationTest.scala|   6 +
 .../kafka/integration/KafkaServerTestHarness.scala |  15 +-
 .../kafka/server/CreateTopicsRequestTest.scala |   6 +
 .../kafka/server/DeleteTopicsRequestTest.scala |  57 +++-
 .../TopicIdWithOldInterBrokerProtocolTest.scala|  60 +++-
 23 files changed, 745 insertions(+), 56 deletions(-)
 copy 
clients/src/main/java/org/apache/kafka/clients/admin/{DeleteTopicsResult.java 
=> DeleteTopicsWithIdsResult.java} (78%)
 copy 
clients/src/main/java/org/apache/kafka/common/errors/{InvalidRequiredAcksException.java
 => UnknownTopicIdException.java} (88%)
 rename core/src/test/scala/unit/kafka/server/{MetadataRequestIBPTest => 
}/TopicIdWithOldInterBrokerProtocolTest.scala (50%)



[kafka] branch trunk updated (462c89e -> 86b9fde)

2021-01-20 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

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


from 462c89e  KAFKA-12211: don't change perm for base/state dir when no 
persistent store (#9904)
 add 86b9fde  KAFKA-10869: Gate topic IDs behind IBP 2.8 (KIP-516) (#9814)

No new revisions were added by this update.

Summary of changes:
 core/src/main/scala/kafka/api/ApiVersion.scala |   2 +-
 .../controller/ControllerChannelManager.scala  |   4 +-
 .../scala/kafka/controller/KafkaController.scala   |  12 ++-
 core/src/main/scala/kafka/server/KafkaApis.scala   |   2 +-
 core/src/main/scala/kafka/server/KafkaConfig.scala |   5 +-
 .../main/scala/kafka/server/ZkAdminManager.scala   |   2 +-
 core/src/main/scala/kafka/zk/AdminZkClient.scala   |  21 ++--
 core/src/main/scala/kafka/zk/KafkaZkClient.scala   |  22 ++--
 core/src/main/scala/kafka/zk/ZkData.scala  |   4 +-
 ...PreferredReplicaLeaderElectionCommandTest.scala |   2 +-
 .../controller/ControllerIntegrationTest.scala | 119 -
 .../kafka/security/auth/ZkAuthorizationTest.scala  |   2 +-
 .../TopicIdWithOldInterBrokerProtocolTest.scala|  66 
 .../scala/unit/kafka/zk/KafkaZkClientTest.scala|  12 +--
 tests/kafkatest/services/kafka/kafka.py|  32 ++
 tests/kafkatest/tests/core/downgrade_test.py   |   8 ++
 tests/kafkatest/tests/core/upgrade_test.py |  14 ++-
 tests/kafkatest/version.py |   8 ++
 18 files changed, 290 insertions(+), 47 deletions(-)
 create mode 100644 
core/src/test/scala/unit/kafka/server/MetadataRequestIBPTest/TopicIdWithOldInterBrokerProtocolTest.scala



[kafka] branch trunk updated: KAFKA-10545: Create topic IDs and propagate to brokers (#9626)

2020-12-18 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

rsivaram 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 1dd1e7f  KAFKA-10545: Create topic IDs and propagate to brokers (#9626)
1dd1e7f is described below

commit 1dd1e7f945d7a8c1dc177223cd88800680f1ff46
Author: Justine Olshan 
AuthorDate: Fri Dec 18 17:19:50 2020 -0500

KAFKA-10545: Create topic IDs and propagate to brokers (#9626)

This change propagates topic ids to brokers in LeaderAndIsr Request. It 
also removes the topic name from the LeaderAndIsr Response, reorganizes the 
response to be sorted by topic, and includes the topic ID.

In addition, the topic ID is persisted to each replica in Log as well as in 
a file on disk. This file is read on startup and if the topic ID exists, it 
will be reloaded.

Reviewers: David Jacot , dengziming 
, Nikhil Bhatia , Rajini 
Sivaram 
---
 checkstyle/suppressions.xml|   2 +-
 .../kafka/common/requests/LeaderAndIsrRequest.java |  57 ++--
 .../common/requests/LeaderAndIsrResponse.java  |  58 +++--
 .../common/message/LeaderAndIsrRequest.json|  12 +-
 .../common/message/LeaderAndIsrResponse.json   |  24 +++-
 .../common/requests/LeaderAndIsrRequestTest.java   |  38 +-
 .../common/requests/LeaderAndIsrResponseTest.java  | 114 
 .../kafka/common/requests/RequestResponseTest.java |  60 ++---
 core/src/main/scala/kafka/api/ApiVersion.scala |   2 +-
 .../controller/ControllerChannelManager.scala  |  10 +-
 .../scala/kafka/controller/KafkaController.scala   |   7 +-
 core/src/main/scala/kafka/log/Log.scala|  21 ++-
 .../scala/kafka/server/PartitionMetadataFile.scala | 144 +
 .../main/scala/kafka/server/ReplicaManager.scala   |  63 +++--
 core/src/main/scala/kafka/zk/KafkaZkClient.scala   |   2 +-
 .../kafka/api/AuthorizerIntegrationTest.scala  |   9 +-
 .../controller/ControllerChannelManagerTest.scala  |  46 +--
 .../test/scala/unit/kafka/log/LogManagerTest.scala |   8 +-
 core/src/test/scala/unit/kafka/log/LogTest.scala   |  46 ++-
 .../kafka/server/BrokerEpochIntegrationTest.scala  |   5 +-
 .../scala/unit/kafka/server/KafkaApisTest.scala|   5 +-
 .../unit/kafka/server/LeaderElectionTest.scala |   7 +-
 .../unit/kafka/server/ReplicaManagerTest.scala | 136 ++-
 .../scala/unit/kafka/server/RequestQuotaTest.scala |   2 +
 .../unit/kafka/server/ServerShutdownTest.scala |   5 +-
 .../apache/kafka/message/MessageDataGenerator.java |   1 -
 26 files changed, 760 insertions(+), 124 deletions(-)

diff --git a/checkstyle/suppressions.xml b/checkstyle/suppressions.xml
index b44e713..8539034 100644
--- a/checkstyle/suppressions.xml
+++ b/checkstyle/suppressions.xml
@@ -100,7 +100,7 @@
   
files="RequestResponseTest.java|FetcherTest.java|KafkaAdminClientTest.java"/>
 
 
+  
files="MemoryRecordsTest|MetricsTest|RequestResponseTest|TestSslUtils|AclAuthorizerBenchmark"/>
 
 
diff --git 
a/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrRequest.java
 
b/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrRequest.java
index 833e025..939212a 100644
--- 
a/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrRequest.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrRequest.java
@@ -17,11 +17,13 @@
 package org.apache.kafka.common.requests;
 
 import org.apache.kafka.common.Node;
+import org.apache.kafka.common.Uuid;
 import org.apache.kafka.common.message.LeaderAndIsrRequestData;
 import 
org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrLiveLeader;
 import 
org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrTopicState;
 import 
org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState;
 import org.apache.kafka.common.message.LeaderAndIsrResponseData;
+import 
org.apache.kafka.common.message.LeaderAndIsrResponseData.LeaderAndIsrTopicError;
 import 
org.apache.kafka.common.message.LeaderAndIsrResponseData.LeaderAndIsrPartitionError;
 import org.apache.kafka.common.protocol.ApiKeys;
 import org.apache.kafka.common.protocol.ByteBufferAccessor;
@@ -43,12 +45,15 @@ public class LeaderAndIsrRequest extends 
AbstractControlRequest {
 public static class Builder extends 
AbstractControlRequest.Builder {
 
 private final List partitionStates;
+private final Map topicIds;
 private final Collection liveLeaders;
 
 public Builder(short version, int controllerId, int controllerEpoch, 
long brokerEpoch,
-   List partitionStates, 
Collection liveLeaders) {
+   List partitionStates, 
Map topicIds,
+   Collection liveLea

[kafka] branch trunk updated (baef516 -> 5c921af)

2020-12-18 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

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


from baef516  Add ConfigurableProducerSpec to Trogdor for improved E2E 
latency tracking. (#9736)
 add 5c921af  KAFKA-10547; Add TopicId in MetadataResponse (#9622)

No new revisions were added by this update.

Summary of changes:
 .../kafka/clients/admin/KafkaAdminClient.java  |  2 +-
 .../kafka/clients/admin/TopicDescription.java  | 12 +
 .../main/java/org/apache/kafka/common/Cluster.java | 37 --
 .../kafka/common/requests/MetadataResponse.java| 21 ++--
 .../common/requests/UpdateMetadataRequest.java | 24 +++--
 .../resources/common/message/MetadataRequest.json  |  6 ++-
 .../resources/common/message/MetadataResponse.json |  4 +-
 .../common/message/UpdateMetadataRequest.json  |  4 +-
 .../common/message/UpdateMetadataResponse.json |  2 +-
 .../kafka/clients/admin/KafkaAdminClientTest.java  |  3 +-
 .../kafka/common/requests/RequestResponseTest.java |  2 +-
 .../common/requests/UpdateMetadataRequestTest.java | 31 +---
 core/src/main/scala/kafka/admin/TopicCommand.scala | 24 +
 core/src/main/scala/kafka/api/ApiVersion.scala | 11 -
 .../controller/ControllerChannelManager.scala  |  9 +++-
 .../main/scala/kafka/server/MetadataCache.scala| 30 
 .../kafka/api/AuthorizerIntegrationTest.scala  |  3 +-
 .../test/scala/unit/kafka/api/ApiVersionTest.scala |  4 +-
 .../controller/ControllerChannelManagerTest.scala  | 57 +-
 .../kafka/server/BrokerEpochIntegrationTest.scala  |  4 +-
 .../scala/unit/kafka/server/KafkaApisTest.scala|  4 +-
 .../unit/kafka/server/MetadataCacheTest.scala  | 19 +---
 .../unit/kafka/server/MetadataRequestTest.scala| 27 ++
 .../scala/unit/kafka/server/RequestQuotaTest.scala |  3 +-
 .../jmh/metadata/MetadataRequestBenchmark.java |  2 +-
 25 files changed, 260 insertions(+), 85 deletions(-)



[kafka] branch trunk updated: KAKFA-10619: Idempotent producer will get authorized once it has a WRITE access to at least one topic (KIP-679) (#9485)

2020-12-18 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

rsivaram 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 ae3a6ed  KAKFA-10619: Idempotent producer will get authorized once it 
has a WRITE access to at least one topic (KIP-679) (#9485)
ae3a6ed is described below

commit ae3a6ed990f91708686d27c6023bac050c422248
Author: Cheng Tan <31675100+d8tlt...@users.noreply.github.com>
AuthorDate: Fri Dec 18 10:08:46 2020 -0800

KAKFA-10619: Idempotent producer will get authorized once it has a WRITE 
access to at least one topic (KIP-679) (#9485)

Includes:
- New API to authorize by resource type
- Default implementation for the method that supports super users and ACLs
- Optimized implementation in AclAuthorizer that supports ACLs, super users 
and allow.everyone.if.no.acl.found
- Benchmarks and tests
- InitProducerIdRequest authorized for Cluster:IdempotentWrite or WRITE to 
any topic, ProduceRequest authorized only for topic even if idempotent

Reviewers: Lucas Bradstreet , Rajini Sivaram 

---
 checkstyle/suppressions.xml|   6 +-
 .../apache/kafka/common/utils/SecurityUtils.java   |  30 ++
 .../apache/kafka/server/authorizer/Authorizer.java | 155 -
 .../kafka/security/authorizer/AclAuthorizer.scala  | 164 -
 .../security/authorizer/AuthorizerWrapper.scala|  57 +++-
 core/src/main/scala/kafka/server/KafkaApis.scala   |  16 +-
 .../kafka/api/AuthorizerIntegrationTest.scala  | 108 +-
 .../security/authorizer/AclAuthorizerTest.scala|  68 ++--
 .../AuthorizerInterfaceDefaultTest.scala   |  94 ++
 .../authorizer/AuthorizerWrapperTest.scala | 106 ++
 .../security/authorizer/BaseAuthorizerTest.scala   | 375 +
 .../kafka/jmh/acl/AclAuthorizerBenchmark.java  | 125 +--
 12 files changed, 1201 insertions(+), 103 deletions(-)

diff --git a/checkstyle/suppressions.xml b/checkstyle/suppressions.xml
index 7fb20ec..69df37d 100644
--- a/checkstyle/suppressions.xml
+++ b/checkstyle/suppressions.xml
@@ -67,13 +67,13 @@
   
files="(Utils|Topic|KafkaLZ4BlockOutputStream|AclData|JoinGroupRequest).java"/>
 
 
+  
files="(ConsumerCoordinator|Fetcher|Sender|KafkaProducer|BufferPool|ConfigDef|RecordAccumulator|KerberosLogin|AbstractRequest|AbstractResponse|Selector|SslFactory|SslTransportLayer|SaslClientAuthenticator|SaslClientCallbackHandler|SaslServerAuthenticator|AbstractCoordinator|TransactionManager|AbstractStickyAssignor|DefaultSslEngineFactory|Authorizer).java"/>
 
 
 
 
+  
files="(ConsumerCoordinator|BufferPool|Fetcher|MetricName|Node|ConfigDef|RecordBatch|SslFactory|SslTransportLayer|MetadataResponse|KerberosLogin|Selector|Sender|Serdes|TokenInformation|Agent|Values|PluginUtils|MiniTrogdorCluster|TasksRequest|KafkaProducer|AbstractStickyAssignor|KafkaRaftClient|Authorizer).java"/>
 
 
@@ -100,7 +100,7 @@
   
files="RequestResponseTest.java|FetcherTest.java|KafkaAdminClientTest.java"/>
 
 
+  
files="MemoryRecordsTest|MetricsTest|TestSslUtils|AclAuthorizerBenchmark"/>
 
 
diff --git 
a/clients/src/main/java/org/apache/kafka/common/utils/SecurityUtils.java 
b/clients/src/main/java/org/apache/kafka/common/utils/SecurityUtils.java
index 12defdc..88a4cfc 100644
--- a/clients/src/main/java/org/apache/kafka/common/utils/SecurityUtils.java
+++ b/clients/src/main/java/org/apache/kafka/common/utils/SecurityUtils.java
@@ -19,6 +19,8 @@ package org.apache.kafka.common.utils;
 import org.apache.kafka.common.acl.AclOperation;
 import org.apache.kafka.common.acl.AclPermissionType;
 import org.apache.kafka.common.config.SecurityConfig;
+import org.apache.kafka.common.resource.PatternType;
+import org.apache.kafka.common.resource.ResourcePattern;
 import org.apache.kafka.common.resource.ResourceType;
 import org.apache.kafka.common.security.auth.SecurityProviderCreator;
 import org.apache.kafka.common.security.auth.KafkaPrincipal;
@@ -146,4 +148,32 @@ public class SecurityUtils {
 }
 return builder.toString();
 }
+
+public static void authorizeByResourceTypeCheckArgs(AclOperation op,
+ResourceType type) {
+if (type == ResourceType.ANY) {
+throw new IllegalArgumentException(
+"Must specify a non-filter resource type for 
authorizeByResourceType");
+}
+
+if (type == ResourceType.UNKNOWN) {
+throw new IllegalArgumentException(
+"Unknown resource type");
+}
+
+if (op == AclOperation.ANY) {
+throw new IllegalArgumentException(
+"Must specify a non-filter operation type for 
authorize

[kafka] branch 2.6 updated: KAFKA-10798; Ensure response is delayed for failed SASL authentication with connection close delay (#9678)

2020-12-16 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/2.6 by this push:
 new f0e234d  KAFKA-10798; Ensure response is delayed for failed SASL 
authentication with connection close delay (#9678)
f0e234d is described below

commit f0e234d8e71a355e848adb40628cbe6f97f35268
Author: Rajini Sivaram 
AuthorDate: Mon Dec 7 16:12:18 2020 +

KAFKA-10798; Ensure response is delayed for failed SASL authentication with 
connection close delay (#9678)

Reviewers: Manikumar Reddy 
---
 .../authenticator/SaslServerAuthenticator.java |  2 +-
 .../apache/kafka/common/network/NioEchoServer.java |  2 +-
 .../SaslAuthenticatorFailureDelayTest.java | 37 --
 3 files changed, 37 insertions(+), 4 deletions(-)

diff --git 
a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticator.java
 
b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticator.java
index ce7bd69..923ddea 100644
--- 
a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticator.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticator.java
@@ -460,7 +460,7 @@ public class SaslServerAuthenticator implements 
Authenticator {
 String errorMessage = "Authentication failed during "
 + reauthInfo.authenticationOrReauthenticationText()
 + " due to invalid credentials with SASL mechanism 
" + saslMechanism;
-sendKafkaResponse(requestContext, new 
SaslAuthenticateResponse(
+buildResponseOnAuthenticateFailure(requestContext, new 
SaslAuthenticateResponse(
 new SaslAuthenticateResponseData()
 
.setErrorCode(Errors.SASL_AUTHENTICATION_FAILED.code())
 .setErrorMessage(errorMessage)));
diff --git 
a/clients/src/test/java/org/apache/kafka/common/network/NioEchoServer.java 
b/clients/src/test/java/org/apache/kafka/common/network/NioEchoServer.java
index d69b184..5d96d5e 100644
--- a/clients/src/test/java/org/apache/kafka/common/network/NioEchoServer.java
+++ b/clients/src/test/java/org/apache/kafka/common/network/NioEchoServer.java
@@ -95,7 +95,7 @@ public class NioEchoServer extends Thread {
 public NioEchoServer(ListenerName listenerName, SecurityProtocol 
securityProtocol, AbstractConfig config,
  String serverHost, ChannelBuilder channelBuilder, 
CredentialCache credentialCache,
  int failedAuthenticationDelayMs, Time time) throws 
Exception {
-this(listenerName, securityProtocol, config, serverHost, 
channelBuilder, credentialCache, 100, time,
+this(listenerName, securityProtocol, config, serverHost, 
channelBuilder, credentialCache, failedAuthenticationDelayMs, time,
 new DelegationTokenCache(ScramMechanism.mechanismNames()));
 }
 
diff --git 
a/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorFailureDelayTest.java
 
b/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorFailureDelayTest.java
index 599345a..19003ed 100644
--- 
a/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorFailureDelayTest.java
+++ 
b/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorFailureDelayTest.java
@@ -45,6 +45,7 @@ import java.net.InetSocketAddress;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 
@@ -55,7 +56,7 @@ import static org.junit.Assert.assertTrue;
 public class SaslAuthenticatorFailureDelayTest {
 private static final int BUFFER_SIZE = 4 * 1024;
 
-private final MockTime time = new MockTime(10);
+private final MockTime time = new MockTime(1);
 private NioEchoServer server;
 private Selector selector;
 private ChannelBuilder channelBuilder;
@@ -119,6 +120,38 @@ public class SaslAuthenticatorFailureDelayTest {
 }
 
 /**
+ * Tests that SASL/SCRAM clients with invalid password fail authentication 
with
+ * connection close delay if configured.
+ */
+@Test
+public void testInvalidPasswordSaslScram() throws Exception {
+String node = "0";
+SecurityProtocol securityProtocol = SecurityProtocol.SASL_SSL;
+TestJaasConfig jaasConfig = configureMechanisms("SCRAM-SHA-256", 
Collections.singletonList("SCRAM-SHA-256"));
+jaasConfig.setClientOptions("SCRAM-SHA-256", TestJaasConfig.USERNAME, 
"invalidpassword");
+
+server = createEchoServer

[kafka] branch 2.7 updated: KAFKA-10798; Ensure response is delayed for failed SASL authentication with connection close delay (#9678)

2020-12-16 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/2.7 by this push:
 new 7f5449f  KAFKA-10798; Ensure response is delayed for failed SASL 
authentication with connection close delay (#9678)
7f5449f is described below

commit 7f5449f341602f84de1427d0626f8c2590e05535
Author: Rajini Sivaram 
AuthorDate: Mon Dec 7 16:12:18 2020 +

KAFKA-10798; Ensure response is delayed for failed SASL authentication with 
connection close delay (#9678)

Reviewers: Manikumar Reddy 
---
 .../authenticator/SaslServerAuthenticator.java |  2 +-
 .../apache/kafka/common/network/NioEchoServer.java |  2 +-
 .../SaslAuthenticatorFailureDelayTest.java | 37 --
 3 files changed, 37 insertions(+), 4 deletions(-)

diff --git 
a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticator.java
 
b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticator.java
index ecd5d4f..f61a181 100644
--- 
a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticator.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticator.java
@@ -463,7 +463,7 @@ public class SaslServerAuthenticator implements 
Authenticator {
 String errorMessage = "Authentication failed during "
 + reauthInfo.authenticationOrReauthenticationText()
 + " due to invalid credentials with SASL mechanism 
" + saslMechanism;
-sendKafkaResponse(requestContext, new 
SaslAuthenticateResponse(
+buildResponseOnAuthenticateFailure(requestContext, new 
SaslAuthenticateResponse(
 new SaslAuthenticateResponseData()
 
.setErrorCode(Errors.SASL_AUTHENTICATION_FAILED.code())
 .setErrorMessage(errorMessage)));
diff --git 
a/clients/src/test/java/org/apache/kafka/common/network/NioEchoServer.java 
b/clients/src/test/java/org/apache/kafka/common/network/NioEchoServer.java
index d69b184..5d96d5e 100644
--- a/clients/src/test/java/org/apache/kafka/common/network/NioEchoServer.java
+++ b/clients/src/test/java/org/apache/kafka/common/network/NioEchoServer.java
@@ -95,7 +95,7 @@ public class NioEchoServer extends Thread {
 public NioEchoServer(ListenerName listenerName, SecurityProtocol 
securityProtocol, AbstractConfig config,
  String serverHost, ChannelBuilder channelBuilder, 
CredentialCache credentialCache,
  int failedAuthenticationDelayMs, Time time) throws 
Exception {
-this(listenerName, securityProtocol, config, serverHost, 
channelBuilder, credentialCache, 100, time,
+this(listenerName, securityProtocol, config, serverHost, 
channelBuilder, credentialCache, failedAuthenticationDelayMs, time,
 new DelegationTokenCache(ScramMechanism.mechanismNames()));
 }
 
diff --git 
a/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorFailureDelayTest.java
 
b/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorFailureDelayTest.java
index 599345a..19003ed 100644
--- 
a/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorFailureDelayTest.java
+++ 
b/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorFailureDelayTest.java
@@ -45,6 +45,7 @@ import java.net.InetSocketAddress;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 
@@ -55,7 +56,7 @@ import static org.junit.Assert.assertTrue;
 public class SaslAuthenticatorFailureDelayTest {
 private static final int BUFFER_SIZE = 4 * 1024;
 
-private final MockTime time = new MockTime(10);
+private final MockTime time = new MockTime(1);
 private NioEchoServer server;
 private Selector selector;
 private ChannelBuilder channelBuilder;
@@ -119,6 +120,38 @@ public class SaslAuthenticatorFailureDelayTest {
 }
 
 /**
+ * Tests that SASL/SCRAM clients with invalid password fail authentication 
with
+ * connection close delay if configured.
+ */
+@Test
+public void testInvalidPasswordSaslScram() throws Exception {
+String node = "0";
+SecurityProtocol securityProtocol = SecurityProtocol.SASL_SSL;
+TestJaasConfig jaasConfig = configureMechanisms("SCRAM-SHA-256", 
Collections.singletonList("SCRAM-SHA-256"));
+jaasConfig.setClientOptions("SCRAM-SHA-256", TestJaasConfig.USERNAME, 
"invalidpassword");
+
+server = createEchoServer

[kafka] branch trunk updated (8db3b1a -> b8ebcc2)

2020-12-07 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

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


from 8db3b1a  KAFKA-10811: Correct the MirrorConnectorsIntegrationTest to 
correctly mask the exit procedures (#9698)
 add b8ebcc2  KAFKA-10798; Ensure response is delayed for failed SASL 
authentication with connection close delay (#9678)

No new revisions were added by this update.

Summary of changes:
 .../authenticator/SaslServerAuthenticator.java |  2 +-
 .../apache/kafka/common/network/NioEchoServer.java |  2 +-
 .../SaslAuthenticatorFailureDelayTest.java | 37 --
 3 files changed, 37 insertions(+), 4 deletions(-)



[kafka] branch trunk updated: KAFKA-10554; Perform follower truncation based on diverging epochs in Fetch response (#9382)

2020-12-03 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

rsivaram 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 7ecc3a5  KAFKA-10554; Perform follower truncation based on diverging 
epochs in Fetch response (#9382)
7ecc3a5 is described below

commit 7ecc3a579a4b13e0cef4bd3129982ea3bc1a9341
Author: Rajini Sivaram 
AuthorDate: Thu Dec 3 10:12:06 2020 +

KAFKA-10554; Perform follower truncation based on diverging epochs in Fetch 
response (#9382)

From IBP 2.7 onwards, fetch responses include diverging epoch and offset in 
fetch responses if lastFetchedEpoch is provided in the fetch request. This PR 
uses that information for truncation and avoids the additional 
OffsetForLeaderEpoch requests in followers when lastFetchedEpoch is known.

Co-authored-by: Jason Gustafson 

Reviewers: Jason Gustafson , Nikhil Bhatia 

---
 core/src/main/scala/kafka/api/ApiVersion.scala |   2 +
 core/src/main/scala/kafka/log/Log.scala|  16 +-
 .../kafka/server/AbstractFetcherManager.scala  |  32 ++--
 .../scala/kafka/server/AbstractFetcherThread.scala | 117 +
 .../kafka/server/ReplicaAlterLogDirsManager.scala  |   2 +-
 .../kafka/server/ReplicaAlterLogDirsThread.scala   |  13 +-
 .../scala/kafka/server/ReplicaFetcherThread.scala  |  17 +-
 .../main/scala/kafka/server/ReplicaManager.scala   |  17 +-
 .../server/DynamicBrokerReconfigurationTest.scala  |   4 +-
 .../kafka/server/AbstractFetcherManagerTest.scala  |   8 +-
 .../kafka/server/AbstractFetcherThreadTest.scala   | 166 ++
 .../AbstractFetcherThreadWithIbp26Test.scala   |  23 +++
 .../server/ReplicaAlterLogDirsThreadTest.scala |  49 +++---
 .../kafka/server/ReplicaFetcherThreadTest.scala| 192 +
 .../unit/kafka/server/ReplicaManagerTest.scala |  28 ++-
 ...eplicationProtocolAcceptanceWithIbp26Test.scala |  29 
 .../util/ReplicaFetcherMockBlockingSend.scala  |  18 +-
 .../jmh/fetcher/ReplicaFetcherThreadBenchmark.java |   7 +-
 18 files changed, 565 insertions(+), 175 deletions(-)

diff --git a/core/src/main/scala/kafka/api/ApiVersion.scala 
b/core/src/main/scala/kafka/api/ApiVersion.scala
index 16e6fb9..fdebc27 100644
--- a/core/src/main/scala/kafka/api/ApiVersion.scala
+++ b/core/src/main/scala/kafka/api/ApiVersion.scala
@@ -130,6 +130,8 @@ object ApiVersion {
 
   val latestVersion: ApiVersion = allVersions.last
 
+  def isTruncationOnFetchSupported(version: ApiVersion): Boolean = version >= 
KAFKA_2_7_IV1
+
   /**
* Return the minimum `ApiVersion` that supports `RecordVersion`.
*/
diff --git a/core/src/main/scala/kafka/log/Log.scala 
b/core/src/main/scala/kafka/log/Log.scala
index 15dc9ce..b325adb 100644
--- a/core/src/main/scala/kafka/log/Log.scala
+++ b/core/src/main/scala/kafka/log/Log.scala
@@ -50,11 +50,11 @@ import scala.collection.mutable.{ArrayBuffer, ListBuffer}
 import scala.collection.{Seq, Set, mutable}
 
 object LogAppendInfo {
-  val UnknownLogAppendInfo = LogAppendInfo(None, -1, RecordBatch.NO_TIMESTAMP, 
-1L, RecordBatch.NO_TIMESTAMP, -1L,
+  val UnknownLogAppendInfo = LogAppendInfo(None, -1, None, 
RecordBatch.NO_TIMESTAMP, -1L, RecordBatch.NO_TIMESTAMP, -1L,
 RecordConversionStats.EMPTY, NoCompressionCodec, NoCompressionCodec, -1, 
-1, offsetsMonotonic = false, -1L)
 
   def unknownLogAppendInfoWithLogStartOffset(logStartOffset: Long): 
LogAppendInfo =
-LogAppendInfo(None, -1, RecordBatch.NO_TIMESTAMP, -1L, 
RecordBatch.NO_TIMESTAMP, logStartOffset,
+LogAppendInfo(None, -1, None, RecordBatch.NO_TIMESTAMP, -1L, 
RecordBatch.NO_TIMESTAMP, logStartOffset,
   RecordConversionStats.EMPTY, NoCompressionCodec, NoCompressionCodec, -1, 
-1,
   offsetsMonotonic = false, -1L)
 
@@ -64,7 +64,7 @@ object LogAppendInfo {
* in unknownLogAppendInfoWithLogStartOffset, but with additiona fields 
recordErrors and errorMessage
*/
   def unknownLogAppendInfoWithAdditionalInfo(logStartOffset: Long, 
recordErrors: Seq[RecordError], errorMessage: String): LogAppendInfo =
-LogAppendInfo(None, -1, RecordBatch.NO_TIMESTAMP, -1L, 
RecordBatch.NO_TIMESTAMP, logStartOffset,
+LogAppendInfo(None, -1, None, RecordBatch.NO_TIMESTAMP, -1L, 
RecordBatch.NO_TIMESTAMP, logStartOffset,
   RecordConversionStats.EMPTY, NoCompressionCodec, NoCompressionCodec, -1, 
-1,
   offsetsMonotonic = false, -1L, recordErrors, errorMessage)
 }
@@ -82,6 +82,7 @@ object LeaderHwChange {
  * @param firstOffset The first offset in the message set unless the message 
format is less than V2 and we are appending
  *to the follower.
  * @param lastOffset The last offset in the message set
+ * @param lastLeaderEpoch The partition leader epoch corresponding to the last 
offset, if available.
  * @param maxTimestamp The maximum timestamp of the message set.
  * @param offsetOfMaxTimest

[kafka] branch trunk updated: KAFKA-10727; Handle Kerberos error during re-login as transient failure in clients (#9605)

2020-11-23 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

rsivaram 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 ed8659b  KAFKA-10727; Handle Kerberos error during re-login as 
transient failure in clients (#9605)
ed8659b is described below

commit ed8659b4a09a4a6798b8077ed2d8fb94b6da
Author: Rajini Sivaram 
AuthorDate: Mon Nov 23 09:04:16 2020 +

KAFKA-10727; Handle Kerberos error during re-login as transient failure in 
clients (#9605)

We use a background thread for Kerberos to perform re-login before tickets 
expire. The thread performs logout() followed by login(), relying on the Java 
library to clear and then populate credentials in Subject. This leaves a timing 
window where clients fail to authenticate because credentials are not 
available. We cannot introduce any form of locking since authentication is 
performed on the network thread. So this commit treats NO_CRED as a transient 
failure rather than a fatal authe [...]

Reviewers: Ron Dagostino , Manikumar Reddy 

---
 .../kafka/common/network/SaslChannelBuilder.java   |   2 +-
 .../authenticator/SaslClientAuthenticator.java |   8 +-
 .../common/security/kerberos/KerberosError.java|  19 
 .../common/security/kerberos/KerberosLogin.java|   9 +-
 .../kafka/server/GssapiAuthenticationTest.scala| 107 ++---
 5 files changed, 126 insertions(+), 19 deletions(-)

diff --git 
a/clients/src/main/java/org/apache/kafka/common/network/SaslChannelBuilder.java 
b/clients/src/main/java/org/apache/kafka/common/network/SaslChannelBuilder.java
index a29148a..f01c4ef 100644
--- 
a/clients/src/main/java/org/apache/kafka/common/network/SaslChannelBuilder.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/network/SaslChannelBuilder.java
@@ -327,7 +327,7 @@ public class SaslChannelBuilder implements ChannelBuilder, 
ListenerReconfigurabl
 }
 }
 
-private Class defaultLoginClass() {
+protected Class defaultLoginClass() {
 if (jaasContexts.containsKey(SaslConfigs.GSSAPI_MECHANISM))
 return KerberosLogin.class;
 if 
(OAuthBearerLoginModule.OAUTHBEARER_MECHANISM.equals(clientSaslMechanism))
diff --git 
a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientAuthenticator.java
 
b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientAuthenticator.java
index 9c99dab..00a4bfc 100644
--- 
a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientAuthenticator.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientAuthenticator.java
@@ -539,15 +539,17 @@ public class SaslClientAuthenticator implements 
Authenticator {
 " Users must configure FQDN of kafka brokers when 
authenticating using SASL and" +
 " `socketChannel.socket().getInetAddress().getHostName()` 
must match the hostname in `principal/hostname@realm`";
 }
-error += " Kafka Client will go to AUTHENTICATION_FAILED state.";
 //Unwrap the SaslException inside `PrivilegedActionException`
 Throwable cause = e.getCause();
 // Treat transient Kerberos errors as non-fatal SaslExceptions 
that are processed as I/O exceptions
 // and all other failures as fatal SaslAuthenticationException.
-if (kerberosError != null && kerberosError.retriable())
+if ((kerberosError != null && kerberosError.retriable()) || 
(kerberosError == null && KerberosError.isRetriableClientGssException(e))) {
+error += " Kafka Client will retry.";
 throw new SaslException(error, cause);
-else
+} else {
+error += " Kafka Client will go to AUTHENTICATION_FAILED 
state.";
 throw new SaslAuthenticationException(error, cause);
+}
 }
 }
 
diff --git 
a/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosError.java
 
b/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosError.java
index 9c76482..4b8e8e0 100644
--- 
a/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosError.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosError.java
@@ -19,6 +19,7 @@ package org.apache.kafka.common.security.kerberos;
 import org.apache.kafka.common.KafkaException;
 import org.apache.kafka.common.security.authenticator.SaslClientAuthenticator;
 import org.apache.kafka.common.utils.Java;
+import org.ietf.jgss.GSSException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -109,4 +110,22 @@ public enum KerberosError {
 }
 return null;
 }
+
+/**
+ * Returns true if the 

[kafka] branch trunk updated (30bc21c -> 77a46b1)

2020-11-18 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

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


from 30bc21c  KAFKA-9628; Replace Produce request/response with automated 
protocol (#9401)
 add 77a46b1  KAFKA-10545: Create topic IDs in ZooKeeper and Controller 
(#9473)

No new revisions were added by this update.

Summary of changes:
 .../scala/kafka/controller/ControllerContext.scala |  27 -
 .../scala/kafka/controller/KafkaController.scala   |  38 +--
 core/src/main/scala/kafka/zk/AdminZkClient.scala   |   8 +-
 core/src/main/scala/kafka/zk/KafkaZkClient.scala   |  85 ++--
 core/src/main/scala/kafka/zk/ZkData.scala  |  36 ---
 ...PreferredReplicaLeaderElectionCommandTest.scala |   4 +-
 .../controller/ControllerIntegrationTest.scala | 110 -
 .../kafka/security/auth/ZkAuthorizationTest.scala  |   8 +-
 .../scala/unit/kafka/zk/KafkaZkClientTest.scala|  29 +++---
 9 files changed, 290 insertions(+), 55 deletions(-)



[kafka] branch trunk updated (30bc21c -> 77a46b1)

2020-11-18 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

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


from 30bc21c  KAFKA-9628; Replace Produce request/response with automated 
protocol (#9401)
 add 77a46b1  KAFKA-10545: Create topic IDs in ZooKeeper and Controller 
(#9473)

No new revisions were added by this update.

Summary of changes:
 .../scala/kafka/controller/ControllerContext.scala |  27 -
 .../scala/kafka/controller/KafkaController.scala   |  38 +--
 core/src/main/scala/kafka/zk/AdminZkClient.scala   |   8 +-
 core/src/main/scala/kafka/zk/KafkaZkClient.scala   |  85 ++--
 core/src/main/scala/kafka/zk/ZkData.scala  |  36 ---
 ...PreferredReplicaLeaderElectionCommandTest.scala |   4 +-
 .../controller/ControllerIntegrationTest.scala | 110 -
 .../kafka/security/auth/ZkAuthorizationTest.scala  |   8 +-
 .../scala/unit/kafka/zk/KafkaZkClientTest.scala|  29 +++---
 9 files changed, 290 insertions(+), 55 deletions(-)



[kafka] branch trunk updated (30bc21c -> 77a46b1)

2020-11-18 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

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


from 30bc21c  KAFKA-9628; Replace Produce request/response with automated 
protocol (#9401)
 add 77a46b1  KAFKA-10545: Create topic IDs in ZooKeeper and Controller 
(#9473)

No new revisions were added by this update.

Summary of changes:
 .../scala/kafka/controller/ControllerContext.scala |  27 -
 .../scala/kafka/controller/KafkaController.scala   |  38 +--
 core/src/main/scala/kafka/zk/AdminZkClient.scala   |   8 +-
 core/src/main/scala/kafka/zk/KafkaZkClient.scala   |  85 ++--
 core/src/main/scala/kafka/zk/ZkData.scala  |  36 ---
 ...PreferredReplicaLeaderElectionCommandTest.scala |   4 +-
 .../controller/ControllerIntegrationTest.scala | 110 -
 .../kafka/security/auth/ZkAuthorizationTest.scala  |   8 +-
 .../scala/unit/kafka/zk/KafkaZkClientTest.scala|  29 +++---
 9 files changed, 290 insertions(+), 55 deletions(-)



[kafka] branch trunk updated (30bc21c -> 77a46b1)

2020-11-18 Thread rsivaram
This is an automated email from the ASF dual-hosted git repository.

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


from 30bc21c  KAFKA-9628; Replace Produce request/response with automated 
protocol (#9401)
 add 77a46b1  KAFKA-10545: Create topic IDs in ZooKeeper and Controller 
(#9473)

No new revisions were added by this update.

Summary of changes:
 .../scala/kafka/controller/ControllerContext.scala |  27 -
 .../scala/kafka/controller/KafkaController.scala   |  38 +--
 core/src/main/scala/kafka/zk/AdminZkClient.scala   |   8 +-
 core/src/main/scala/kafka/zk/KafkaZkClient.scala   |  85 ++--
 core/src/main/scala/kafka/zk/ZkData.scala  |  36 ---
 ...PreferredReplicaLeaderElectionCommandTest.scala |   4 +-
 .../controller/ControllerIntegrationTest.scala | 110 -
 .../kafka/security/auth/ZkAuthorizationTest.scala  |   8 +-
 .../scala/unit/kafka/zk/KafkaZkClientTest.scala|  29 +++---
 9 files changed, 290 insertions(+), 55 deletions(-)



[kafka] branch trunk updated (f49c6c2 -> 846e356)

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

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


from f49c6c2  KAFKA-10661; Add new resigned state for graceful 
shutdown/initialization (#9531)
 add 846e356  MINOR: Log resource pattern of ACL updates at INFO level 
(#9578)

No new revisions were added by this update.

Summary of changes:
 core/src/main/scala/kafka/security/authorizer/AclAuthorizer.scala | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)



[kafka] branch trunk updated (f49c6c2 -> 846e356)

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

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


from f49c6c2  KAFKA-10661; Add new resigned state for graceful 
shutdown/initialization (#9531)
 add 846e356  MINOR: Log resource pattern of ACL updates at INFO level 
(#9578)

No new revisions were added by this update.

Summary of changes:
 core/src/main/scala/kafka/security/authorizer/AclAuthorizer.scala | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)



[kafka] branch trunk updated (f49c6c2 -> 846e356)

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

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


from f49c6c2  KAFKA-10661; Add new resigned state for graceful 
shutdown/initialization (#9531)
 add 846e356  MINOR: Log resource pattern of ACL updates at INFO level 
(#9578)

No new revisions were added by this update.

Summary of changes:
 core/src/main/scala/kafka/security/authorizer/AclAuthorizer.scala | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)



  1   2   3   4   5   6   7   8   >