[jira] [Resolved] (KAFKA-16226) Java client: Performance regression in Trogdor benchmark with high partition counts

2024-03-14 Thread Ismael Juma (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-16226?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ismael Juma resolved KAFKA-16226.
-
Resolution: Fixed

> Java client: Performance regression in Trogdor benchmark with high partition 
> counts
> ---
>
> Key: KAFKA-16226
> URL: https://issues.apache.org/jira/browse/KAFKA-16226
> Project: Kafka
>  Issue Type: Bug
>  Components: clients
>Affects Versions: 3.7.0, 3.6.1
>Reporter: Mayank Shekhar Narula
>Assignee: Mayank Shekhar Narula
>Priority: Major
>  Labels: kip-951
> Fix For: 3.6.2, 3.8.0, 3.7.1
>
> Attachments: baseline_lock_profile.png, kafka_15415_lock_profile.png
>
>
> h1. Background
> https://issues.apache.org/jira/browse/KAFKA-15415 implemented optimisation in 
> java-client to skip backoff period if client knows of a newer leader, for 
> produce-batch being retried.
> h1. What changed
> The implementation introduced a regression noticed on a trogdor-benchmark 
> running with high partition counts(36000!).
> With regression, following metrics changed on the produce side.
>  # record-queue-time-avg: increased from 20ms to 30ms.
>  # request-latency-avg: increased from 50ms to 100ms.
> h1. Why it happened
> As can be seen from the original 
> [PR|https://github.com/apache/kafka/pull/14384] 
> RecordAccmulator.partitionReady() & drainBatchesForOneNode() started using 
> synchronised method Metadata.currentLeader(). This has led to increased 
> synchronization between KafkaProducer's application-thread that call send(), 
> and background-thread that actively send producer-batches to leaders.
> Lock profiles clearly show increased synchronisation in KAFKA-15415 
> PR(highlighted in {color:#de350b}Red{color}) Vs baseline ( see below ). Note 
> the synchronisation is much worse for paritionReady() in this benchmark as 
> its called for each partition, and it has 36k partitions!
> h3. Lock Profile: Kafka-15415
> !kafka_15415_lock_profile.png!
> h3. Lock Profile: Baseline
> !baseline_lock_profile.png!
> h1. Fix
> Synchronization has to be reduced between 2 threads in order to address this. 
> [https://github.com/apache/kafka/pull/15323] is a fix for it, as it avoids 
> using Metadata.currentLeader() instead rely on Cluster.leaderFor().
> With the fix, lock-profile & metrics are similar to baseline.
>  



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-16096) Drop broker and tools support for Java 11 in Kafka 4.0 (deprecate in 3.7) (KIP-1013)

2024-01-09 Thread Ismael Juma (Jira)
Ismael Juma created KAFKA-16096:
---

 Summary: Drop broker and tools support for Java 11 in Kafka 4.0 
(deprecate in 3.7) (KIP-1013)
 Key: KAFKA-16096
 URL: https://issues.apache.org/jira/browse/KAFKA-16096
 Project: Kafka
  Issue Type: Improvement
Reporter: Ismael Juma






--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Resolved] (KAFKA-15493) Ensure system tests work with Java 21

2023-12-26 Thread Ismael Juma (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-15493?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ismael Juma resolved KAFKA-15493.
-
Resolution: Fixed

We can mark this as fixed. I ran the system tests with Java 21 and the results 
were the same as Java 17.

> Ensure system tests work with Java 21
> -
>
> Key: KAFKA-15493
> URL: https://issues.apache.org/jira/browse/KAFKA-15493
> Project: Kafka
>  Issue Type: Improvement
>Reporter: Ismael Juma
>Assignee: Said BOUDJELDA
>Priority: Major
> Fix For: 3.7.0
>
> Attachments: image-2023-09-28-02-11-49-196.png, 
> image-2023-09-28-02-12-33-807.png
>
>
> Run the system tests as described below with Java 21:
> [https://github.com/apache/kafka/tree/trunk/tests]
> One relevant portion:
> Run tests with a different JVM (it may be as easy as replacing 11 with 21)
> {code:java}
> bash tests/docker/ducker-ak up -j 'openjdk:11'; 
> tests/docker/run_tests.sh{code}



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-15959) Replace byte handling classes with synchronized methods

2023-12-02 Thread Ismael Juma (Jira)
Ismael Juma created KAFKA-15959:
---

 Summary: Replace byte handling classes with synchronized methods
 Key: KAFKA-15959
 URL: https://issues.apache.org/jira/browse/KAFKA-15959
 Project: Kafka
  Issue Type: Improvement
Reporter: Ismael Juma


The JDK has a number of old byte handling classes that have a number of 
synchronized methods. This wasn't too bad until biased locking was disabled by 
default in Java 17 and removed in Java 21.

The overhead now can be significant if one such method happens to be in the hot 
path. And such overhead is unnecessary if the classes are used by a single 
thread (which is very common).

The classes we should replace:
 # ByteArrayInputStream
 # 
ByteArrayOutputStream
 # 
DataOutputStream
 # BufferedInputStream
 # BufferedOutputStream



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Resolved] (KAFKA-14527) Move `kafka.security` from `core` to separate module

2023-11-25 Thread Ismael Juma (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-14527?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ismael Juma resolved KAFKA-14527.
-
Resolution: Won't Do

Closing as "Won't do" for now. We'll start with KAFKA-15842 and then decide if 
move needs to be done.

> Move `kafka.security` from `core` to separate module
> 
>
> Key: KAFKA-14527
> URL: https://issues.apache.org/jira/browse/KAFKA-14527
> Project: Kafka
>  Issue Type: Improvement
>Reporter: Ismael Juma
>Assignee: Omnia Ibrahim
>Priority: Major
>
> A possible module name would be `server-security`. We should consider moving 
> `StandardAuthorizer` and `org.apache.kafka.server.authorizer.Authorizer` to 
> this module too.
> See KAFKA-14524 for more context.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-15899) Move kafka.security package from core to server module

2023-11-25 Thread Ismael Juma (Jira)
Ismael Juma created KAFKA-15899:
---

 Summary: Move kafka.security package from core to server module
 Key: KAFKA-15899
 URL: https://issues.apache.org/jira/browse/KAFKA-15899
 Project: Kafka
  Issue Type: Sub-task
Reporter: Ismael Juma
Assignee: Omnia Ibrahim






--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Resolved] (KAFKA-14526) Move `kafka.network` from `core` to separate module

2023-11-25 Thread Ismael Juma (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-14526?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ismael Juma resolved KAFKA-14526.
-
Resolution: Won't Do

Marking this as "Won't do" for now in favor of KAFKA-15852. Once that's 
completed, we can figure out if additional modules make sense.

> Move `kafka.network` from `core` to separate module
> ---
>
> Key: KAFKA-14526
> URL: https://issues.apache.org/jira/browse/KAFKA-14526
> Project: Kafka
>  Issue Type: Improvement
>Reporter: Ismael Juma
>Priority: Major
>
> Name to be decided, perhaps `server-network` or something along those lines.
> More context in KAFKA-14524.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Resolved] (KAFKA-14528) Move replication code from `core` to separate module

2023-11-25 Thread Ismael Juma (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-14528?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ismael Juma resolved KAFKA-14528.
-
Resolution: Won't Do

Marking this as "Won't Do" for now in favor of KAFKA-15852. Once that is 
completed, we can look at creating additional modules if that makes sense.

> Move replication code from `core` to separate module
> 
>
> Key: KAFKA-14528
> URL: https://issues.apache.org/jira/browse/KAFKA-14528
> Project: Kafka
>  Issue Type: Improvement
>Reporter: Ismael Juma
>Priority: Major
>
> A potential module name could be `replication`. The relevant classes would be 
> `ReplicaManager`, `*FetcherThread`, etc.
> See KAFKA-14524 for more context.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-15874) Add metric and request log attribute for deprecated request api versions

2023-11-21 Thread Ismael Juma (Jira)
Ismael Juma created KAFKA-15874:
---

 Summary: Add metric and request log attribute for deprecated 
request api versions
 Key: KAFKA-15874
 URL: https://issues.apache.org/jira/browse/KAFKA-15874
 Project: Kafka
  Issue Type: Sub-task
Reporter: Ismael Juma
Assignee: Ismael Juma
 Fix For: 3.7.0






--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Resolved] (KAFKA-15854) Move Java classes from kafka.server to the server module

2023-11-19 Thread Ismael Juma (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-15854?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ismael Juma resolved KAFKA-15854.
-
Fix Version/s: 3.7.0
   Resolution: Fixed

> Move Java classes from kafka.server to the server module
> 
>
> Key: KAFKA-15854
> URL: https://issues.apache.org/jira/browse/KAFKA-15854
> Project: Kafka
>  Issue Type: Sub-task
>Reporter: Ismael Juma
>Assignee: Ismael Juma
>Priority: Major
> Fix For: 3.7.0
>
>




--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Resolved] (KAFKA-14819) Publish a single kafka (aka core) Maven artifact in Apache Kafka 4.0 (KIP-897)

2023-11-19 Thread Ismael Juma (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-14819?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ismael Juma resolved KAFKA-14819.
-
Fix Version/s: (was: 4.0.0)
   Resolution: Won't Fix

As described in the KIP, we're taking a different approach.

> Publish a single kafka (aka core) Maven artifact in Apache Kafka 4.0 (KIP-897)
> --
>
> Key: KAFKA-14819
> URL: https://issues.apache.org/jira/browse/KAFKA-14819
> Project: Kafka
>  Issue Type: Improvement
>Reporter: Ismael Juma
>Assignee: Ismael Juma
>Priority: Blocker
>
> Please see KIP for details:
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-897%3A+Publish+a+single+kafka+%28aka+core%29+Maven+artifact+in+Apache+Kafka+4.0



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-15854) Move Java classes from kafka.server to the server module

2023-11-18 Thread Ismael Juma (Jira)
Ismael Juma created KAFKA-15854:
---

 Summary: Move Java classes from kafka.server to the server module
 Key: KAFKA-15854
 URL: https://issues.apache.org/jira/browse/KAFKA-15854
 Project: Kafka
  Issue Type: Sub-task
Reporter: Ismael Juma






--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-15853) Move KafkaConfig to server module

2023-11-18 Thread Ismael Juma (Jira)
Ismael Juma created KAFKA-15853:
---

 Summary: Move KafkaConfig to server module
 Key: KAFKA-15853
 URL: https://issues.apache.org/jira/browse/KAFKA-15853
 Project: Kafka
  Issue Type: Sub-task
Reporter: Ismael Juma


The server module is a Java-only module, so this also requires converting from 
Scala to Java.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-15852) Move broker code from `core` to `broker` module

2023-11-18 Thread Ismael Juma (Jira)
Ismael Juma created KAFKA-15852:
---

 Summary: Move broker code from `core` to `broker` module
 Key: KAFKA-15852
 URL: https://issues.apache.org/jira/browse/KAFKA-15852
 Project: Kafka
  Issue Type: Bug
Reporter: Ismael Juma


The relevant packages would be `kafka.server`, `kafka.cluster`, etc.

See KAFKA-14524 for more context.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-15623) Migrate remaining tests in streams module to JUnit 5

2023-10-17 Thread Ismael Juma (Jira)
Ismael Juma created KAFKA-15623:
---

 Summary: Migrate remaining tests in streams module to JUnit 5
 Key: KAFKA-15623
 URL: https://issues.apache.org/jira/browse/KAFKA-15623
 Project: Kafka
  Issue Type: Sub-task
Reporter: Ismael Juma






--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Resolved] (KAFKA-12220) Replace PowerMock by Mockito

