[
https://issues.apache.org/jira/browse/KAFKA-14729?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17690218#comment-17690218
]
Luke Chen commented on KAFKA-14729:
---
The change makes sense to me. But I'm more intere
[
https://issues.apache.org/jira/browse/KAFKA-14713?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17690215#comment-17690215
]
Tamas commented on KAFKA-14713:
---
The difference (at least for my use case) is on the Kafka
[
https://issues.apache.org/jira/browse/KAFKA-14729?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
RivenSun updated KAFKA-14729:
-
Description:
h2. case situation:
1. The business program occupies a large amount of memory, causing the
chia7712 commented on code in PR #13266:
URL: https://github.com/apache/kafka/pull/13266#discussion_r1109343158
##
connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/entities/PluginInfoTest.java:
##
@@ -19,17 +19,19 @@
import org.apache.kafka.connect.runtime.is
[
https://issues.apache.org/jira/browse/KAFKA-14729?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17690166#comment-17690166
]
RivenSun edited comment on KAFKA-14729 at 2/17/23 6:06 AM:
---
1.
[
https://issues.apache.org/jira/browse/KAFKA-14729?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17690166#comment-17690166
]
RivenSun commented on KAFKA-14729:
--
1. After careful consideration, in the case of Kafk
C0urante commented on PR #13266:
URL: https://github.com/apache/kafka/pull/13266#issuecomment-1434145501
@chia7712 would you have a moment to take a look at this?
I'm going to bed now; feel free to merge if everything looks alright.
--
This is an automated message from the Apache Gi
[
https://issues.apache.org/jira/browse/KAFKA-14729?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17690151#comment-17690151
]
RivenSun commented on KAFKA-14729:
--
Hi [~guozhang] , [~showuon]
Could you give some s
[
https://issues.apache.org/jira/browse/KAFKA-14729?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
RivenSun updated KAFKA-14729:
-
Affects Version/s: (was: 3.3.2)
> The kafakConsumer pollForFetches(timer) method takes up a lot of c
[
https://issues.apache.org/jira/browse/KAFKA-14729?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
RivenSun updated KAFKA-14729:
-
Description:
h2. case situation:
1. The business program occupies a large amount of memory, causing the
RivenSun created KAFKA-14729:
Summary: The kafakConsumer pollForFetches(timer) method takes up a
lot of cpu due to the abnormal exit of the heartbeat thread
Key: KAFKA-14729
URL: https://issues.apache.org/jira/browse/
philipnee opened a new pull request, #13269:
URL: https://github.com/apache/kafka/pull/13269
*We want to ensure checkpointing the progress after completing the
restoration to prevent losing the progress and needing to restore from scratch
### Committer Checklist (excluded from commit
kowshik commented on PR #13268:
URL: https://github.com/apache/kafka/pull/13268#issuecomment-1434094602
Hi @junrao and @satishd -- Please could you help review this PR?
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use t
kowshik opened a new pull request, #13268:
URL: https://github.com/apache/kafka/pull/13268
Some of the `LeaderEndpoint` interface methods used a tuple of `(Int, Long)`
as return value to represent epoch & offset. In this PR, I have replaced those
occcurences with the existing `OffsetAndEpoc
philipnee commented on code in PR #13265:
URL: https://github.com/apache/kafka/pull/13265#discussion_r1109286591
##
clients/src/main/java/org/apache/kafka/clients/consumer/StubbedAsyncKafkaConsumer.java:
##
@@ -0,0 +1,548 @@
+/*
+ * Licensed to the Apache Software Foundation (AS
philipnee commented on code in PR #13265:
URL: https://github.com/apache/kafka/pull/13265#discussion_r1109283896
##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEvent.java:
##
@@ -19,23 +19,35 @@
/**
* This is the abstract definition of
vcrfxia commented on code in PR #13250:
URL: https://github.com/apache/kafka/pull/13250#discussion_r1109282015
##
streams/src/main/java/org/apache/kafka/streams/state/internals/VersionedKeyValueToBytesStoreAdapter.java:
##
@@ -0,0 +1,196 @@
+/*
+ * Licensed to the Apache Softwar
philipnee commented on code in PR #13265:
URL: https://github.com/apache/kafka/pull/13265#discussion_r1109264154
##
clients/src/main/java/org/apache/kafka/clients/consumer/StubbedAsyncKafkaConsumer.java:
##
@@ -0,0 +1,548 @@
+/*
+ * Licensed to the Apache Software Foundation (AS
vcrfxia commented on code in PR #13250:
URL: https://github.com/apache/kafka/pull/13250#discussion_r1109276286
##
streams/src/main/java/org/apache/kafka/streams/state/internals/VersionedKeyValueToBytesStoreAdapter.java:
##
@@ -0,0 +1,196 @@
+/*
+ * Licensed to the Apache Softwar
showuon merged PR #13234:
URL: https://github.com/apache/kafka/pull/13234
--
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-unsubscr...@kafka.apache.
[
https://issues.apache.org/jira/browse/KAFKA-14719?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
shenxingwuying updated KAFKA-14719:
---
Summary: Fix a return code when broker bootstrapped just now. (was: Talk
about a problem: a
philipnee commented on code in PR #13265:
URL: https://github.com/apache/kafka/pull/13265#discussion_r1109260958
##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java:
##
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Softw
satishd commented on PR #13234:
URL: https://github.com/apache/kafka/pull/13234#issuecomment-1434009975
One failing test is not related to this change, ready to be merged.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and us
showuon commented on PR #13234:
URL: https://github.com/apache/kafka/pull/13234#issuecomment-1433985820
I'll take a look today.
--
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 commen
guozhangwang commented on code in PR #13192:
URL: https://github.com/apache/kafka/pull/13192#discussion_r1109189751
##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetFetcher.java:
##
@@ -0,0 +1,717 @@
+/*
+ * Licensed to the Apache Software Foundation (A
guozhangwang commented on code in PR #13192:
URL: https://github.com/apache/kafka/pull/13192#discussion_r1109189406
##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/MetadataFetcher.java:
##
@@ -0,0 +1,805 @@
+/*
+ * Licensed to the Apache Software Foundation
jeffkbkim opened a new pull request, #13267:
URL: https://github.com/apache/kafka/pull/13267
RPCProducerIdManager initiates an async request to the controller to grab a
block of producer IDs and then blocks waiting for a response from the
controller.
This is done in the request handl
junrao commented on code in PR #13255:
URL: https://github.com/apache/kafka/pull/13255#discussion_r1109170546
##
storage/src/main/java/org/apache/kafka/storage/internals/log/LogAppendInfo.java:
##
@@ -0,0 +1,324 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under o
guozhangwang commented on code in PR #13025:
URL: https://github.com/apache/kafka/pull/13025#discussion_r1109175366
##
streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java:
##
@@ -3159,7 +3159,7 @@ private void addRecord(final MockConsumer
mo
mjsax commented on code in PR #13251:
URL: https://github.com/apache/kafka/pull/13251#discussion_r1109171225
##
streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingVersionedKeyValueBytesStore.java:
##
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Softwa
guozhangwang commented on PR #13190:
URL: https://github.com/apache/kafka/pull/13190#issuecomment-1433949296
> is it intentional to continue w/o sleep on the backoff timer?
Yes that's intentional. For those four exceptions, we'd like to send the
follow-up request right away since the
mjsax commented on code in PR #13250:
URL: https://github.com/apache/kafka/pull/13250#discussion_r1109169480
##
streams/src/main/java/org/apache/kafka/streams/state/internals/VersionedKeyValueToBytesStoreAdapter.java:
##
@@ -0,0 +1,196 @@
+/*
+ * Licensed to the Apache Software
[
https://issues.apache.org/jira/browse/KAFKA-14253?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Guozhang Wang resolved KAFKA-14253.
---
Fix Version/s: 3.5.0
Resolution: Fixed
> StreamsPartitionAssignor should print the me
guozhangwang merged PR #13253:
URL: https://github.com/apache/kafka/pull/13253
--
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-unsubscr...@kafka.ap
bachmanity1 commented on PR #13261:
URL: https://github.com/apache/kafka/pull/13261#issuecomment-1433923878
@ijuma could you please review this? Thanks!
--
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
C0urante commented on PR #13262:
URL: https://github.com/apache/kafka/pull/13262#issuecomment-1433903556
Backported to 3.4. There were merge conflicts in the
`ExactlyOnceWorkerSourceTaskTest` suite (caused by the Mockito migration) on
3.3; I may revisit at some point if I can find the time,
[
https://issues.apache.org/jira/browse/KAFKA-14727?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Chris Egerton updated KAFKA-14727:
--
Fix Version/s: 3.4.1
> Connect EOS mode should periodically call task commit
> ---
C0urante commented on PR #13261:
URL: https://github.com/apache/kafka/pull/13261#issuecomment-1433898716
Hi @bachmanity1! This is a bit outside my area of expertise and it's
probably best for another committer to review.
--
This is an automated message from the Apache Git Service.
To resp
[
https://issues.apache.org/jira/browse/KAFKA-14727?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Chris Egerton resolved KAFKA-14727.
---
Fix Version/s: 3.5.0
Resolution: Fixed
> Connect EOS mode should periodically call ta
[
https://issues.apache.org/jira/browse/KAFKA-14713?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17690055#comment-17690055
]
Matthias J. Sax edited comment on KAFKA-14713 at 2/16/23 11:52 PM:
---
[
https://issues.apache.org/jira/browse/KAFKA-14713?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17690055#comment-17690055
]
Matthias J. Sax commented on KAFKA-14713:
-
Thanks for getting back. Glad it's re
C0urante merged PR #13262:
URL: https://github.com/apache/kafka/pull/13262
--
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-unsubscr...@kafka.apache
C0urante commented on PR #13262:
URL: https://github.com/apache/kafka/pull/13262#issuecomment-1433896571
CI looks good, with some acceptable exceptions:
- Failures due to the recent changes to the Connect `PluginInfoTest`, which
are addressed separately in https://github.com/apache/ka
C0urante opened a new pull request, #13266:
URL: https://github.com/apache/kafka/pull/13266
The refactorings in https://github.com/apache/kafka/pull/13219 are largely
very helpful; however, the changes to the `PluginInfoTest` suite have caused
the test to begin failing.
This PR rever
bachmanity1 commented on PR #13246:
URL: https://github.com/apache/kafka/pull/13246#issuecomment-1433802673
@showuon
> Basically, the size > buffer.remaining() check is originally not trying to
check if the size can be allocated in the JVM, it's just a sanity check to see
if this size i
kirktrue commented on code in PR #13192:
URL: https://github.com/apache/kafka/pull/13192#discussion_r1109086646
##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetFetcher.java:
##
@@ -0,0 +1,717 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF)
bachmanity1 commented on PR #13261:
URL: https://github.com/apache/kafka/pull/13261#issuecomment-1433794388
@showuon @C0urante could you please review this? Thanks.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
kirktrue opened a new pull request, #13265:
URL: https://github.com/apache/kafka/pull/13265
*More detailed description of your change,
if necessary. The PR title and PR message become
the squashed commit message, so use a separate
comment to ping reviewers.*
*Summary of testin
dejan2609 commented on PR #13263:
URL: https://github.com/apache/kafka/pull/13263#issuecomment-1433744998
Only handfull of tests failed (those errors are obviosly unrelated to this
small change).
FYI @ijuma
--
This is an automated message from the Apache Git Service.
To respond to
vcrfxia opened a new pull request, #13264:
URL: https://github.com/apache/kafka/pull/13264
As part of introducing versioned key-value stores in
[KIP-889](https://cwiki.apache.org/confluence/display/KAFKA/KIP-889%3A+Versioned+State+Stores),
we want to lift the existing DSL restriction that K
[
https://issues.apache.org/jira/browse/KAFKA-14728?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Dejan Stojadinović updated KAFKA-14728:
---
Description:
*Note*: this ticket blocks Gradle major version upgrade (_*7 -->> 8*_):
philipnee commented on PR #13253:
URL: https://github.com/apache/kafka/pull/13253#issuecomment-1433651333
Failures seem unrelated
```
Build / JDK 11 and Scala 2.13 /
testReplicationIsCreatingTopicsUsingProvidedForwardingAdmin() –
org.apache.kafka.connect.mirror.integration.MirrorConne
[
https://issues.apache.org/jira/browse/KAFKA-14727?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Chris Egerton updated KAFKA-14727:
--
Affects Version/s: 3.3.2
3.3.1
3.3.0
> Connect E
C0urante commented on PR #13262:
URL: https://github.com/apache/kafka/pull/13262#issuecomment-1433640900
> This is covered by the existing `testPollReturnsNoRecords` where the two
arguments to verifyTransactions are different.
Ah yes, thanks. Sorry for missing that.
> Also the
gharris1727 commented on PR #13262:
URL: https://github.com/apache/kafka/pull/13262#issuecomment-1433589831
> Could we also add a case to ExactlyOnceWorkerSourceTaskTest that ensures
that SourceTask::commit is invoked even if the task doesn't produce any records?
This is covered by th
[
https://issues.apache.org/jira/browse/KAFKA-14722?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17689379#comment-17689379
]
Spacrocket edited comment on KAFKA-14722 at 2/16/23 7:02 PM:
-
pprovenzano commented on code in PR #13114:
URL: https://github.com/apache/kafka/pull/13114#discussion_r1108894205
##
core/src/test/scala/unit/kafka/server/AlterUserScramCredentialsRequestTest.scala:
##
@@ -260,11 +271,13 @@ class AlterUserScramCredentialsRequestTest extends
Ba
philipnee commented on code in PR #13192:
URL: https://github.com/apache/kafka/pull/13192#discussion_r1107931054
##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetFetcher.java:
##
@@ -0,0 +1,717 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF)
dajac commented on code in PR #13196:
URL: https://github.com/apache/kafka/pull/13196#discussion_r1108852113
##
core/src/main/scala/kafka/cluster/Partition.scala:
##
@@ -44,11 +44,40 @@ import org.apache.kafka.common.utils.Time
import org.apache.kafka.common.{IsolationLevel, To
[
https://issues.apache.org/jira/browse/KAFKA-14565?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Chris Egerton updated KAFKA-14565:
--
Fix Version/s: 3.3.3
> Interceptor Resource Leak
> -
>
>
jolshan commented on code in PR #13196:
URL: https://github.com/apache/kafka/pull/13196#discussion_r1108835492
##
core/src/main/scala/kafka/cluster/Partition.scala:
##
@@ -44,11 +44,40 @@ import org.apache.kafka.common.utils.Time
import org.apache.kafka.common.{IsolationLevel,
[
https://issues.apache.org/jira/browse/KAFKA-14565?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Chris Egerton updated KAFKA-14565:
--
Fix Version/s: 3.4.1
> Interceptor Resource Leak
> -
>
>
[
https://issues.apache.org/jira/browse/KAFKA-14565?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Chris Egerton resolved KAFKA-14565.
---
Resolution: Fixed
> Interceptor Resource Leak
> -
>
>
C0urante merged PR #13168:
URL: https://github.com/apache/kafka/pull/13168
--
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-unsubscr...@kafka.apache
dejan2609 opened a new pull request, #13263:
URL: https://github.com/apache/kafka/pull/13263
Blocks this PR: #13205
Related JIRA ticket: https://issues.apache.org/jira/browse/KAFKA-14728
Rationale:
- build works fine in trunk with Gradle 7.6 and spotless gradle plugin
6.13.0
C0urante commented on code in PR #13262:
URL: https://github.com/apache/kafka/pull/13262#discussion_r1108806764
##
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ExactlyOnceWorkerSourceTask.java:
##
@@ -258,6 +258,21 @@ private void commitTransaction() {
Dejan Stojadinović created KAFKA-14728:
--
Summary: Remove 'spotlessScalaCheck' task (out of Jenkinsfile)
Key: KAFKA-14728
URL: https://issues.apache.org/jira/browse/KAFKA-14728
Project: Kafka
gharris1727 commented on PR #13178:
URL: https://github.com/apache/kafka/pull/13178#issuecomment-1433443867
I've split the periodic commit fix into a separate PR
(https://github.com/apache/kafka/pull/13262) as it is not related to the
changes here, only the test changes. This PR depends on
gharris1727 opened a new pull request, #13262:
URL: https://github.com/apache/kafka/pull/13262
Source tasks in non-EOS mode periodically call SourceTask::commit even if no
records are returned from poll.
This change adds that behavior to EOS mode by considering empty batches to
be dispat
Greg Harris created KAFKA-14727:
---
Summary: Connect EOS mode should periodically call task commit
Key: KAFKA-14727
URL: https://issues.apache.org/jira/browse/KAFKA-14727
Project: Kafka
Issue Typ
bachmanity1 opened a new pull request, #13261:
URL: https://github.com/apache/kafka/pull/13261
After reading data of type `BYTES`, `COMPACT_BYTES`, `NULLABLE_BYTES` or
`COMPACT_NULLABLE_BYTES` returned `ByteBuffer` might have a capacity that is
larger than its limit, thus these data types m
dajac commented on code in PR #13196:
URL: https://github.com/apache/kafka/pull/13196#discussion_r1108755781
##
core/src/main/scala/kafka/cluster/Partition.scala:
##
@@ -285,6 +314,17 @@ class Partition(val topicPartition: TopicPartition,
// If ReplicaAlterLogDir command is i
dajac commented on code in PR #13196:
URL: https://github.com/apache/kafka/pull/13196#discussion_r1108755031
##
core/src/main/scala/kafka/cluster/Partition.scala:
##
@@ -44,11 +44,40 @@ import org.apache.kafka.common.utils.Time
import org.apache.kafka.common.{IsolationLevel, To
dajac commented on code in PR #13196:
URL: https://github.com/apache/kafka/pull/13196#discussion_r1108753117
##
core/src/main/scala/kafka/cluster/Partition.scala:
##
@@ -44,11 +44,40 @@ import org.apache.kafka.common.utils.Time
import org.apache.kafka.common.{IsolationLevel, To
dajac commented on code in PR #13196:
URL: https://github.com/apache/kafka/pull/13196#discussion_r1108752686
##
core/src/main/scala/kafka/cluster/Partition.scala:
##
@@ -318,6 +358,25 @@ class Partition(val topicPartition: TopicPartition,
def inSyncReplicaIds: Set[Int] = pa
vcrfxia commented on code in PR #13252:
URL: https://github.com/apache/kafka/pull/13252#discussion_r1108726927
##
streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredVersionedKeyValueStore.java:
##
@@ -0,0 +1,227 @@
+/*
+ * Licensed to the Apache Software Found
vcrfxia commented on code in PR #13252:
URL: https://github.com/apache/kafka/pull/13252#discussion_r1107700533
##
streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredVersionedKeyValueStore.java:
##
@@ -0,0 +1,226 @@
+/*
+ * Licensed to the Apache Software Found
C0urante commented on PR #13178:
URL: https://github.com/apache/kafka/pull/13178#issuecomment-1433313709
Hmmm... we make an effort to periodically invoke `SourceTask::commit` in
non-EOS mode, even if there have been no new records written and there are no
offsets to commit. I think we shoul
mimaison merged PR #13219:
URL: https://github.com/apache/kafka/pull/13219
--
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-unsubscr...@kafka.apache
fvaleri commented on PR #13127:
URL: https://github.com/apache/kafka/pull/13127#issuecomment-1433155607
> Regarding moving to CommandDefaultOptions, is it better if we do it in a
follow up PR as that is not directly connected to migrating to tools module.
Hi @vamossagar12, I'm ok with
ijuma commented on PR #13234:
URL: https://github.com/apache/kafka/pull/13234#issuecomment-1433083812
I won't have a chance to review for a few more days - if @showuon can do it
sooner, let's not block on me.
--
This is an automated message from the Apache Git Service.
To respond to the m
ijuma commented on PR #13205:
URL: https://github.com/apache/kafka/pull/13205#issuecomment-1433081403
Yeah, that's a security feature. I suggest filing a JIRA about disabling
spotlessScalaCheck with the rationale and have a single PR for that change.
--
This is an automated message from t
satishd merged PR #13235:
URL: https://github.com/apache/kafka/pull/13235
--
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-unsubscr...@kafka.apache.
satishd commented on PR #13234:
URL: https://github.com/apache/kafka/pull/13234#issuecomment-1433080348
Thanks @ijuma and @showuon for the review. Resolved the conflicts and pushed
the changes.
--
This is an automated message from the Apache Git Service.
To respond to the message, please
satishd commented on PR #13235:
URL: https://github.com/apache/kafka/pull/13235#issuecomment-1433078952
Test failures are not related to this change.
--
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
tinaselenge commented on PR #13172:
URL: https://github.com/apache/kafka/pull/13172#issuecomment-1433059514
Thank you @fvaleri and @clolov for reviewing the PR.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL
clolov opened a new pull request, #13260:
URL: https://github.com/apache/kafka/pull/13260
Implementation of
https://cwiki.apache.org/confluence/display/KAFKA/KIP-902%3A+Upgrade+Zookeeper+to+3.8.1
--
This is an automated message from the Apache Git Service.
To respond to the message, pleas
vamossagar12 commented on PR #13095:
URL: https://github.com/apache/kafka/pull/13095#issuecomment-1433009209
Tests passed.
--
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.
T
[
https://issues.apache.org/jira/browse/KAFKA-14698?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17689733#comment-17689733
]
Luke Chen commented on KAFKA-14698:
---
[~akhileshchg] , are you still up to this issue?
nizhikov commented on PR #13247:
URL: https://github.com/apache/kafka/pull/13247#issuecomment-1432995629
@mimaison Please, share your feedback. Are you ready to review and merge
current changes (only case classes and options moved to java code) or should I
continue work and rewrite the whol
dejan2609 commented on PR #13205:
URL: https://github.com/apache/kafka/pull/13205#issuecomment-1432981371
🤔 Interesting, altough 'spotlessScalaCheck' is removed out of Jenkinsfile
that task is still executed on Jenkins:
![image](https://user-images.githubusercontent.com/19467899/219
clolov opened a new pull request, #13259:
URL: https://github.com/apache/kafka/pull/13259
I ran IntelliJ IDEA's Code Inspection and corrected occurrences of the
following:
*
https://www.jetbrains.com/help/idea/list-of-java-inspections.html#inheritance-issues
(method is identical to its
fvaleri commented on code in PR #13214:
URL: https://github.com/apache/kafka/pull/13214#discussion_r1108345004
##
tools/src/main/java/org/apache/kafka/tools/MessageReader.java:
##
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * con
omkreddy commented on code in PR #13114:
URL: https://github.com/apache/kafka/pull/13114#discussion_r1108339368
##
core/src/test/scala/unit/kafka/server/DescribeUserScramCredentialsRequestTest.scala:
##
@@ -17,18 +17,22 @@
package kafka.server
import java.util
-import java.u
mimaison commented on code in PR #13257:
URL: https://github.com/apache/kafka/pull/13257#discussion_r1108283525
##
docs/ops.html:
##
@@ -3550,22 +3550,220 @@ Deploying
Considerations
-Kafka server's process.role should be set to either
broker or controller but not bo
mimaison commented on code in PR #13256:
URL: https://github.com/apache/kafka/pull/13256#discussion_r1108269059
##
docs/ops.html:
##
@@ -3550,22 +3550,220 @@ Deploying
Considerations
-Kafka server's process.role should be set to either
broker or controller but not bo
[
https://issues.apache.org/jira/browse/KAFKA-14304?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17689676#comment-17689676
]
Luke Chen commented on KAFKA-14304:
---
[~mumrah] , I'd like to contribute to this featur
showuon commented on PR #13258:
URL: https://github.com/apache/kafka/pull/13258#issuecomment-1432842837
@mumrah @akhileshchg , call for review. Thanks.
--
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 g
showuon opened a new pull request, #13258:
URL: https://github.com/apache/kafka/pull/13258
Checked the code, and confirmed the controlled shutdown for ZK brokers
during migration can either send to ZK controller, or send to KRaft controller
via lifecycle manager based on the controller is Z
[
https://issues.apache.org/jira/browse/KAFKA-14447?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Luke Chen reassigned KAFKA-14447:
-
Assignee: Luke Chen
> Controlled shutdown for ZK brokers during migration
> ---
1 - 100 of 107 matches
Mail list logo