[jira] [Commented] (KAFKA-14184) Kafka streams application crashes due to "UnsupportedOperationException: this should not happen: timestamp() is not supported in standby tasks."

2022-11-17 Thread Suresh Rukmangathan (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14184?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17635723#comment-17635723 ] Suresh Rukmangathan commented on KAFKA-14184: - Few updates:- # We are using

[GitHub] [kafka] dajac commented on pull request #12845: KAFKA-14367; Add `JoinGroup` to the new `GroupCoordinator` interface

2022-11-17 Thread GitBox
dajac commented on PR #12845: URL: https://github.com/apache/kafka/pull/12845#issuecomment-1319647546 @jeffkbkim Thanks for the review. I have addressed your comments. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use

[GitHub] [kafka] dajac commented on a diff in pull request #12845: KAFKA-14367; Add `JoinGroup` to the new `GroupCoordinator` interface

2022-11-17 Thread GitBox
dajac commented on code in PR #12845: URL: https://github.com/apache/kafka/pull/12845#discussion_r1026102820 ## core/src/test/scala/unit/kafka/server/KafkaApisTest.scala: ## @@ -2524,196 +2530,208 @@ class KafkaApisTest { assertEquals(MemoryRecords.EMPTY,

[GitHub] [kafka] dajac commented on a diff in pull request #12845: KAFKA-14367; Add `JoinGroup` to the new `GroupCoordinator` interface

2022-11-17 Thread GitBox
dajac commented on code in PR #12845: URL: https://github.com/apache/kafka/pull/12845#discussion_r1026101492 ## core/src/test/scala/unit/kafka/server/KafkaApisTest.scala: ## @@ -2524,196 +2530,208 @@ class KafkaApisTest { assertEquals(MemoryRecords.EMPTY,

[GitHub] [kafka] dajac commented on a diff in pull request #12845: KAFKA-14367; Add `JoinGroup` to the new `GroupCoordinator` interface

2022-11-17 Thread GitBox
dajac commented on code in PR #12845: URL: https://github.com/apache/kafka/pull/12845#discussion_r1026099763 ## core/src/main/scala/kafka/server/KafkaApis.scala: ## @@ -1647,69 +1656,51 @@ class KafkaApis(val requestChannel: RequestChannel, } } - def

[GitHub] [kafka] dajac commented on a diff in pull request #12845: KAFKA-14367; Add `JoinGroup` to the new `GroupCoordinator` interface

2022-11-17 Thread GitBox
dajac commented on code in PR #12845: URL: https://github.com/apache/kafka/pull/12845#discussion_r1026098011 ## core/src/main/scala/kafka/server/KafkaApis.scala: ## @@ -161,6 +166,12 @@ class KafkaApis(val requestChannel: RequestChannel, * Top-level method that handles all

[jira] [Commented] (KAFKA-14184) Kafka streams application crashes due to "UnsupportedOperationException: this should not happen: timestamp() is not supported in standby tasks."

2022-11-17 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14184?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17635680#comment-17635680 ] Matthias J. Sax commented on KAFKA-14184: - The stack trace is highly suspicious. Based on the

[jira] [Commented] (KAFKA-14374) Kafka streams losing messages in State Store during first launch of app

2022-11-17 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14374?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17635667#comment-17635667 ] Matthias J. Sax commented on KAFKA-14374: - Thanks for creating a ticket. I am not sure right now

[GitHub] [kafka] jeffkbkim commented on a diff in pull request #12845: KAFKA-14367; Add `JoinGroup` to the new `GroupCoordinator` interface

2022-11-17 Thread GitBox
jeffkbkim commented on code in PR #12845: URL: https://github.com/apache/kafka/pull/12845#discussion_r1025980136 ## core/src/main/scala/kafka/server/KafkaApis.scala: ## @@ -161,6 +166,12 @@ class KafkaApis(val requestChannel: RequestChannel, * Top-level method that handles

[GitHub] [kafka] showuon merged pull request #12868: MINOR: fix syntax typo

2022-11-17 Thread GitBox
showuon merged PR #12868: URL: https://github.com/apache/kafka/pull/12868 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail:

[GitHub] [kafka] hachikuji opened a new pull request, #12873: WIP: Idempotent producer simulation

2022-11-17 Thread GitBox
hachikuji opened a new pull request, #12873: URL: https://github.com/apache/kafka/pull/12873 As many have noticed, the idempotent producer implementation is complex. This patch attempts to introduce a new simulation-based testing methodology which which enables testing of more complex

[GitHub] [kafka] ableegoldman commented on a diff in pull request #12869: KAFKA-14382: wait for current rebalance to complete before triggering followup

2022-11-17 Thread GitBox
ableegoldman commented on code in PR #12869: URL: https://github.com/apache/kafka/pull/12869#discussion_r1025891177 ## streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java: ## @@ -611,9 +616,11 @@ boolean runLoop() {

[GitHub] [kafka] ableegoldman commented on a diff in pull request #12869: KAFKA-14382: wait for current rebalance to complete before triggering followup

2022-11-17 Thread GitBox
ableegoldman commented on code in PR #12869: URL: https://github.com/apache/kafka/pull/12869#discussion_r1025887660 ## streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java: ## @@ -298,6 +298,7 @@ public boolean isRunning() { private volatile

[GitHub] [kafka] C0urante commented on pull request #12295: KAFKA-13586: Prevent exception thrown during connector update from crashing distributed herder

2022-11-17 Thread GitBox
C0urante commented on PR #12295: URL: https://github.com/apache/kafka/pull/12295#issuecomment-1319344550 @dstelljes Good news! Now that we've eliminated the need to mock the static APIs for the `RestClient` and `Plugins` classes, we don't have to worry about the issues posed by them in the

[GitHub] [kafka] soarez opened a new pull request, #12872: KAFKA-14303 Producer.send without record key and batch.size=0 goes into infinite loop

2022-11-17 Thread GitBox
soarez opened a new pull request, #12872: URL: https://github.com/apache/kafka/pull/12872 Cherry-picked and slightly modified commit 5bd556a49b. The change was made in line in `core/src/test/scala/integration/kafka/api/PlaintextProducerSendTest.scala: 61` to remove the `threadMode`

[GitHub] [kafka] C0urante merged pull request #12866: KAFKA-14346: Remove hard-to-mock javax.crypto calls

2022-11-17 Thread GitBox
C0urante merged PR #12866: URL: https://github.com/apache/kafka/pull/12866 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail:

[GitHub] [kafka] C0urante commented on pull request #12866: KAFKA-14346: Remove hard-to-mock javax.crypto calls

2022-11-17 Thread GitBox
C0urante commented on PR #12866: URL: https://github.com/apache/kafka/pull/12866#issuecomment-1319327985 Test failures are unrelated; merging... -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to

[GitHub] [kafka] C0urante merged pull request #12828: KAFKA-14346: Remove hard-to-mock RestClient calls

2022-11-17 Thread GitBox
C0urante merged PR #12828: URL: https://github.com/apache/kafka/pull/12828 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail:

[GitHub] [kafka] C0urante commented on pull request #12828: KAFKA-14346: Remove hard-to-mock RestClient calls

2022-11-17 Thread GitBox
C0urante commented on PR #12828: URL: https://github.com/apache/kafka/pull/12828#issuecomment-1319311409 Test failures are unrelated; merging... -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to

[GitHub] [kafka] junrao commented on pull request #12752: KAFKA-14303 Fix batch.size=0 in BuiltInPartitioner

2022-11-17 Thread GitBox
junrao commented on PR #12752: URL: https://github.com/apache/kafka/pull/12752#issuecomment-1319300532 @soarez : I tried to cherry-pick the PR to the 3.3. branch, but got a compilation error. Do you think you could submit a separate PR for 3.3? Thanks. ``` [Error]

[GitHub] [kafka] junrao merged pull request #12752: KAFKA-14303 Fix batch.size=0 in BuiltInPartitioner

2022-11-17 Thread GitBox
junrao merged PR #12752: URL: https://github.com/apache/kafka/pull/12752 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail:

[jira] [Updated] (KAFKA-14372) RackAwareReplicaSelector should choose a replica from the isr

2022-11-17 Thread Jeff Kim (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14372?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jeff Kim updated KAFKA-14372: - Description: The default replica selector chooses a replica solely on whether the broker.rack matches

[jira] [Updated] (KAFKA-14354) Add 'isDeleted' parameter when stopping a Connector

2022-11-17 Thread Hector Geraldino (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14354?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Hector Geraldino updated KAFKA-14354: - Description: It would be useful for Connectors to know when its instance is being

[jira] [Updated] (KAFKA-14354) Add 'isDeleted' parameter when stopping a Connector

2022-11-17 Thread Hector Geraldino (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14354?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Hector Geraldino updated KAFKA-14354: - Summary: Add 'isDeleted' parameter when stopping a Connector (was: Add 'destroyed()'

[jira] [Updated] (KAFKA-14372) RackAwareReplicaSelector should choose a replica from the isr

2022-11-17 Thread Jeff Kim (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14372?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jeff Kim updated KAFKA-14372: - Description: The default replica selector chooses a replica solely on whether the broker.rack matches

[GitHub] [kafka] C0urante commented on a diff in pull request #12800: KAFKA-14342: KafkaOffsetBackingStore should clear offsets for source partitions on tombstone messages

2022-11-17 Thread GitBox
C0urante commented on code in PR #12800: URL: https://github.com/apache/kafka/pull/12800#discussion_r1025529528 ## connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaOffsetBackingStore.java: ## @@ -325,11 +325,12 @@ public Future set(final Map values, final

[GitHub] [kafka] C0urante commented on a diff in pull request #12800: KAFKA-14342: KafkaOffsetBackingStore should clear offsets for source partitions on tombstone messages

2022-11-17 Thread GitBox
C0urante commented on code in PR #12800: URL: https://github.com/apache/kafka/pull/12800#discussion_r1025541314 ## connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaOffsetBackingStoreTest.java: ## @@ -329,6 +329,60 @@ public void testGetSetNull() throws

[jira] [Assigned] (KAFKA-14396) Flaky memory leak tests rely on System.gc for correctness

2022-11-17 Thread Greg Harris (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14396?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Greg Harris reassigned KAFKA-14396: --- Assignee: Greg Harris > Flaky memory leak tests rely on System.gc for correctness >

[GitHub] [kafka] C0urante commented on a diff in pull request #12866: KAFKA-14346: Remove hard-to-mock javax.crypto calls

2022-11-17 Thread GitBox
C0urante commented on code in PR #12866: URL: https://github.com/apache/kafka/pull/12866#discussion_r1025503951 ## connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/CryptoLibrary.java: ## @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software

[GitHub] [kafka] dajac commented on a diff in pull request #12847: KAFKA-14367; Add `SyncGroup` to the new `GroupCoordinator` interface

2022-11-17 Thread GitBox
dajac commented on code in PR #12847: URL: https://github.com/apache/kafka/pull/12847#discussion_r1025502783 ## core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorAdapterTest.scala: ## @@ -135,4 +133,71 @@ class GroupCoordinatorAdapterTest {

[GitHub] [kafka] C0urante commented on a diff in pull request #12828: KAFKA-14346: Remove hard-to-mock RestClient calls

2022-11-17 Thread GitBox
C0urante commented on code in PR #12828: URL: https://github.com/apache/kafka/pull/12828#discussion_r1025489778 ## connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestClient.java: ## @@ -43,10 +43,38 @@ import java.util.concurrent.ExecutionException;

[GitHub] [kafka] vamossagar12 commented on a diff in pull request #12803: KAFKA-13602: Adding ability to multicast records.

2022-11-17 Thread GitBox
vamossagar12 commented on code in PR #12803: URL: https://github.com/apache/kafka/pull/12803#discussion_r1025489083 ## streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsMetadataState.java: ## @@ -459,12 +461,23 @@ private List

[GitHub] [kafka] gharris1727 commented on a diff in pull request #12866: KAFKA-14346: Remove hard-to-mock javax.crypto calls

2022-11-17 Thread GitBox
gharris1727 commented on code in PR #12866: URL: https://github.com/apache/kafka/pull/12866#discussion_r1025479365 ## connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/CryptoLibrary.java: ## @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software

[GitHub] [kafka] dajac commented on pull request #12870: KAFKA-14367; Add `OffsetFetch` to the new `GroupCoordinator` interface

2022-11-17 Thread GitBox
dajac commented on PR #12870: URL: https://github.com/apache/kafka/pull/12870#issuecomment-1318954178 I have to add a few unit tests in KafkaApisTest. There are none at the moment. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to

[jira] [Updated] (KAFKA-14398) Update EndToEndAuthorizerTest.scala to test with ZK and KRAFT quorum servers

2022-11-17 Thread Manikumar (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14398?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar updated KAFKA-14398: -- Fix Version/s: 3.4.0 > Update EndToEndAuthorizerTest.scala to test with ZK and KRAFT quorum servers >

[jira] [Updated] (KAFKA-14375) Remove use of "authorizer-properties" in EndToEndAuthorizationTest.scala

2022-11-17 Thread Manikumar (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14375?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar updated KAFKA-14375: -- Fix Version/s: 3.4.0 > Remove use of "authorizer-properties" in EndToEndAuthorizationTest.scala >

[jira] [Resolved] (KAFKA-14375) Remove use of "authorizer-properties" in EndToEndAuthorizationTest.scala

2022-11-17 Thread Proven Provenzano (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14375?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Proven Provenzano resolved KAFKA-14375. --- Resolution: Fixed > Remove use of "authorizer-properties" in

[GitHub] [kafka] gharris1727 commented on a diff in pull request #12828: KAFKA-14346: Remove hard-to-mock RestClient calls

2022-11-17 Thread GitBox
gharris1727 commented on code in PR #12828: URL: https://github.com/apache/kafka/pull/12828#discussion_r1025464638 ## connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestClient.java: ## @@ -43,10 +43,38 @@ import java.util.concurrent.ExecutionException;

[jira] [Assigned] (KAFKA-14398) Update EndToEndAuthorizerTest.scala to test with ZK and KRAFT quorum servers

2022-11-17 Thread Proven Provenzano (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14398?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Proven Provenzano reassigned KAFKA-14398: - Assignee: Proven Provenzano > Update EndToEndAuthorizerTest.scala to test with

[jira] [Created] (KAFKA-14398) Update EndToEndAuthorizerTest.scala to test with ZK and KRAFT quorum servers

2022-11-17 Thread Proven Provenzano (Jira)
Proven Provenzano created KAFKA-14398: - Summary: Update EndToEndAuthorizerTest.scala to test with ZK and KRAFT quorum servers Key: KAFKA-14398 URL: https://issues.apache.org/jira/browse/KAFKA-14398

[GitHub] [kafka] mumrah merged pull request #12860: Add RPC changes, records, and config from KIP-866

2022-11-17 Thread GitBox
mumrah merged PR #12860: URL: https://github.com/apache/kafka/pull/12860 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail:

[GitHub] [kafka] omkreddy merged pull request #12843: KAFKA-14375: Remove use of "authorizer-properties" from EndToEndAuthorizerTest

2022-11-17 Thread GitBox
omkreddy merged PR #12843: URL: https://github.com/apache/kafka/pull/12843 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail:

[GitHub] [kafka] C0urante commented on a diff in pull request #12866: KAFKA-14346: Remove hard-to-mock javax.crypto calls

2022-11-17 Thread GitBox
C0urante commented on code in PR #12866: URL: https://github.com/apache/kafka/pull/12866#discussion_r1025383865 ## connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/CryptoLibrary.java: ## @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software

[GitHub] [kafka] mumrah commented on a diff in pull request #12815: KIP-866 Part 1

2022-11-17 Thread GitBox
mumrah commented on code in PR #12815: URL: https://github.com/apache/kafka/pull/12815#discussion_r1025397641 ## core/src/main/scala/kafka/migration/ZkMigrationClient.scala: ## @@ -0,0 +1,359 @@ +package kafka.migration + +import kafka.api.LeaderAndIsr +import

[GitHub] [kafka] vvcephei commented on a diff in pull request #12869: KAFKA-14382: wait for current rebalance to complete before triggering followup

2022-11-17 Thread GitBox
vvcephei commented on code in PR #12869: URL: https://github.com/apache/kafka/pull/12869#discussion_r1025368358 ## streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java: ## @@ -298,6 +298,7 @@ public boolean isRunning() { private volatile

[GitHub] [kafka] C0urante commented on a diff in pull request #12828: KAFKA-14346: Remove hard-to-mock RestClient calls

2022-11-17 Thread GitBox
C0urante commented on code in PR #12828: URL: https://github.com/apache/kafka/pull/12828#discussion_r1025340543 ## connect/runtime/src/test/java/org/apache/kafka/connect/integration/RestForwardingIntegrationTest.java: ## @@ -0,0 +1,256 @@ +/* + * Licensed to the Apache Software

[GitHub] [kafka] C0urante commented on a diff in pull request #12828: KAFKA-14346: Remove hard-to-mock RestClient calls

2022-11-17 Thread GitBox
C0urante commented on code in PR #12828: URL: https://github.com/apache/kafka/pull/12828#discussion_r1025336265 ## connect/runtime/src/test/java/org/apache/kafka/connect/integration/RestForwardingIntegrationTest.java: ## @@ -0,0 +1,256 @@ +/* + * Licensed to the Apache Software

[GitHub] [kafka] dajac opened a new pull request, #12870: KAFKA-14367; Add `OffsetFetch` to the new `GroupCoordinator` interface

2022-11-17 Thread GitBox
dajac opened a new pull request, #12870: URL: https://github.com/apache/kafka/pull/12870 This patch adds `OffsetFetch` to the new `GroupCoordinator` interface and updates `KafkaApis` to use it. The changes in `KafkaApis` are larger than what I was hoping for. I think that we should

[GitHub] [kafka] divijvaidya commented on pull request #12590: KAFKA-7109: Close fetch sessions on close of consumer

2022-11-17 Thread GitBox
divijvaidya commented on PR #12590: URL: https://github.com/apache/kafka/pull/12590#issuecomment-1318726600 @dajac please take a look! -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the

[jira] [Commented] (KAFKA-14362) Same message consumed by two consumers in the same group after client restart

2022-11-17 Thread Mikael (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14362?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17635370#comment-17635370 ] Mikael commented on KAFKA-14362: When comparing successful test runs with those that generate duplicate

[jira] [Updated] (KAFKA-9156) LazyTimeIndex & LazyOffsetIndex may cause niobufferoverflow in concurrent state

2022-11-17 Thread Bob Tiernay (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9156?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Bob Tiernay updated KAFKA-9156: --- Attachment: image-2022-11-17-09-02-20-774.png > LazyTimeIndex & LazyOffsetIndex may cause

[jira] [Commented] (KAFKA-9156) LazyTimeIndex & LazyOffsetIndex may cause niobufferoverflow in concurrent state

2022-11-17 Thread Bob Tiernay (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9156?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17635367#comment-17635367 ] Bob Tiernay commented on KAFKA-9156: We too are hitting this on AWS MSK Kafka version 2.8.1:

[jira] [Comment Edited] (KAFKA-6579) Consolidate window store and session store unit tests into a single class

2022-11-17 Thread Ahmed Sobeh (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-6579?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17635340#comment-17635340 ] Ahmed Sobeh edited comment on KAFKA-6579 at 11/17/22 12:44 PM: --- will give

[jira] [Commented] (KAFKA-6579) Consolidate window store and session store unit tests into a single class

2022-11-17 Thread Ahmed Sobeh (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-6579?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17635340#comment-17635340 ] Ahmed Sobeh commented on KAFKA-6579: will give this a shoft if it's ok [~teamurko]  > Consolidate

[jira] [Assigned] (KAFKA-10409) Refactor Kafka Streams RocksDb iterators

2022-11-17 Thread Ahmed Sobeh (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10409?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ahmed Sobeh reassigned KAFKA-10409: --- Assignee: (was: Ahmed Sobeh) > Refactor Kafka Streams RocksDb iterators >

[jira] (KAFKA-10409) Refactor Kafka Streams RocksDb iterators

2022-11-17 Thread Ahmed Sobeh (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10409 ] Ahmed Sobeh deleted comment on KAFKA-10409: - was (Author: JIRAUSER295920): WIll pick this up and take a look if it's ok with everyone watching > Refactor Kafka Streams RocksDb iterators >

[jira] [Assigned] (KAFKA-10409) Refactor Kafka Streams RocksDb iterators

2022-11-17 Thread Ahmed Sobeh (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10409?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ahmed Sobeh reassigned KAFKA-10409: --- Assignee: Ahmed Sobeh > Refactor Kafka Streams RocksDb iterators >

[jira] [Commented] (KAFKA-10409) Refactor Kafka Streams RocksDb iterators

2022-11-17 Thread Ahmed Sobeh (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10409?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17635338#comment-17635338 ] Ahmed Sobeh commented on KAFKA-10409: - WIll pick this up and take a look if it's ok with everyone

[GitHub] [kafka] vamossagar12 commented on pull request #12802: KAFKA-14311: Connect Worker clean shutdown does not cleanly stop connectors/tasks

2022-11-17 Thread GitBox
vamossagar12 commented on PR #12802: URL: https://github.com/apache/kafka/pull/12802#issuecomment-1318504959 @gharris1727 , i believe this ticket was created by you. Can you take a look at this PR? -- This is an automated message from the Apache Git Service. To respond to the message,

[GitHub] [kafka] vamossagar12 closed pull request #12826: Using Timer class to track expiry in IncrementalCooperativeAssignor

2022-11-17 Thread GitBox
vamossagar12 closed pull request #12826: Using Timer class to track expiry in IncrementalCooperativeAssignor URL: https://github.com/apache/kafka/pull/12826 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above

[GitHub] [kafka] vamossagar12 commented on pull request #12826: Using Timer class to track expiry in IncrementalCooperativeAssignor

2022-11-17 Thread GitBox
vamossagar12 commented on PR #12826: URL: https://github.com/apache/kafka/pull/12826#issuecomment-1318417587 @gharris1727 Thanks for the confirmation! Let me close this PR. BTW: `we can leave this long as-is for a little longer.` this was nice :) -- This is an automated message

[GitHub] [kafka] patrik-marton commented on a diff in pull request #12846: KAFKA-14293: Basic Auth filter should set the SecurityContext after a…

2022-11-17 Thread GitBox
patrik-marton commented on code in PR #12846: URL: https://github.com/apache/kafka/pull/12846#discussion_r1024994387 ## connect/basic-auth-extension/src/main/java/org/apache/kafka/connect/rest/basic/auth/extension/JaasBasicAuthFilter.java: ## @@ -174,4 +153,84 @@ public void

[GitHub] [kafka] patrik-marton commented on a diff in pull request #12846: KAFKA-14293: Basic Auth filter should set the SecurityContext after a…

2022-11-17 Thread GitBox
patrik-marton commented on code in PR #12846: URL: https://github.com/apache/kafka/pull/12846#discussion_r1024987213 ## connect/basic-auth-extension/src/main/java/org/apache/kafka/connect/rest/basic/auth/extension/JaasBasicAuthFilter.java: ## @@ -174,4 +153,84 @@ public void

[GitHub] [kafka] mimaison merged pull request #12839: KAFKA-14346: Replace static mocking of WorkerConfig::lookupKafkaClusterId

2022-11-17 Thread GitBox
mimaison merged PR #12839: URL: https://github.com/apache/kafka/pull/12839 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail:

[GitHub] [kafka] dajac commented on pull request #12845: KAFKA-14367; Add `JoinGroup` to the new `GroupCoordinator` interface

2022-11-17 Thread GitBox
dajac commented on PR #12845: URL: https://github.com/apache/kafka/pull/12845#issuecomment-1318257181 Rebased the PR. Ready for second round. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the