2023-10-17 Thread Ismael Juma (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-12220?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ismael Juma resolved KAFKA-12220.
-
Resolution: Fixed

> Replace PowerMock by Mockito
> 
>
> Key: KAFKA-12220
> URL: https://issues.apache.org/jira/browse/KAFKA-12220
> Project: Kafka
>  Issue Type: Improvement
>Reporter: Chia-Ping Tsai
>Assignee: Chia-Ping Tsai
>Priority: Major
>
> We are migrating project from junit 4 to junit 5 (KAFKA-7339). PowerMock, 
> however, does not support junit 5 totally 
> (https://github.com/powermock/powermock/issues/830). Hence, we ought to 
> replace PowerMock by Mockito before migrating to junit 5 since rewriting all 
> tests which are depending on PowerMock can bring a bunch of changes.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Reopened] (KAFKA-12220) Replace PowerMock by Mockito

2023-10-17 Thread Ismael Juma (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-12220?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ismael Juma reopened KAFKA-12220:
-

> Replace PowerMock by Mockito
> 
>
> Key: KAFKA-12220
> URL: https://issues.apache.org/jira/browse/KAFKA-12220
> Project: Kafka
>  Issue Type: Improvement
>Reporter: Chia-Ping Tsai
>Assignee: Chia-Ping Tsai
>Priority: Major
>
> We are migrating project from junit 4 to junit 5 (KAFKA-7339). PowerMock, 
> however, does not support junit 5 totally 
> (https://github.com/powermock/powermock/issues/830). Hence, we ought to 
> replace PowerMock by Mockito before migrating to junit 5 since rewriting all 
> tests which are depending on PowerMock can bring a bunch of changes.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Resolved] (KAFKA-12220) Replace PowerMock by Mockito

2023-10-17 Thread Ismael Juma (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-12220?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ismael Juma resolved KAFKA-12220.
-
Resolution: Duplicate

The remaining connect tests that need to be migrated are being tracked via 
https://issues.apache.org/jira/browse/KAFKA-14132.

> Replace PowerMock by Mockito
> 
>
> Key: KAFKA-12220
> URL: https://issues.apache.org/jira/browse/KAFKA-12220
> Project: Kafka
>  Issue Type: Improvement
>Reporter: Chia-Ping Tsai
>Assignee: Chia-Ping Tsai
>Priority: Major
>
> We are migrating project from junit 4 to junit 5 (KAFKA-7339). PowerMock, 
> however, does not support junit 5 totally 
> (https://github.com/powermock/powermock/issues/830). Hence, we ought to 
> replace PowerMock by Mockito before migrating to junit 5 since rewriting all 
> tests which are depending on PowerMock can bring a bunch of changes.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Resolved] (KAFKA-13414) Replace Powermock/EasyMock by Mockito in connect.storage package

2023-10-17 Thread Ismael Juma (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-13414?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ismael Juma resolved KAFKA-13414.
-
Resolution: Fixed

KafkaConfigBackingStoreTest is being tracked via KAFKA-14132, marking this as 
fixed.

> Replace Powermock/EasyMock by Mockito in connect.storage package
> 
>
> Key: KAFKA-13414
> URL: https://issues.apache.org/jira/browse/KAFKA-13414
> Project: Kafka
>  Issue Type: Sub-task
>  Components: KafkaConnect
>Reporter: Mickael Maison
>Assignee: Christo Lolov
>Priority: Major
>




--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Resolved] (KAFKA-7686) Remove PowerMock from Connect Tests

2023-10-17 Thread Ismael Juma (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-7686?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ismael Juma resolved KAFKA-7686.

Resolution: Duplicate

> Remove PowerMock from Connect Tests
> ---
>
> Key: KAFKA-7686
> URL: https://issues.apache.org/jira/browse/KAFKA-7686
> Project: Kafka
>  Issue Type: Task
>  Components: KafkaConnect
>Reporter: Magesh kumar Nandakumar
>Assignee: Magesh kumar Nandakumar
>Priority: Minor
>
> Remove PowerMock from Connect Tests



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-15610) Fix `CoreUtils.swallow()` test gaps

2023-10-16 Thread Ismael Juma (Jira)
Ismael Juma created KAFKA-15610:
---

 Summary: Fix `CoreUtils.swallow()` test gaps
 Key: KAFKA-15610
 URL: https://issues.apache.org/jira/browse/KAFKA-15610
 Project: Kafka
  Issue Type: Bug
Reporter: Ismael Juma


For example, it should verify that the passed in `logging` is used in case of 
an exception. We found that there is no test for this in 
https://github.com/apache/kafka/pull/14529#discussion_r1355277747.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-15599) Move KafkaMetadataLog and RaftManager to metadata module

2023-10-12 Thread Ismael Juma (Jira)
Ismael Juma created KAFKA-15599:
---

 Summary: Move KafkaMetadataLog and RaftManager to metadata module
 Key: KAFKA-15599
 URL: https://issues.apache.org/jira/browse/KAFKA-15599
 Project: Kafka
  Issue Type: Sub-task
Reporter: Ismael Juma
Assignee: Ismael Juma






--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Resolved] (KAFKA-15485) Support building with Java 21 (LTS release)

2023-09-25 Thread Ismael Juma (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-15485?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ismael Juma resolved KAFKA-15485.
-
Resolution: Fixed

> Support building with Java 21 (LTS release)
> ---
>
> Key: KAFKA-15485
> URL: https://issues.apache.org/jira/browse/KAFKA-15485
> Project: Kafka
>  Issue Type: Improvement
>Reporter: Divij Vaidya
>Assignee: Ismael Juma
>Priority: Major
> Fix For: 3.7.0
>
>
> JDK-21 is the latest LTS release which reached GA on 19th Sept 2023. This 
> ticket aims to upgrade JDK used by Kafka to JDK-21 (currently it's JDK20).
> Thanks to proactive work done by [~ijuma] earlier [1][2][3], I do not 
> anticipate major hiccups while upgrading to JDK-21.
> As part of this JIRA we want to:
> 1. Upgrade Kafka to JDK 21
> 2. Replace the CI build for JDK 20 with JDK 21 (similar to [3] below)
> 3. Update the README (see[4]) to mention Kafka's support for JDK-21
> [1] [https://github.com/apache/kafka/pull/13840]
> [2] [https://github.com/apache/kafka/pull/13582]
> [3] [https://github.com/apache/kafka/pull/12948] 
> [4] [https://github.com/apache/kafka/pull/14061] 



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-15493) Ensure system tests work with Java 21

2023-09-24 Thread Ismael Juma (Jira)
Ismael Juma created KAFKA-15493:
---

 Summary: Ensure system tests work with Java 21
 Key: KAFKA-15493
 URL: https://issues.apache.org/jira/browse/KAFKA-15493
 Project: Kafka
  Issue Type: Improvement
Reporter: Ismael Juma
 Fix For: 3.7.0






--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-15492) Enable spotbugs when building with Java 21

2023-09-24 Thread Ismael Juma (Jira)
Ismael Juma created KAFKA-15492:
---

 Summary: Enable spotbugs when building with Java 21
 Key: KAFKA-15492
 URL: https://issues.apache.org/jira/browse/KAFKA-15492
 Project: Kafka
  Issue Type: Improvement
Reporter: Ismael Juma


The latest version of spotbugs (4.7.3) doesn't support Java 21. In order not to 
delay Java 21 support, we disabled spotbugs when building with Java 21. This 
should be reverted once we upgrade to a version of spotbugs that supports Java 
21.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-15117) SslTransportLayerTest.testValidEndpointIdentificationCN fails with Java 20

2023-06-23 Thread Ismael Juma (Jira)
Ismael Juma created KAFKA-15117:
---

 Summary: SslTransportLayerTest.testValidEndpointIdentificationCN 
fails with Java 20
 Key: KAFKA-15117
 URL: https://issues.apache.org/jira/browse/KAFKA-15117
 Project: Kafka
  Issue Type: Bug
Reporter: Ismael Juma


 

All variations fail as seen below. These tests have been disabled when run with 
Java 20 for now.
{code:java}
Gradle Test Run :clients:test > Gradle Test Executor 12 > SslTransportLayerTest 
> testValidEndpointIdentificationCN(Args) > [1] tlsProtocol=TLSv1.2, 
useInlinePem=false FAILED
    org.opentest4j.AssertionFailedError: Channel 0 was not ready after 30 
seconds ==> expected:  but was: 
        at 
app//org.junit.jupiter.api.AssertionFailureBuilder.build(AssertionFailureBuilder.java:151)
        at 
app//org.junit.jupiter.api.AssertionFailureBuilder.buildAndThrow(AssertionFailureBuilder.java:132)
        at app//org.junit.jupiter.api.AssertTrue.failNotTrue(AssertTrue.java:63)
        at app//org.junit.jupiter.api.AssertTrue.assertTrue(AssertTrue.java:36)
        at app//org.junit.jupiter.api.Assertions.assertTrue(Assertions.java:211)
        at 
app//org.apache.kafka.common.network.NetworkTestUtils.waitForChannelReady(NetworkTestUtils.java:107)
        at 
app//org.apache.kafka.common.network.NetworkTestUtils.checkClientConnection(NetworkTestUtils.java:70)
        at 
app//org.apache.kafka.common.network.SslTransportLayerTest.verifySslConfigs(SslTransportLayerTest.java:1296)
        at 
app//org.apache.kafka.common.network.SslTransportLayerTest.testValidEndpointIdentificationCN(SslTransportLayerTest.java:202)
org.apache.kafka.common.network.SslTransportLayerTest.testValidEndpointIdentificationCN(Args)[2]
 failed, log available in 
/home/ijuma/src/kafka/clients/build/reports/testOutput/org.apache.kafka.common.network.SslTransportLayerTest.testValidEndpointIdentificationCN(Args)[2].test.stdout
Gradle Test Run :clients:test > Gradle Test Executor 12 > SslTransportLayerTest 
> testValidEndpointIdentificationCN(Args) > [2] tlsProtocol=TLSv1.2, 
useInlinePem=true FAILED
    org.opentest4j.AssertionFailedError: Channel 0 was not ready after 30 
seconds ==> expected:  but was: 
        at 
app//org.junit.jupiter.api.AssertionFailureBuilder.build(AssertionFailureBuilder.java:151)
        at 
app//org.junit.jupiter.api.AssertionFailureBuilder.buildAndThrow(AssertionFailureBuilder.java:132)
        at app//org.junit.jupiter.api.AssertTrue.failNotTrue(AssertTrue.java:63)
        at app//org.junit.jupiter.api.AssertTrue.assertTrue(AssertTrue.java:36)
        at app//org.junit.jupiter.api.Assertions.assertTrue(Assertions.java:211)
        at 
app//org.apache.kafka.common.network.NetworkTestUtils.waitForChannelReady(NetworkTestUtils.java:107)
        at 
app//org.apache.kafka.common.network.NetworkTestUtils.checkClientConnection(NetworkTestUtils.java:70)
        at 
app//org.apache.kafka.common.network.SslTransportLayerTest.verifySslConfigs(SslTransportLayerTest.java:1296)
        at 
app//org.apache.kafka.common.network.SslTransportLayerTest.testValidEndpointIdentificationCN(SslTransportLayerTest.java:202)
org.apache.kafka.common.network.SslTransportLayerTest.testValidEndpointIdentificationCN(Args)[3]
 failed, log available in 
/home/ijuma/src/kafka/clients/build/reports/testOutput/org.apache.kafka.common.network.SslTransportLayerTest.testValidEndpointIdentificationCN(Args)[3].test.stdout
Gradle Test Run :clients:test > Gradle Test Executor 12 > SslTransportLayerTest 
> testValidEndpointIdentificationCN(Args) > [3] tlsProtocol=TLSv1.3, 
useInlinePem=false FAILED
    org.opentest4j.AssertionFailedError: Channel 0 was not ready after 30 
seconds ==> expected:  but was: 
        at 
app//org.junit.jupiter.api.AssertionFailureBuilder.build(AssertionFailureBuilder.java:151)
        at 
app//org.junit.jupiter.api.AssertionFailureBuilder.buildAndThrow(AssertionFailureBuilder.java:132)
        at app//org.junit.jupiter.api.AssertTrue.failNotTrue(AssertTrue.java:63)
        at app//org.junit.jupiter.api.AssertTrue.assertTrue(AssertTrue.java:36)
        at app//org.junit.jupiter.api.Assertions.assertTrue(Assertions.java:211)
        at 
app//org.apache.kafka.common.network.NetworkTestUtils.waitForChannelReady(NetworkTestUtils.java:107)
        at 
app//org.apache.kafka.common.network.NetworkTestUtils.checkClientConnection(NetworkTestUtils.java:70)
        at 
app//org.apache.kafka.common.network.SslTransportLayerTest.verifySslConfigs(SslTransportLayerTest.java:1296)
        at 
app//org.apache.kafka.common.network.SslTransportLayerTest.testValidEndpointIdentificationCN(SslTransportLayerTest.java:202)
{code}
 



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-14892) Harmonize package names in storage module

2023-04-11 Thread Ismael Juma (Jira)
Ismael Juma created KAFKA-14892:
---

 Summary: Harmonize package names in storage module
 Key: KAFKA-14892
 URL: https://issues.apache.org/jira/browse/KAFKA-14892
 Project: Kafka
  Issue Type: Improvement
Reporter: Ismael Juma


We currently have:
 # org.apache.kafka.server.log.remote.storage: public api in storage-api module
 # org.apache.kafka.server.log.remote: private api in storage module
 # org.apache.kafka.storage.internals.log: private api in storage module

A way to make this consistent could be:
 # org.apache.kafka.storage.* or org.apache.kafka.storage.api.*: public api in 
storage-api module
 # org.apache.kafka.storage.internals.log.remote: private api in storage module
 # org.apache.kafka.storage.internals.log: private api in storage module (stays 
the same)



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-14819) Publish a single kafka (aka core) Maven artifact in Apache Kafka 4.0 (KIP-897)

2023-03-18 Thread Ismael Juma (Jira)
Ismael Juma created KAFKA-14819:
---

 Summary: Publish a single kafka (aka core) Maven artifact in 
Apache Kafka 4.0 (KIP-897)
 Key: KAFKA-14819
 URL: https://issues.apache.org/jira/browse/KAFKA-14819
 Project: Kafka
  Issue Type: Improvement
Reporter: Ismael Juma
Assignee: Ismael Juma
 Fix For: 4.0.0


Please see KIP for details:

https://cwiki.apache.org/confluence/display/KAFKA/KIP-897%3A+Publish+a+single+kafka+%28aka+core%29+Maven+artifact+in+Apache+Kafka+4.0



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Resolved] (KAFKA-13900) Support Java 9 direct ByteBuffer Checksum methods

2023-03-17 Thread Ismael Juma (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-13900?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ismael Juma resolved KAFKA-13900.
-
Resolution: Fixed

> Support Java 9 direct ByteBuffer Checksum methods
> -
>
> Key: KAFKA-13900
> URL: https://issues.apache.org/jira/browse/KAFKA-13900
> Project: Kafka
>  Issue Type: Improvement
>  Components: clients, core
>Affects Versions: 3.1.1
>Reporter: Francesco Nigro
>Priority: Minor
>  Labels: performance, performance-benchmark
> Fix For: 3.4.0
>
>
> Java 9 has added a new Checksum method that can makes uses of ByteBuffer(s) 
> (see [Java 9's 
> Checksum::update|https://docs.oracle.com/javase/9/docs/api/java/util/zip/Checksum.html#update-java.nio.ByteBuffer-]):
>  Kafka already provides specific support for Java 9's Cr32C, hence it makes 
> sense it's going to use the most optimized version of it for direct 
> ByteBuffers as well (read-only or not), instead of performing a byte-per-byte 
> computation.
>  
> I'm aware that currently the client's Buffer pools aren't using direct 
> ByteBuffer, but having full support for it can open the door to future 
> interesting optimizations on it.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Resolved] (KAFKA-14792) Race condition in LazyIndex.get()

2023-03-07 Thread Ismael Juma (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-14792?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ismael Juma resolved KAFKA-14792.
-
Fix Version/s: 3.5.0
   Resolution: Fixed

> Race condition in LazyIndex.get()
> -
>
> Key: KAFKA-14792
> URL: https://issues.apache.org/jira/browse/KAFKA-14792
> Project: Kafka
>  Issue Type: Bug
>Reporter: Ismael Juma
>Assignee: Ismael Juma
>Priority: Blocker
> Fix For: 3.5.0
>
>
> `LazyIndex.get()` has a race condition that can result in a 
> ClassCastException being thrown in some cases.
> This was introduced when it was rewritten from Scala to Java.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-14792) Race condition in LazyIndex.get()

2023-03-07 Thread Ismael Juma (Jira)
Ismael Juma created KAFKA-14792:
---

 Summary: Race condition in LazyIndex.get()
 Key: KAFKA-14792
 URL: https://issues.apache.org/jira/browse/KAFKA-14792
 Project: Kafka
  Issue Type: Bug
Reporter: Ismael Juma
Assignee: Ismael Juma


`LazyIndex.get()` has a race condition that can result in a ClassCastException 
being thrown at times.

This was introduced when it was rewritten from Scala to Java.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Resolved] (KAFKA-14680) Gradle version upgrade 7 -->> 8

2023-02-27 Thread Ismael Juma (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-14680?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ismael Juma resolved KAFKA-14680.
-
Fix Version/s: 3.5.0
   Resolution: Fixed

> Gradle version upgrade 7 -->> 8
> ---
>
> Key: KAFKA-14680
> URL: https://issues.apache.org/jira/browse/KAFKA-14680
> Project: Kafka
>  Issue Type: Improvement
>  Components: build
>Reporter: Dejan Stojadinović
>Assignee: Dejan Stojadinović
>Priority: Major
> Fix For: 3.5.0
>
>
> +*Gradle 8 release notes:*+
>  * *8.0*
>  ** [https://github.com/gradle/gradle/releases/tag/v8.0.0]
>  ** 
> [https://docs.gradle.org/8.0/release-notes.html|https://docs.gradle.org/8.0-rc-3/release-notes.html]
>  *  *8.0.1:*
>  ** [https://github.com/gradle/gradle/releases/tag/v8.0.1]
>  ** [https://docs.gradle.org/8.0.1/release-notes.html]
>  ** [https://github.com/gradle/gradle/milestone/229?closed=1]
> *Upgrade notes:* 
> [https://docs.gradle.org/8.0/userguide/upgrading_version_7.html#changes_8.0|https://docs.gradle.org/8.0/userguide/upgrading_version_7.html#changes_8.0]



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Resolved] (KAFKA-14749) Re-enable 'spotlessScalaCheck' task (in Jenkinsfile)

2023-02-27 Thread Ismael Juma (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-14749?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ismael Juma resolved KAFKA-14749.
-
  Assignee: Ismael Juma
Resolution: Fixed

We found a way to re-enable this when running with Java 11 or newer, see 
https://github.com/apache/kafka/pull/13311.

> Re-enable 'spotlessScalaCheck' task (in Jenkinsfile)
> 
>
> Key: KAFKA-14749
> URL: https://issues.apache.org/jira/browse/KAFKA-14749
> Project: Kafka
>  Issue Type: Task
>  Components: build
>Reporter: Dejan Stojadinović
>Assignee: Ismael Juma
>Priority: Blocker
> Fix For: 3.5.0
>
>
> {*}Description{*}:
> We were forced to remove 'spotlessScalaCheck' (see KAFKA-14728) but we should 
> bring it back when circumstances change (i.e. when Apache Kafka 4.0 drops 
> support for Java 8).
> Related github issue comment: 
> [https://github.com/apache/kafka/pull/13263#issuecomment-1441825913]
>  
>  
>  
>  



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Resolved] (KAFKA-14688) Move org.apache.kafka.server.log.internals to org.apache.kafka.storage.internals.log

2023-02-10 Thread Ismael Juma (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-14688?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ismael Juma resolved KAFKA-14688.
-
Resolution: Fixed

> Move org.apache.kafka.server.log.internals to 
> org.apache.kafka.storage.internals.log
> 
>
> Key: KAFKA-14688
> URL: https://issues.apache.org/jira/browse/KAFKA-14688
> Project: Kafka
>  Issue Type: Sub-task
>Reporter: Satish Duggana
>Assignee: Satish Duggana
>Priority: Major
> Fix For: 3.5.0
>
>
> This is a followup item from https://github.com/apache/kafka/pull/13046 with 
> the 
> [comment|https://github.com/apache/kafka/pull/13046#discussion_r1063953030]



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Resolved] (KAFKA-13663) IllegalMonitorStateException in ProducerMetadata.awaitUpdate

2023-01-21 Thread Ismael Juma (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-13663?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ismael Juma resolved KAFKA-13663.
-
Resolution: Duplicate

Marking as duplicate of KAFKA-10902 since that has a reference to the JDK bug.

> IllegalMonitorStateException in ProducerMetadata.awaitUpdate
> 
>
> Key: KAFKA-13663
> URL: https://issues.apache.org/jira/browse/KAFKA-13663
> Project: Kafka
>  Issue Type: Bug
>  Components: clients
>Affects Versions: 3.0.0
>Reporter: RivenSun
>Assignee: Guozhang Wang
>Priority: Major
>
> Since our service kafka-clients was upgraded to {*}2.8.1{*}, sometimes the 
> producer will throw an exception when sending a message, and once this 
> exception occurs, it will continue and will not recover by itself.
> {code:java}
> java.lang.IllegalMonitorStateException: current thread is not owner
>         at java.base/java.lang.Object.wait(Native Method)
>         at 
> org.apache.kafka.common.utils.SystemTime.waitObject(SystemTime.java:55)
>         at 
> org.apache.kafka.clients.producer.internals.ProducerMetadata.awaitUpdate(ProducerMetadata.java:119)
>         at 
> org.apache.kafka.clients.producer.KafkaProducer.waitOnMetadata(KafkaProducer.java:1047)
>         at 
> org.apache.kafka.clients.producer.KafkaProducer.doSend(KafkaProducer.java:906)
>         at 
> org.apache.kafka.clients.producer.KafkaProducer.send(KafkaProducer.java:885)
>  {code}
>  
>  
> The version of kafka-clients used before is {*}2.2.2{*}, and this exception 
> has never occurred
> Comparing the changes in the kafkaClient source code, I found that there is a 
> change, it is very likely that this is a *regression bug.*
> In version 2.2.2, the `Metadata#awaitUpdate` method is called in 
> `KafkaProducer#waitOnMetadata`
> But since version {*}2.3.0{*}, `ProducerMetadata#awaitUpdate` is called in 
> `KafkaProducer#waitOnMetadata`
> The most crucial point is:
> `Object#wait(long timeoutMillis)` method,
> In the `Metadata#awaitUpdate` method, thread safety is ensured by the 
> `synchronized` keyword on the `Metadata#awaitUpdate` method;
> In the `ProducerMetadata#awaitUpdate` method, in addition to the 
> `synchronized` on the `ProducerMetadata#awaitUpdate` method, 
> {color:#FF}*there is a second `synchronized` in the 
> `SystemTime#waitObject` method;*{color}
> Although we all know that `synchronized` is reentrant, it is not clear 
> whether the implementation of `ProducerMetadata#awaitUpdate` is inconsistent 
> in different versions of JDK, resulting in this exception; what is certain is 
> that this exception has never occurred before the 2.2.2 version of 
> KafkaProducer
>  
> h2. Suggestion:
> We can keep only one `synchronized` and remove the `synchronized` on the 
> `ProducerMetadata#awaitUpdate` method



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Resolved] (KAFKA-10637) KafkaProducer: IllegalMonitorStateException

2023-01-21 Thread Ismael Juma (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-10637?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ismael Juma resolved KAFKA-10637.
-
Resolution: Duplicate

Marking as duplicate of KAFKA-10902 since that has a reference to the JDK bug.

> KafkaProducer: IllegalMonitorStateException 
> 
>
> Key: KAFKA-10637
> URL: https://issues.apache.org/jira/browse/KAFKA-10637
> Project: Kafka
>  Issue Type: Bug
>  Components: producer 
>Affects Versions: 2.5.1
>Reporter: Lefteris Katiforis
>Priority: Major
>
> Kafka producer throws the following exception:
> {code:java}
> {\"log\":\"java.lang.IllegalMonitorStateException: current thread is not 
> owner\\n\",\"stream\":\"stdout\",\"time\":\"2020-10-23T12:48:50.415014714Z\"}"}
>  java.base/java.lang.Object.wait(Native 
> Method)\\n\",\"stream\":\"stdout\",\"time\":\"2020-10-23T12:48:50.41502027Z\"}"}
> org.apache.kafka.common.utils.SystemTime.waitObject(SystemTime.java:55)\\n\",\"stream\":\"stdout\",\"time\":\"2020-10-23T12:48:50.415024923Z\"}"}
> at 
> org.apache.kafka.clients.producer.internals.ProducerMetadata.awaitUpdate(ProducerMetadata.java:119)\\n\",\"stream\":\"stdout\",\"time\":\"2020-10-23T12:48:50.415029863Z\"}"}
> org.apache.kafka.clients.producer.KafkaProducer.waitOnMetadata(KafkaProducer.java:1029)\\n\",\"stream\":\"stdout\",\"time\":\"2020-10-23T12:48:50.415034336Z\"}"}
> org.apache.kafka.clients.producer.KafkaProducer.doSend(KafkaProducer.java:883)\\n\",\"stream\":\"stdout\",\"time\":\"2020-10-23T12:48:50.415038722Z\"}"}
> org.apache.kafka.clients.producer.KafkaProducer.send(KafkaProducer.java:862)\\n\",\"stream\":\"stdout\",\"time\":\"2020-10-23T12:48:50.415042939Z\"}"}
> org.springframework.kafka.core.DefaultKafkaProducerFactory$CloseSafeProducer.send(DefaultKafkaProducerFactory.java:781)\\n\",\"stream\":\"stdout\",\"time\":\"2020-10-23T12:48:50.415047238Z\"}"}
> org.springframework.kafka.core.KafkaTemplate.doSend(KafkaTemplate.java:562)\\n\",\"stream\":\"stdout\",\"time\":\"2020-10-23T12:48:50.415051555Z\"}"}
> org.springframework.kafka.core.KafkaTemplate.send(KafkaTemplate.java:369)\\n\",\"stream\":\"stdout\",\"time\":\"2020-10-23T12:48:50.415055882Z\"}"}{code}



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Resolved] (KAFKA-5841) Open old index files with read-only permission

2023-01-15 Thread Ismael Juma (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-5841?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ismael Juma resolved KAFKA-5841.

Resolution: Not A Problem

We have a constructor parameter for this, so I don't think we need this method.

> Open old index files with read-only permission
> --
>
> Key: KAFKA-5841
> URL: https://issues.apache.org/jira/browse/KAFKA-5841
> Project: Kafka
>  Issue Type: Improvement
>Reporter: Jason Gustafson
>Assignee: huxihx
>Priority: Major
>
> Since old index files do not change, we may as well drop the write permission 
> needed when opening them. From the doc comments in {{OffsetIndex}}, it sounds 
> like we may have had this implemented at one point:
> {code}
>  * Index files can be opened in two ways: either as an empty, mutable index 
> that allows appends or
>  * an immutable read-only index file that has previously been populated. The 
> makeReadOnly method will turn a mutable file into an 
>  * immutable one and truncate off any extra bytes. This is done when the 
> index file is rolled over.
> {code}
> So we should either support this or (if there is good reason not to) update 
> the comment.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Resolved] (KAFKA-14568) Move FetchDataInfo and related to storage module

2023-01-12 Thread Ismael Juma (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-14568?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ismael Juma resolved KAFKA-14568.
-
Resolution: Fixed

> Move FetchDataInfo and related to storage module
> 
>
> Key: KAFKA-14568
> URL: https://issues.apache.org/jira/browse/KAFKA-14568
> Project: Kafka
>  Issue Type: Sub-task
>Reporter: Federico Valeri
>Assignee: Federico Valeri
>Priority: Major
> Fix For: 3.5.0
>
>




--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-14607) Move Scheduler/KafkaScheduler to server-common

2023-01-07 Thread Ismael Juma (Jira)
Ismael Juma created KAFKA-14607:
---

 Summary: Move Scheduler/KafkaScheduler to server-common
 Key: KAFKA-14607
 URL: https://issues.apache.org/jira/browse/KAFKA-14607
 Project: Kafka
  Issue Type: Sub-task
Reporter: Ismael Juma
Assignee: Ismael Juma
 Fix For: 3.5.0






--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Resolved] (KAFKA-14571) ZkMetadataCache.getClusterMetadata is missing rack information in aliveNodes

2023-01-04 Thread Ismael Juma (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-14571?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ismael Juma resolved KAFKA-14571.
-
Resolution: Fixed

> ZkMetadataCache.getClusterMetadata is missing rack information in aliveNodes
> 
>
> Key: KAFKA-14571
> URL: https://issues.apache.org/jira/browse/KAFKA-14571
> Project: Kafka
>  Issue Type: Bug
>  Components: core
>Affects Versions: 3.3
>Reporter: Edoardo Comar
>Assignee: Edoardo Comar
>Priority: Minor
> Fix For: 3.4.0, 3.3.3
>
>
> ZkMetadataCache.getClusterMetadata returns a Cluster object where the 
> aliveNodes are missing their rack info.
> when ZkMetadataCache updates the metadataSnapshot, includes the rack in 
> `aliveBrokers` but not in `aliveNodes` 



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-14560) Remove old client protocol API versions in Kafka 4.0 (KIP-896)

2023-01-03 Thread Ismael Juma (Jira)
Ismael Juma created KAFKA-14560:
---

 Summary: Remove old client protocol API versions in Kafka 4.0 
(KIP-896)
 Key: KAFKA-14560
 URL: https://issues.apache.org/jira/browse/KAFKA-14560
 Project: Kafka
  Issue Type: Improvement
Reporter: Ismael Juma
Assignee: Ismael Juma


Please see KIP for details:

https://cwiki.apache.org/confluence/display/KAFKA/KIP-896%3A+Remove+old+client+protocol+API+versions+in+Kafka+4.0



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-14552) Remove no longer required server protocol versions in Kafka 4.0

2022-12-24 Thread Ismael Juma (Jira)
Ismael Juma created KAFKA-14552:
---

 Summary: Remove no longer required server protocol versions in 
Kafka 4.0
 Key: KAFKA-14552
 URL: https://issues.apache.org/jira/browse/KAFKA-14552
 Project: Kafka
  Issue Type: Improvement
Reporter: Ismael Juma
 Fix For: 4.0.0


Kafka 4.0 will remove support for zk mode and kraft mode became production 
ready in Kafka 3.3. Furthermore, migration from zk mode to kraft mode will 
require upgrading to the bridge release first (likely 3.5, but could also be 
3.6).

This provides an opportunity to remove exclusively server side protocols 
versions that only exist to allow direct upgrades from versions older than 3.n 
where n is either 0 (KRaft preview), 3 (KRaft production ready) or 5 (bridge 
release). We should decide on the right `n`.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Resolved] (KAFKA-14479) Move CleanerConfig to storage module

2022-12-21 Thread Ismael Juma (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-14479?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ismael Juma resolved KAFKA-14479.
-
Resolution: Duplicate

Doing it as part of KAFKA-14478 instead.

> Move CleanerConfig to storage module
> 
>
> Key: KAFKA-14479
> URL: https://issues.apache.org/jira/browse/KAFKA-14479
> Project: Kafka
>  Issue Type: Sub-task
>Reporter: Ismael Juma
>Assignee: Ismael Juma
>Priority: Major
>




--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Resolved] (KAFKA-14466) Refactor ClassLoaderAwareRemoteStorageManager.scala to ClassLoaderAwareRemoteStorageManager.java and move it to storage module.

2022-12-19 Thread Ismael Juma (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-14466?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ismael Juma resolved KAFKA-14466.
-
  Reviewer: Ismael Juma
Resolution: Fixed

> Refactor ClassLoaderAwareRemoteStorageManager.scala to  
> ClassLoaderAwareRemoteStorageManager.java and move it to storage module.
> 
>
> Key: KAFKA-14466
> URL: https://issues.apache.org/jira/browse/KAFKA-14466
> Project: Kafka
>  Issue Type: Sub-task
>Reporter: Satish Duggana
>Assignee: Satish Duggana
>Priority: Major
> Fix For: 3.5.0
>
>
> https://github.com/apache/kafka/pull/11390#discussion_r1043982906



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-14528) Move replication code from `core` to separate module

2022-12-17 Thread Ismael Juma (Jira)
Ismael Juma created KAFKA-14528:
---

 Summary: Move replication code from `core` to separate module
 Key: KAFKA-14528
 URL: https://issues.apache.org/jira/browse/KAFKA-14528
 Project: Kafka
  Issue Type: Improvement
Reporter: Ismael Juma


A potential module name could be `replication`. The relevant classes would be 
`ReplicaManager`, `*FetcherThread`, etc.

See KAFKA-14524 for more context.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-14527) Move `kafka.security` to separate module

2022-12-17 Thread Ismael Juma (Jira)
Ismael Juma created KAFKA-14527:
---

 Summary: Move `kafka.security` to separate module
 Key: KAFKA-14527
 URL: https://issues.apache.org/jira/browse/KAFKA-14527
 Project: Kafka
  Issue Type: Improvement
Reporter: Ismael Juma


A possible module name would be `server-security`.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-14526) Move `kafka.network` from `core` to separate module

2022-12-17 Thread Ismael Juma (Jira)
Ismael Juma created KAFKA-14526:
---

 Summary: Move `kafka.network` from `core` to separate module
 Key: KAFKA-14526
 URL: https://issues.apache.org/jira/browse/KAFKA-14526
 Project: Kafka
  Issue Type: Improvement
Reporter: Ismael Juma


Name to be decided, perhaps `server-network` or something along those lines.

More context in KAFKA-14524.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-14525) Move CLI tools from `core` to `tools` module

2022-12-17 Thread Ismael Juma (Jira)
Ismael Juma created KAFKA-14525:
---

 Summary: Move CLI tools from `core` to `tools` module
 Key: KAFKA-14525
 URL: https://issues.apache.org/jira/browse/KAFKA-14525
 Project: Kafka
  Issue Type: Improvement
Reporter: Ismael Juma


More specifically, tools that don't require access to `core` classes and 
communicate via the kafka protocol (typically by using the client classes) 
should be moved to the `tools` module.

See KAFKA-14524 for more context.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-14524) Modularize `core` monolith

2022-12-17 Thread Ismael Juma (Jira)
Ismael Juma created KAFKA-14524:
---

 Summary: Modularize `core` monolith
 Key: KAFKA-14524
 URL: https://issues.apache.org/jira/browse/KAFKA-14524
 Project: Kafka
  Issue Type: Improvement
Reporter: Ismael Juma






--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-14523) Move RemoteLogManager to the storage module

2022-12-17 Thread Ismael Juma (Jira)
Ismael Juma created KAFKA-14523:
---

 Summary: Move RemoteLogManager to the storage module
 Key: KAFKA-14523
 URL: https://issues.apache.org/jira/browse/KAFKA-14523
 Project: Kafka
  Issue Type: Sub-task
Reporter: Ismael Juma






--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-14522) Move RemoteIndexCache and ClassLoaderAwareRemoteStorageManager to the storage module

2022-12-17 Thread Ismael Juma (Jira)
Ismael Juma created KAFKA-14522:
---

 Summary: Move RemoteIndexCache and 
ClassLoaderAwareRemoteStorageManager to the storage module
 Key: KAFKA-14522
 URL: https://issues.apache.org/jira/browse/KAFKA-14522
 Project: Kafka
  Issue Type: Sub-task
Reporter: Ismael Juma






--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-14521) Replace BrokerCompressionCodec with BrokerCompressionType

2022-12-17 Thread Ismael Juma (Jira)
Ismael Juma created KAFKA-14521:
---

 Summary: Replace BrokerCompressionCodec with BrokerCompressionType
 Key: KAFKA-14521
 URL: https://issues.apache.org/jira/browse/KAFKA-14521
 Project: Kafka
  Issue Type: Sub-task
Reporter: Ismael Juma
Assignee: Ismael Juma


This a requirement to move `LogValidator` to the `storage` module.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-14490) Consider using UncheckdIOException instead of IOException in the log layer

2022-12-14 Thread Ismael Juma (Jira)
Ismael Juma created KAFKA-14490:
---

 Summary: Consider using UncheckdIOException instead of IOException 
in the log layer
 Key: KAFKA-14490
 URL: https://issues.apache.org/jira/browse/KAFKA-14490
 Project: Kafka
  Issue Type: Sub-task
Reporter: Ismael Juma


IOException is a checked exception, which makes it difficult to use with 
lambdas. We should consider using UncheckdIOException instead.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-14489) Adjust visibility of classes moved to storage module

2022-12-14 Thread Ismael Juma (Jira)
Ismael Juma created KAFKA-14489:
---

 Summary: Adjust visibility of classes moved to storage module
 Key: KAFKA-14489
 URL: https://issues.apache.org/jira/browse/KAFKA-14489
 Project: Kafka
  Issue Type: Sub-task
Reporter: Ismael Juma


Once the log layer has been completely migrated to the storage module, we 
should adjust the visibility of classes that are only used within the log layer 
to be package private.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-14487) Move LogManager to storage module

2022-12-14 Thread Ismael Juma (Jira)
Ismael Juma created KAFKA-14487:
---

 Summary: Move LogManager to storage module
 Key: KAFKA-14487
 URL: https://issues.apache.org/jira/browse/KAFKA-14487
 Project: Kafka
  Issue Type: Sub-task
Reporter: Ismael Juma






--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-14488) Move log layer tests to storage module

2022-12-14 Thread Ismael Juma (Jira)
Ismael Juma created KAFKA-14488:
---

 Summary: Move log layer tests to storage module
 Key: KAFKA-14488
 URL: https://issues.apache.org/jira/browse/KAFKA-14488
 Project: Kafka
  Issue Type: Sub-task
Reporter: Ismael Juma


This should be split into multiple tasks.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-14486) Move LogCleanerManager to storage module

2022-12-14 Thread Ismael Juma (Jira)
Ismael Juma created KAFKA-14486:
---

 Summary: Move LogCleanerManager to storage module
 Key: KAFKA-14486
 URL: https://issues.apache.org/jira/browse/KAFKA-14486
 Project: Kafka
  Issue Type: Sub-task
Reporter: Ismael Juma






--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-14485) Move LogCleaner to storage module

2022-12-14 Thread Ismael Juma (Jira)
Ismael Juma created KAFKA-14485:
---

 Summary: Move LogCleaner to storage module
 Key: KAFKA-14485
 URL: https://issues.apache.org/jira/browse/KAFKA-14485
 Project: Kafka
  Issue Type: Sub-task
Reporter: Ismael Juma






--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-14484) Move UnifiedLog to storage module

2022-12-14 Thread Ismael Juma (Jira)
Ismael Juma created KAFKA-14484:
---

 Summary: Move UnifiedLog to storage module
 Key: KAFKA-14484
 URL: https://issues.apache.org/jira/browse/KAFKA-14484
 Project: Kafka
  Issue Type: Sub-task
Reporter: Ismael Juma






--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-14483) Move LocalLog to storage module

2022-12-14 Thread Ismael Juma (Jira)
Ismael Juma created KAFKA-14483:
---

 Summary: Move LocalLog to storage module
 Key: KAFKA-14483
 URL: https://issues.apache.org/jira/browse/KAFKA-14483
 Project: Kafka
  Issue Type: Sub-task
Reporter: Ismael Juma






--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-14482) Move LogLoader to storage module

2022-12-14 Thread Ismael Juma (Jira)
Ismael Juma created KAFKA-14482:
---

 Summary: Move LogLoader to storage module
 Key: KAFKA-14482
 URL: https://issues.apache.org/jira/browse/KAFKA-14482
 Project: Kafka
  Issue Type: Sub-task
Reporter: Ismael Juma






--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-14481) Move LogSegment/LogSegments to storage

2022-12-14 Thread Ismael Juma (Jira)
Ismael Juma created KAFKA-14481:
---

 Summary: Move LogSegment/LogSegments to storage
 Key: KAFKA-14481
 URL: https://issues.apache.org/jira/browse/KAFKA-14481
 Project: Kafka
  Issue Type: Sub-task
Reporter: Ismael Juma






--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-14480) Move ProducerStateManager to storage module

2022-12-14 Thread Ismael Juma (Jira)
Ismael Juma created KAFKA-14480:
---

 Summary: Move ProducerStateManager to storage module
 Key: KAFKA-14480
 URL: https://issues.apache.org/jira/browse/KAFKA-14480
 Project: Kafka
  Issue Type: Sub-task
Reporter: Ismael Juma






--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-14478) Move LogConfig to storage module

2022-12-14 Thread Ismael Juma (Jira)
Ismael Juma created KAFKA-14478:
---

 Summary: Move LogConfig to storage module
 Key: KAFKA-14478
 URL: https://issues.apache.org/jira/browse/KAFKA-14478
 Project: Kafka
  Issue Type: Sub-task
Reporter: Ismael Juma
Assignee: Ismael Juma






--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-14479) Move CleanerConfig to storage

2022-12-14 Thread Ismael Juma (Jira)
Ismael Juma created KAFKA-14479:
---

 Summary: Move CleanerConfig to storage
 Key: KAFKA-14479
 URL: https://issues.apache.org/jira/browse/KAFKA-14479
 Project: Kafka
  Issue Type: Sub-task
Reporter: Ismael Juma
Assignee: Ismael Juma






--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-14477) Move LogValidator to storage module

2022-12-14 Thread Ismael Juma (Jira)
Ismael Juma created KAFKA-14477:
---

 Summary: Move LogValidator to storage module
 Key: KAFKA-14477
 URL: https://issues.apache.org/jira/browse/KAFKA-14477
 Project: Kafka
  Issue Type: Sub-task
Reporter: Ismael Juma






--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-14476) Move OffsetMap to storage module

2022-12-14 Thread Ismael Juma (Jira)
Ismael Juma created KAFKA-14476:
---

 Summary: Move OffsetMap to storage module
 Key: KAFKA-14476
 URL: https://issues.apache.org/jira/browse/KAFKA-14476
 Project: Kafka
  Issue Type: Sub-task
Reporter: Ismael Juma
Assignee: Ismael Juma






--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-14475) Move TimestampIndex/LazyIndex to storage module

2022-12-14 Thread Ismael Juma (Jira)
Ismael Juma created KAFKA-14475:
---

 Summary: Move TimestampIndex/LazyIndex to storage module
 Key: KAFKA-14475
 URL: https://issues.apache.org/jira/browse/KAFKA-14475
 Project: Kafka
  Issue Type: Sub-task
Reporter: Ismael Juma






--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-14474) Move OffsetIndex to storage module

2022-12-14 Thread Ismael Juma (Jira)
Ismael Juma created KAFKA-14474:
---

 Summary: Move OffsetIndex to storage module
 Key: KAFKA-14474
 URL: https://issues.apache.org/jira/browse/KAFKA-14474
 Project: Kafka
  Issue Type: Sub-task
Reporter: Ismael Juma
Assignee: Ismael Juma






--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-14473) Move AbstractIndex to storage module

2022-12-14 Thread Ismael Juma (Jira)
Ismael Juma created KAFKA-14473:
---

 Summary: Move AbstractIndex to storage module
 Key: KAFKA-14473
 URL: https://issues.apache.org/jira/browse/KAFKA-14473
 Project: Kafka
  Issue Type: Sub-task
Reporter: Ismael Juma
Assignee: Ismael Juma






--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-14471) Move IndexEntry and related to storage module

2022-12-14 Thread Ismael Juma (Jira)
Ismael Juma created KAFKA-14471:
---

 Summary: Move IndexEntry and related to storage module
 Key: KAFKA-14471
 URL: https://issues.apache.org/jira/browse/KAFKA-14471
 Project: Kafka
  Issue Type: Sub-task
Reporter: Ismael Juma
Assignee: Ismael Juma






--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-14472) Move TransactionIndex and related to storage module

2022-12-14 Thread Ismael Juma (Jira)
Ismael Juma created KAFKA-14472:
---

 Summary: Move TransactionIndex and related to storage module
 Key: KAFKA-14472
 URL: https://issues.apache.org/jira/browse/KAFKA-14472
 Project: Kafka
  Issue Type: Sub-task
Reporter: Ismael Juma






--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-14470) Move log layer to storage module

2022-12-14 Thread Ismael Juma (Jira)
Ismael Juma created KAFKA-14470:
---

 Summary: Move log layer to storage module
 Key: KAFKA-14470
 URL: https://issues.apache.org/jira/browse/KAFKA-14470
 Project: Kafka
  Issue Type: Improvement
Reporter: Ismael Juma
Assignee: Ismael Juma


We introduced the `storage` module as part of KIP-405, but the existing log 
layer remains in the `core` module. Moving the log layer to the `storage` 
module would be another step towards improved modularity and build times 
(similar to `metadata`, `raft` and `group-coordinator`).

We should do this in an incremental manner to make the code review process 
easier. I will create separate tasks, each one mapping to one pull request.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Resolved] (KAFKA-9397) Deprecate Direct Zookeeper access in Kafka Administrative Tools

2022-09-05 Thread Ismael Juma (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-9397?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ismael Juma resolved KAFKA-9397.

Resolution: Fixed

> Deprecate Direct Zookeeper access in Kafka Administrative Tools
> ---
>
> Key: KAFKA-9397
> URL: https://issues.apache.org/jira/browse/KAFKA-9397
> Project: Kafka
>  Issue Type: Improvement
>  Components: tools
>Affects Versions: 2.5.0
>Reporter: Colin McCabe
>Assignee: Colin McCabe
>Priority: Major
> Fix For: 2.5.0
>
>
> KIP-555: Deprecate Direct Zookeeper access in Kafka Administrative Tools



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Reopened] (KAFKA-12887) Do not trigger user-customized ExceptionalHandler for RTE

2022-08-10 Thread Ismael Juma (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-12887?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ismael Juma reopened KAFKA-12887:
-

> Do not trigger user-customized ExceptionalHandler for RTE
> -
>
> Key: KAFKA-12887
> URL: https://issues.apache.org/jira/browse/KAFKA-12887
> Project: Kafka
>  Issue Type: Improvement
>  Components: streams
>Reporter: Guozhang Wang
>Assignee: Josep Prat
>Priority: Major
> Fix For: 3.1.0
>
>
> Today in StreamThread we have a try-catch block that captures all {{Throwable 
> e}} and then triggers {{this.streamsUncaughtExceptionHandler.accept(e)}}. 
> However, there are possible RTEs such as IllegalState/IllegalArgument 
> exceptions which are usually caused by bugs, etc. In such cases we should not 
> let users to decide what to do with these exceptions, but should let Streams 
> itself to enforce the decision, e.g. in the IllegalState/IllegalArgument we 
> should fail fast to notify the potential error.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-13779) System test verifying key updates with TLS 1.3

2022-03-29 Thread Ismael Juma (Jira)
Ismael Juma created KAFKA-13779:
---

 Summary: System test verifying key updates with TLS 1.3
 Key: KAFKA-13779
 URL: https://issues.apache.org/jira/browse/KAFKA-13779
 Project: Kafka
  Issue Type: Test
Reporter: Ismael Juma


KAFKA-13418 identified a gap in our TLS 1.3 implementation. The fix for that 
included a unit test that forced a key update, which is a start. It would be 
good to verify that the system behaves correctly when a key update is triggered 
automatically and it's more practical to achieve that via a system test [as 
explained in the PR|https://github.com/apache/kafka/pull/11966].

The system test can be a simple produce/consume workload with TLS 1.3 and Java 
11 (or newer) and it must set the security property `jdk.tls.keyLimits` to a 
number smaller than the bytes encrypted by each client in the test.



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[jira] [Resolved] (KAFKA-6706) NETWORK_EXCEPTION and REQUEST_TIMED_OUT in mirrormaker producer after 1.0 broker upgrade

2022-02-13 Thread Ismael Juma (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-6706?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ismael Juma resolved KAFKA-6706.

Fix Version/s: 1.1.1
   Resolution: Fixed

> NETWORK_EXCEPTION and REQUEST_TIMED_OUT in mirrormaker producer after 1.0 
> broker upgrade
> 
>
> Key: KAFKA-6706
> URL: https://issues.apache.org/jira/browse/KAFKA-6706
> Project: Kafka
>  Issue Type: Bug
>  Components: core, network
>Affects Versions: 1.0.0
>Reporter: Di Shang
>Priority: Blocker
>  Labels: mirror-maker
> Fix For: 1.1.1
>
>
> We have 2 clusters A and B with 4 brokers each, we use mirrormaker to 
> replicate topics from A to B. 
>  We recently upgraded our brokers from 0.10.2.0 to 1.0.0, after the upgrade 
> we started seeing the mirrormaker task showing producer errors and 
> intermittently dying. 
>  We tried using 1.0.0 and 0.10.2.0 mirrormaker, both have the same problem. 
> Downgrading cluster B brokers back to 0.10.2.0 and the problem went away, so 
> we think it's a server side problem.
> There are 2 types of errors: REQUEST_TIMED_OUT and NETWORK_EXCEPTION. For 
> testing, I used a topic *logging* with 20 partitions and 3 replicas (same on 
> cluster A and B), the source topic has 50+ million msg.
> (this is from mirrormaker 1.0 at info level, the 0.10.2.0 log is very similar)
> {noformat}
> 22 Mar 2018 02:16:07.407 [kafka-producer-network-thread | producer-1] WARN 
> org.apache.kafka.clients.producer.internals.Sender warn(line:251) [Producer 
> clientId=producer-1] Got error produce response with correlation id 35122 on 
> topic-partition logging-7, retrying (2147483646 attempts left). Error: 
> REQUEST_TIMED_OUT
>  22 Mar 2018 02:17:49.731 [kafka-producer-network-thread | producer-1] WARN 
> org.apache.kafka.clients.producer.internals.Sender warn(line:251) [Producer 
> clientId=producer-1] Got error produce response with correlation id 51572 on 
> topic-partition logging-7, retrying (2147483646 attempts left). Error: 
> REQUEST_TIMED_OUT
>  22 Mar 2018 02:18:33.903 [kafka-producer-network-thread | producer-1] WARN 
> org.apache.kafka.clients.producer.internals.Sender warn(line:251) [Producer 
> clientId=producer-1] Got error produce response with correlation id 57785 on 
> topic-partition logging-5, retrying (2147483646 attempts left). Error: 
> REQUEST_TIMED_OUT
>  22 Mar 2018 02:21:21.399 [kafka-producer-network-thread | producer-1] WARN 
> org.apache.kafka.clients.producer.internals.Sender warn(line:251) [Producer 
> clientId=producer-1] Got error produce response with correlation id 85406 on 
> topic-partition logging-18, retrying (2147483646 attempts left). Error: 
> REQUEST_TIMED_OUT
>  22 Mar 2018 02:25:22.278 [kafka-producer-network-thread | producer-1] WARN 
> org.apache.kafka.clients.producer.internals.Sender warn(line:251) [Producer 
> clientId=producer-1] Got error produce response with correlation id 128047 on 
> topic-partition logging-5, retrying (2147483646 attempts left). Error: 
> REQUEST_TIMED_OUT
>  22 Mar 2018 02:26:17.154 [kafka-producer-network-thread | producer-1] WARN 
> org.apache.kafka.clients.producer.internals.Sender warn(line:251) [Producer 
> clientId=producer-1] Got error produce response with correlation id 137049 on 
> topic-partition logging-18, retrying (2147483646 attempts left). Error: 
> REQUEST_TIMED_OUT
>  22 Mar 2018 02:27:57.358 [kafka-producer-network-thread | producer-1] WARN 
> org.apache.kafka.clients.producer.internals.Sender warn(line:251) [Producer 
> clientId=producer-1] Got error produce response with correlation id 153976 on 
> topic-partition logging-5, retrying (2147483646 attempts left). Error: 
> REQUEST_TIMED_OUT
>  22 Mar 2018 02:27:57.779 [kafka-producer-network-thread | producer-1] WARN 
> org.apache.kafka.clients.producer.internals.Sender warn(line:251) [Producer 
> clientId=producer-1] Got error produce response with correlation id 154077 on 
> topic-partition logging-2, retrying (2147483646 attempts left). Error: 
> NETWORK_EXCEPTION
>  22 Mar 2018 02:27:57.780 [kafka-producer-network-thread | producer-1] WARN 
> org.apache.kafka.clients.producer.internals.Sender warn(line:251) [Producer 
> clientId=producer-1] Got error produce response with correlation id 154077 on 
> topic-partition logging-10, retrying (2147483646 attempts left). Error: 
> NETWORK_EXCEPTION
>  22 Mar 2018 02:27:57.780 [kafka-producer-network-thread | producer-1] WARN 
> org.apache.kafka.clients.producer.internals.Sender warn(line:251) [Producer 
> clientId=producer-1] Got error produce response with correlation id 154077 on 
> topic-partition logging-18, retrying (2147483646 attempts left). Error: 
> NETWORK_EXCEPTION
>  22 Mar 2018 02:27:57.781 [kafka-producer-network-thread | producer-1] WARN 
> 

[jira] [Resolved] (KAFKA-4231) ZK metadata went inconsistent when migrating 0.10.0.0 -> 0.10.0.1

2022-02-13 Thread Ismael Juma (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-4231?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ismael Juma resolved KAFKA-4231.

Resolution: Cannot Reproduce

> ZK metadata went inconsistent when migrating 0.10.0.0 -> 0.10.0.1
> -
>
> Key: KAFKA-4231
> URL: https://issues.apache.org/jira/browse/KAFKA-4231
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 0.10.0.1
> Environment: 3-node cluster on AWS, 700+ topics, created and dropped 
> on regular basis, we use Java Kafka client 0.9.0.1 and ZKUtils 0.9.0.1 to 
> create/drop topics automatically
>Reporter: Sergey Alaev
>Priority: Major
>
> 1. we updated our Kafka cluster 0.10.0.0 -> 0.10.0.1 6 days ago
> 2. yesterday, we got huge amount of server logs looking like:
> [2016-09-29 12:33:27,055] ERROR [ReplicaFetcherThread-0-1], Error for 
> partition 
> [test-customer-recipient-4115239346516610_test-recipient-4115239346516610,0] 
> to broker 1:org.apache.kafka.common.errors
> .UnknownTopicOrPartitionException: This server does not host this 
> topic-partition. (kafka.server.ReplicaFetcherThread)
> 3. Today I've restarted all three nodes and they failed to start throwing 
> exception featured below.
> Investigation showed that ZK:
>  /config/topics contained 729 entries
> /admin/delete_topics contained 484 entries, all of them present in 
> /config/topics
> /brokers/topics were missing 6 entries present in /config/topics! This was 
> the cause of startup failure
> Removing those 6 entries from /config/topics fixed this issue.
> I'm sure that we didn't change ZK data manually, the only possible culpits 
> are kafka-server 0.10.0.1 and ZKUtils 0.9.0.1 (used only to delete topics)
> [2016-09-29 12:52:11,082] ERROR Error while electing or becoming leader on 
> broker 2 (kafka.server.ZookeeperLeaderElector)
> kafka.common.KafkaException: Can't parse json string: null
> at kafka.utils.Json$.liftedTree1$1(Json.scala:40)
> at kafka.utils.Json$.parseFull(Json.scala:36)
> at 
> kafka.utils.ZkUtils$$anonfun$getReplicaAssignmentForTopics$1.apply(ZkUtils.scala:610)
> at 
> kafka.utils.ZkUtils$$anonfun$getReplicaAssignmentForTopics$1.apply(ZkUtils.scala:606)
> at 
> scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
> at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
> at 
> kafka.utils.ZkUtils.getReplicaAssignmentForTopics(ZkUtils.scala:606)
> at 
> kafka.controller.KafkaController.initializeControllerContext(KafkaController.scala:744)
> at 
> kafka.controller.KafkaController.onControllerFailover(KafkaController.scala:333)
> at 
> kafka.controller.KafkaController$$anonfun$1.apply$mcV$sp(KafkaController.scala:166)
> at 
> kafka.server.ZookeeperLeaderElector.elect(ZookeeperLeaderElector.scala:84)
> at 
> kafka.server.ZookeeperLeaderElector$LeaderChangeListener$$anonfun$handleDataDeleted$1.apply$mcZ$sp(ZookeeperLeaderElector.scala:146)
> at 
> kafka.server.ZookeeperLeaderElector$LeaderChangeListener$$anonfun$handleDataDeleted$1.apply(ZookeeperLeaderElector.scala:141)
> at 
> kafka.server.ZookeeperLeaderElector$LeaderChangeListener$$anonfun$handleDataDeleted$1.apply(ZookeeperLeaderElector.scala:141)
> at kafka.utils.CoreUtils$.inLock(CoreUtils.scala:231)
> at 
> kafka.server.ZookeeperLeaderElector$LeaderChangeListener.handleDataDeleted(ZookeeperLeaderElector.scala:141)
> at org.I0Itec.zkclient.ZkClient$9.run(ZkClient.java:824)
> at org.I0Itec.zkclient.ZkEventThread.run(ZkEventThread.java:71)
> Caused by: java.lang.NullPointerException
> Server configuration:
> [2016-09-29 12:40:58,190] INFO KafkaConfig values:
> advertised.host.name = null
> metric.reporters = []
> quota.producer.default = 9223372036854775807
> offsets.topic.num.partitions = 50
> log.flush.interval.messages = 9223372036854775807
> auto.create.topics.enable = true
> controller.socket.timeout.ms = 3
> log.flush.interval.ms = null
> principal.builder.class = class 
> org.apache.kafka.common.security.auth.DefaultPrincipalBuilder
> replica.socket.receive.buffer.bytes = 65536
> min.insync.replicas = 2
> replica.fetch.wait.max.ms = 500
> num.recovery.threads.per.data.dir = 1
> ssl.keystore.type = JKS
> sasl.mechanism.inter.broker.protocol = GSSAPI
> default.replication.factor = 3
> ssl.truststore.password = [hidden]
> log.preallocate = false
> sasl.kerberos.principal.to.local.rules = [DEFAULT]
> fetch.purgatory.purge.interval.requests = 1000
> ssl.endpoint.identification.algorithm = null
> replica.socket.timeout.ms = 3
>  

[jira] [Resolved] (KAFKA-7304) memory leakage in org.apache.kafka.common.network.Selector

2022-02-13 Thread Ismael Juma (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-7304?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ismael Juma resolved KAFKA-7304.

Fix Version/s: 2.0.1
 Assignee: Rajini Sivaram
   Resolution: Fixed

Marking as fixed since the leaks were fixed.

> memory leakage in org.apache.kafka.common.network.Selector
> --
>
> Key: KAFKA-7304
> URL: https://issues.apache.org/jira/browse/KAFKA-7304
> Project: Kafka
>  Issue Type: Bug
>  Components: core
>Affects Versions: 1.1.0, 1.1.1
>Reporter: Yu Yang
>Assignee: Rajini Sivaram
>Priority: Major
> Fix For: 2.0.1
>
> Attachments: 7304.v4.txt, 7304.v7.txt, Screen Shot 2018-08-16 at 
> 11.04.16 PM.png, Screen Shot 2018-08-16 at 11.06.38 PM.png, Screen Shot 
> 2018-08-16 at 12.41.26 PM.png, Screen Shot 2018-08-16 at 4.26.19 PM.png, 
> Screen Shot 2018-08-17 at 1.03.35 AM.png, Screen Shot 2018-08-17 at 1.04.32 
> AM.png, Screen Shot 2018-08-17 at 1.05.30 AM.png, Screen Shot 2018-08-28 at 
> 11.09.45 AM.png, Screen Shot 2018-08-29 at 10.49.03 AM.png, Screen Shot 
> 2018-08-29 at 10.50.47 AM.png, Screen Shot 2018-09-29 at 10.38.12 PM.png, 
> Screen Shot 2018-09-29 at 10.38.38 PM.png, Screen Shot 2018-09-29 at 8.34.50 
> PM.png
>
>
> We are testing secured writing to kafka through ssl. Testing at small scale, 
> ssl writing to kafka was fine. However, when we enabled ssl writing at a 
> larger scale (>40k clients write concurrently), the kafka brokers soon hit 
> OutOfMemory issue with 4G memory setting. We have tried with increasing the 
> heap size to 10Gb, but encountered the same issue. 
> We took a few heap dumps , and found that most of the heap memory is 
> referenced through org.apache.kafka.common.network.Selector objects.  There 
> are two Channel maps field in Selector. It seems that somehow the objects is 
> not deleted from the map in a timely manner. 
> One observation is that the memory leak seems relate to kafka partition 
> leader changes. If there is broker restart etc. in the cluster that caused 
> partition leadership change, the brokers may hit the OOM issue faster. 
> {code}
> private final Map channels;
> private final Map closingChannels;
> {code}
> Please see the  attached images and the following link for sample gc 
> analysis. 
> http://gceasy.io/my-gc-report.jsp?p=c2hhcmVkLzIwMTgvMDgvMTcvLS1nYy5sb2cuMC5jdXJyZW50Lmd6LS0yLTM5LTM0
> the command line for running kafka: 
> {code}
> java -Xms10g -Xmx10g -XX:NewSize=512m -XX:MaxNewSize=512m 
> -Xbootclasspath/p:/usr/local/libs/bcp -XX:MetaspaceSize=128m -XX:+UseG1GC 
> -XX:MaxGCPauseMillis=25 -XX:InitiatingHeapOccupancyPercent=35 
> -XX:G1HeapRegionSize=16M -XX:MinMetaspaceFreeRatio=25 
> -XX:MaxMetaspaceFreeRatio=75 -XX:+PrintGCDetails -XX:+PrintGCDateStamps 
> -XX:+PrintTenuringDistribution -Xloggc:/var/log/kafka/gc.log 
> -XX:+UseGCLogFileRotation -XX:NumberOfGCLogFiles=40 -XX:GCLogFileSize=50M 
> -Djava.awt.headless=true 
> -Dlog4j.configuration=file:/etc/kafka/log4j.properties 
> -Dcom.sun.management.jmxremote 
> -Dcom.sun.management.jmxremote.authenticate=false 
> -Dcom.sun.management.jmxremote.ssl=false 
> -Dcom.sun.management.jmxremote.port= 
> -Dcom.sun.management.jmxremote.rmi.port= -cp /usr/local/libs/*  
> kafka.Kafka /etc/kafka/server.properties
> {code}
> We use java 1.8.0_102, and has applied a TLS patch on reducing 
> X509Factory.certCache map size from 750 to 20. 
> {code}
> java -version
> java version "1.8.0_102"
> Java(TM) SE Runtime Environment (build 1.8.0_102-b14)
> Java HotSpot(TM) 64-Bit Server VM (build 25.102-b14, mixed mode)
> {code}



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[jira] [Resolved] (KAFKA-2561) Optionally support OpenSSL for SSL/TLS

2022-02-13 Thread Ismael Juma (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-2561?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ismael Juma resolved KAFKA-2561.

Resolution: Won't Do

> Optionally support OpenSSL for SSL/TLS 
> ---
>
> Key: KAFKA-2561
> URL: https://issues.apache.org/jira/browse/KAFKA-2561
> Project: Kafka
>  Issue Type: New Feature
>  Components: security
>Affects Versions: 0.9.0.0
>Reporter: Ismael Juma
>Priority: Major
>
> JDK's `SSLEngine` is unfortunately a bit slow (KAFKA-2431 covers this in more 
> detail). We should consider supporting OpenSSL for SSL/TLS. Initial 
> experiments on my laptop show that it performs a lot better:
> {code}
> start.time, end.time, data.consumed.in.MB, MB.sec, data.consumed.in.nMsg, 
> nMsg.sec, config
> 2015-09-21 14:41:58:245, 2015-09-21 14:47:02:583, 28610.2295, 94.0081, 
> 3000, 98574.6111, Java 8u60/server auth JDK 
> SSLEngine/TLS_ECDHE_RSA_WITH_AES_128_CBC_SHA
> 2015-09-21 14:38:24:526, 2015-09-21 14:40:19:941, 28610.2295, 247.8900, 
> 3000, 259931.5514, Java 8u60/server auth 
> OpenSslEngine/TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256
> 2015-09-21 14:49:03:062, 2015-09-21 14:50:27:764, 28610.2295, 337.7751, 
> 3000, 354182.9000, Java 8u60/plaintext
> {code}
> Extracting the throughput figures:
> * JDK SSLEngine: 94 MB/s
> * OpenSSL SSLEngine: 247 MB/s
> * Plaintext: 337 MB/s (code from trunk, so no zero-copy due to KAFKA-2517)
> In order to get these figures, I used Netty's `OpenSslEngine` by hacking 
> `SSLFactory` to use Netty's `SslContextBuilder` and made a few changes to 
> `SSLTransportLayer` in order to workaround differences in behaviour between 
> `OpenSslEngine` and JDK's SSLEngine (filed 
> https://github.com/netty/netty/issues/4235 and 
> https://github.com/netty/netty/issues/4238 upstream).



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[jira] [Resolved] (KAFKA-13270) Kafka may fail to connect to ZooKeeper, retry forever, and never start

2021-09-06 Thread Ismael Juma (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-13270?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ismael Juma resolved KAFKA-13270.
-
  Assignee: Ismael Juma  (was: Ron Dagostino)
Resolution: Fixed

> Kafka may fail to connect to ZooKeeper, retry forever, and never start
> --
>
> Key: KAFKA-13270
> URL: https://issues.apache.org/jira/browse/KAFKA-13270
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 3.0.0
>Reporter: Ron Dagostino
>Assignee: Ismael Juma
>Priority: Blocker
> Fix For: 3.0.0
>
>
> The implementation of https://issues.apache.org/jira/browse/ZOOKEEPER-3593 in 
> ZooKeeper version 3.6.0 decreased the default value for the ZooKeeper 
> client's `jute.maxbuffer` configuration from 4MB to 1MB.  This can cause a 
> problem if Kafka tries to retrieve a large amount of data across many znodes 
> -- in such a case the ZooKeeper client will repeatedly emit a message of the 
> form "java.io.IOException: Packet len <> is out of range" and the Kafka 
> broker will never connect to ZooKeeper and fail to make progress on the 
> startup sequence.  We can avoid the potential for this issue to occur by 
> explicitly setting the value to 4MB whenever we create a new ZooKeeper client 
> as long as no explicit value has been set via the `jute.maxbuffer` system 
> property.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Created] (KAFKA-13275) Remove --add-opens from build.gradle (where possible)

2021-09-05 Thread Ismael Juma (Jira)
Ismael Juma created KAFKA-13275:
---

 Summary: Remove --add-opens from build.gradle (where possible)
 Key: KAFKA-13275
 URL: https://issues.apache.org/jira/browse/KAFKA-13275
 Project: Kafka
  Issue Type: Improvement
Reporter: Ismael Juma


We added several `--add-opens` to make a number of tests pass with Java 17 (via 
KAFKA-13273).

It should be possible to remove many (most?) of them by adjusting the tests.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Created] (KAFKA-13274) Ensure system tests run successfully with Java 17

2021-09-05 Thread Ismael Juma (Jira)
Ismael Juma created KAFKA-13274:
---

 Summary: Ensure system tests run successfully with Java 17
 Key: KAFKA-13274
 URL: https://issues.apache.org/jira/browse/KAFKA-13274
 Project: Kafka
  Issue Type: Improvement
Reporter: Ismael Juma






--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Created] (KAFKA-13273) Add support for Java 17

2021-09-05 Thread Ismael Juma (Jira)
Ismael Juma created KAFKA-13273:
---

 Summary: Add support for Java 17
 Key: KAFKA-13273
 URL: https://issues.apache.org/jira/browse/KAFKA-13273
 Project: Kafka
  Issue Type: Improvement
Reporter: Ismael Juma
Assignee: Ismael Juma
 Fix For: 3.1.0






--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Resolved] (KAFKA-13116) KIP-724: Adjust system tests due to KAFKA-12944

2021-07-23 Thread Ismael Juma (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-13116?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ismael Juma resolved KAFKA-13116.
-
Resolution: Fixed

> KIP-724: Adjust system tests due to KAFKA-12944
> ---
>
> Key: KAFKA-13116
> URL: https://issues.apache.org/jira/browse/KAFKA-13116
> Project: Kafka
>  Issue Type: Sub-task
>Reporter: Ismael Juma
>Assignee: Ismael Juma
>Priority: Blocker
> Fix For: 3.0.0
>
>
> Several system tests involving legacy message formats are failing due to 
> KAFKA-12944:
> http://confluent-kafka-system-test-results.s3-us-west-2.amazonaws.com/2021-07-21--001.system-test-kafka-trunk--1626872410--confluentinc--master–038bdaa4df/report.html
> All system tests that write data with legacy message formats need to use IBP 
> 2.8 or lower.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Created] (KAFKA-13116) Adjust system tests due to KAFKA-12944

2021-07-21 Thread Ismael Juma (Jira)
Ismael Juma created KAFKA-13116:
---

 Summary: Adjust system tests due to KAFKA-12944
 Key: KAFKA-13116
 URL: https://issues.apache.org/jira/browse/KAFKA-13116
 Project: Kafka
  Issue Type: Sub-task
Reporter: Ismael Juma
Assignee: Ismael Juma
 Fix For: 3.0.0


Several system tests involving legacy message formats are failing due to 
KAFKA-12944:

http://confluent-kafka-system-test-results.s3-us-west-2.amazonaws.com/2021-07-21--001.system-test-kafka-trunk--1626872410--confluentinc--master–038bdaa4df/report.html

All system tests that write data with legacy message formats need to use IBP 
2.8 or lower.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Resolved] (KAFKA-12418) Make sure it's ok not to include test jars in the release tarball

2021-07-21 Thread Ismael Juma (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-12418?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ismael Juma resolved KAFKA-12418.
-
Resolution: Done

> Make sure it's ok not to include test jars in the release tarball
> -
>
> Key: KAFKA-12418
> URL: https://issues.apache.org/jira/browse/KAFKA-12418
> Project: Kafka
>  Issue Type: Improvement
>Reporter: Ismael Juma
>Assignee: Ismael Juma
>Priority: Blocker
>
> As of [https://github.com/apache/kafka/pull/10203,] the release tarball no 
> longer includes includes test, sources, javadoc and test sources jars. These 
> are still published to the Maven Central repository.
> This seems like a good change and 3.0.0 would be a good time to do it, but 
> filing this JIRA to follow up and make sure before said release.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Resolved] (KAFKA-12588) Remove deprecated --zookeeper in shell commands

2021-07-19 Thread Ismael Juma (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-12588?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ismael Juma resolved KAFKA-12588.
-
Resolution: Fixed

> Remove deprecated --zookeeper in shell commands
> ---
>
> Key: KAFKA-12588
> URL: https://issues.apache.org/jira/browse/KAFKA-12588
> Project: Kafka
>  Issue Type: Task
>Reporter: Luke Chen
>Assignee: Luke Chen
>Priority: Blocker
> Fix For: 3.0.0
>
>
> At first check, there are still 4 commands existing *--zookeeper* option. 
> Should be removed in V3.0.0
>  
> _preferredReplicaLeaderElectionCommand_
> _ConfigCommand_
> _ReassignPartitionsCommand_
> _TopicCommand_



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Created] (KAFKA-13093) KIP-724: Log compaction should write new segments with record version v2

2021-07-15 Thread Ismael Juma (Jira)
Ismael Juma created KAFKA-13093:
---

 Summary: KIP-724: Log compaction should write new segments with 
record version v2
 Key: KAFKA-13093
 URL: https://issues.apache.org/jira/browse/KAFKA-13093
 Project: Kafka
  Issue Type: Sub-task
Reporter: Ismael Juma
Assignee: Ismael Juma
 Fix For: 3.1.0


If IBP is 3.0 or higher. Currently, log compaction retains the record format of 
the record batch that was retained.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Resolved] (KAFKA-12921) Upgrade ZSTD JNI from 1.4.9-1 to 1.5.0-1

2021-06-30 Thread Ismael Juma (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-12921?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ismael Juma resolved KAFKA-12921.
-
Resolution: Fixed

> Upgrade ZSTD JNI from 1.4.9-1 to 1.5.0-1
> 
>
> Key: KAFKA-12921
> URL: https://issues.apache.org/jira/browse/KAFKA-12921
> Project: Kafka
>  Issue Type: Improvement
>  Components: build
>Affects Versions: 3.0.0
>Reporter: David Christle
>Priority: Major
> Fix For: 3.0.0
>
>




--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Resolved] (KAFKA-12786) Getting SslTransportLayerTest error

2021-06-23 Thread Ismael Juma (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-12786?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ismael Juma resolved KAFKA-12786.
-
Resolution: Duplicate

Since it's the same root cause as KAFKA-12790, marking as a duplicate of that.

> Getting SslTransportLayerTest error 
> 
>
> Key: KAFKA-12786
> URL: https://issues.apache.org/jira/browse/KAFKA-12786
> Project: Kafka
>  Issue Type: Bug
>  Components: unit tests
> Environment: Ububtu 20.04
>Reporter: Sibelle
>Assignee: Ismael Juma
>Priority: Major
>  Labels: beginner
> Attachments: Error.png
>
>
> SaslAuthenticatorTest > testRepeatedValidSaslPlainOverSsl() PASSED
> org.apache.kafka.common.network.SslTransportLayerTest.testUnsupportedTLSVersion(Args)[1]
>  failed, log available in 
> /kafka/clients/build/reports/testOutput/org.apache.kafka.common.network.SslTransportLayerTest.testUnsupportedTLSVersion(Args)[1].test.stdout
> SslTransportLayerTest > [1] tlsProtocol=TLSv1.2, useInlinePem=false FAILED
> org.opentest4j.AssertionFailedError: Condition not met within timeout 
> 15000. Metric not updated failed-authentication-total expected:<1.0> but 
> was:<0.0> ==> expected:  but was: 
> at org.junit.jupiter.api.AssertionUtils.fail(AssertionUtils.java:55)
> at org.junit.jupiter.api.AssertTrue.assertTrue(AssertTrue.java:40)
> at org.junit.jupiter.api.Assertions.assertTrue(Assertions.java:193)
> at 
> org.apache.kafka.test.TestUtils.lambda$waitForCondition$3(TestUtils.java:320)
> at 
> org.apache.kafka.test.TestUtils.retryOnExceptionWithTimeout(TestUtils.java:368)
> at 
> org.apache.kafka.test.TestUtils.waitForCondition(TestUtils.java:317)
> at 
> org.apache.kafka.test.TestUtils.waitForCondition(TestUtils.java:301)
> at 
> org.apache.kafka.common.network.NioEchoServer.waitForMetrics(NioEchoServer.java:196)
> at 
> org.apache.kafka.common.network.NioEchoServer.verifyAuthenticationMetrics(NioEchoServer.java:155)
> at 
> org.apache.kafka.common.network.SslTransportLayerTest.testUnsupportedTLSVersion(SslTransportLayerTest.java:644)



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Created] (KAFKA-12945) Remove configs: port, host.name and advertized variants in 3.0

2021-06-12 Thread Ismael Juma (Jira)
Ismael Juma created KAFKA-12945:
---

 Summary: Remove configs: port, host.name and advertized variants 
in 3.0
 Key: KAFKA-12945
 URL: https://issues.apache.org/jira/browse/KAFKA-12945
 Project: Kafka
  Issue Type: Bug
Reporter: Ismael Juma
Assignee: Ismael Juma
 Fix For: 3.0.0


[0.10.0.0-rc2|https://github.com/apache/kafka/releases/tag/0.10.0.0-rc2]



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Created] (KAFKA-12944) KIP-724: Always write record batches with message format v2

2021-06-11 Thread Ismael Juma (Jira)
Ismael Juma created KAFKA-12944:
---

 Summary: KIP-724: Always write record batches with message format 
v2
 Key: KAFKA-12944
 URL: https://issues.apache.org/jira/browse/KAFKA-12944
 Project: Kafka
  Issue Type: Sub-task
Reporter: Ismael Juma
Assignee: Ismael Juma
 Fix For: 3.0.0






--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Created] (KAFKA-12941) Remove excluded Streams tests when running with JDK 16

2021-06-11 Thread Ismael Juma (Jira)
Ismael Juma created KAFKA-12941:
---

 Summary: Remove excluded Streams tests when running with JDK 16
 Key: KAFKA-12941
 URL: https://issues.apache.org/jira/browse/KAFKA-12941
 Project: Kafka
  Issue Type: Sub-task
Reporter: Ismael Juma






--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Created] (KAFKA-12942) Remove Connect excluded tests when running with JDK 16

2021-06-11 Thread Ismael Juma (Jira)
Ismael Juma created KAFKA-12942:
---

 Summary: Remove Connect excluded tests when running with JDK 16
 Key: KAFKA-12942
 URL: https://issues.apache.org/jira/browse/KAFKA-12942
 Project: Kafka
  Issue Type: Sub-task
Reporter: Ismael Juma






--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Created] (KAFKA-12940) Enable JDK 16 builds in Jenkins

2021-06-11 Thread Ismael Juma (Jira)
Ismael Juma created KAFKA-12940:
---

 Summary: Enable JDK 16 builds in Jenkins
 Key: KAFKA-12940
 URL: https://issues.apache.org/jira/browse/KAFKA-12940
 Project: Kafka
  Issue Type: Sub-task
Reporter: Ismael Juma
Assignee: Ismael Juma






--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Resolved] (KAFKA-12905) Replace EasyMock and PowerMock with Mockito for NamedCacheMetricsTest

2021-06-11 Thread Ismael Juma (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-12905?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ismael Juma resolved KAFKA-12905.
-
Resolution: Fixed

> Replace EasyMock and PowerMock with Mockito for NamedCacheMetricsTest
> -
>
> Key: KAFKA-12905
> URL: https://issues.apache.org/jira/browse/KAFKA-12905
> Project: Kafka
>  Issue Type: Sub-task
>Reporter: YI-CHEN WANG
>Assignee: YI-CHEN WANG
>Priority: Major
> Fix For: 3.0.0
>
> Attachments: image-2021-06-08-21-42-15-727.png
>
>
> For 
> [Kafka-7438|https://issues.apache.org/jira/browse/KAFKA-7438?jql=project%20%3D%20KAFKA%20AND%20status%20in%20(Open%2C%20Reopened)%20AND%20assignee%20in%20(EMPTY)%20AND%20text%20~%20%22mockito%22]



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Resolved] (KAFKA-12924) Replace EasyMock and PowerMock with Mockito in streams module(metrics)

2021-06-11 Thread Ismael Juma (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-12924?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ismael Juma resolved KAFKA-12924.
-
Fix Version/s: 3.0.0
   Resolution: Fixed

> Replace EasyMock and PowerMock with Mockito in streams module(metrics)
> --
>
> Key: KAFKA-12924
> URL: https://issues.apache.org/jira/browse/KAFKA-12924
> Project: Kafka
>  Issue Type: Sub-task
>  Components: streams, unit tests
>Reporter: YI-CHEN WANG
>Assignee: YI-CHEN WANG
>Priority: Major
> Fix For: 3.0.0
>
>




--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Created] (KAFKA-12930) Deprecate support for Scala 2.12 in Kafka 3.0

2021-06-10 Thread Ismael Juma (Jira)
Ismael Juma created KAFKA-12930:
---

 Summary: Deprecate support for Scala 2.12 in Kafka 3.0
 Key: KAFKA-12930
 URL: https://issues.apache.org/jira/browse/KAFKA-12930
 Project: Kafka
  Issue Type: Sub-task
Reporter: Ismael Juma
Assignee: Ismael Juma
 Fix For: 3.0.0






--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Created] (KAFKA-12929) KIP-750: Deprecate Java 8 in Kafka 3.0

2021-06-10 Thread Ismael Juma (Jira)
Ismael Juma created KAFKA-12929:
---

 Summary: KIP-750: Deprecate Java 8 in Kafka 3.0
 Key: KAFKA-12929
 URL: https://issues.apache.org/jira/browse/KAFKA-12929
 Project: Kafka
  Issue Type: Sub-task
Reporter: Ismael Juma
Assignee: Ismael Juma
 Fix For: 3.0.0






--
This message was sent by Atlassian Jira
(v8.3.4#803005)


  1   2   3   4   5   6   7   8   9   10   >