satishd commented on code in PR #14382:
URL: https://github.com/apache/kafka/pull/14382#discussion_r1325382836
##
docs/ops.html:
##
@@ -3859,6 +3859,98 @@ Finalizing the migration
# Other configs ...
+
+6.11 Tiered Storage
+
+Tiered Storage
Overview
+
+Kafka data is
satishd commented on code in PR #14382:
URL: https://github.com/apache/kafka/pull/14382#discussion_r1325382836
##
docs/ops.html:
##
@@ -3859,6 +3859,98 @@ Finalizing the migration
# Other configs ...
+
+6.11 Tiered Storage
+
+Tiered Storage
Overview
+
+Kafka data is
philipnee commented on code in PR #14364:
URL: https://github.com/apache/kafka/pull/14364#discussion_r1325383326
##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManager.java:
##
@@ -0,0 +1,260 @@
+/*
+ * Licensed to the Apache Software
philipnee commented on code in PR #14364:
URL: https://github.com/apache/kafka/pull/14364#discussion_r1325382420
##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManager.java:
##
@@ -0,0 +1,260 @@
+/*
+ * Licensed to the Apache Software
philipnee commented on code in PR #14364:
URL: https://github.com/apache/kafka/pull/14364#discussion_r1325381139
##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManager.java:
##
@@ -0,0 +1,260 @@
+/*
+ * Licensed to the Apache Software
showuon commented on PR #14382:
URL: https://github.com/apache/kafka/pull/14382#issuecomment-1718792709
@satishd , in the latest commit, I added:
1. a note to say this is an early access feature and should not use in
production env.
2. a limitation section to describe the limitation
philipnee commented on code in PR #14364:
URL: https://github.com/apache/kafka/pull/14364#discussion_r1325368170
##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/DefaultBackgroundThread.java:
##
@@ -168,28 +171,41 @@ public DefaultBackgroundThread(final Time
philipnee commented on PR #14386:
URL: https://github.com/apache/kafka/pull/14386#issuecomment-1718779561
Hey @junrao - Addressed your comments. I'm holding off on your request for
`Could we change toString to include the fields in the parent class?` as I
think it would be easier if I do
philipnee commented on code in PR #14386:
URL: https://github.com/apache/kafka/pull/14386#discussion_r1325364414
##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManager.java:
##
@@ -0,0 +1,210 @@
+/*
+ * Licensed to the Apache Software
philipnee commented on code in PR #14386:
URL: https://github.com/apache/kafka/pull/14386#discussion_r1325355774
##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManager.java:
##
@@ -0,0 +1,210 @@
+/*
+ * Licensed to the Apache Software
philipnee commented on code in PR #14386:
URL: https://github.com/apache/kafka/pull/14386#discussion_r1325353270
##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java:
##
@@ -203,16 +204,17 @@ public static class UnsentRequest {
philipnee commented on code in PR #14386:
URL: https://github.com/apache/kafka/pull/14386#discussion_r1325352296
##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java:
##
@@ -381,41 +377,41 @@ public String toString() {
}
/**
philipnee commented on code in PR #14386:
URL: https://github.com/apache/kafka/pull/14386#discussion_r1325349813
##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java:
##
@@ -106,6 +104,7 @@ public NetworkClientDelegate.PollResult
philipnee commented on code in PR #14386:
URL: https://github.com/apache/kafka/pull/14386#discussion_r1325349677
##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitApplicationEvent.java:
##
@@ -19,22 +19,23 @@
import
philipnee commented on code in PR #14386:
URL: https://github.com/apache/kafka/pull/14386#discussion_r1325345190
##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManager.java:
##
@@ -0,0 +1,210 @@
+/*
+ * Licensed to the Apache Software
philipnee commented on code in PR #14386:
URL: https://github.com/apache/kafka/pull/14386#discussion_r1325342579
##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManager.java:
##
@@ -0,0 +1,210 @@
+/*
+ * Licensed to the Apache Software
philipnee commented on code in PR #14386:
URL: https://github.com/apache/kafka/pull/14386#discussion_r1325341358
##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManager.java:
##
@@ -0,0 +1,210 @@
+/*
+ * Licensed to the Apache Software
[
https://issues.apache.org/jira/browse/KAFKA-15245?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Kamal Chandraprakash updated KAFKA-15245:
-
Fix Version/s: 3.6.0
(was: 3.7.0)
> Improve Tiered
[
https://issues.apache.org/jira/browse/KAFKA-15245?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Kamal Chandraprakash updated KAFKA-15245:
-
Description: Rename existing tiered storage metrics to remove ambiguity
(was:
[
https://issues.apache.org/jira/browse/KAFKA-15245?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17764959#comment-17764959
]
Kamal Chandraprakash commented on KAFKA-15245:
--
KAFKA-15094 ticket is already filed to
[
https://issues.apache.org/jira/browse/KAFKA-15453?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Kamal Chandraprakash updated KAFKA-15453:
-
Fix Version/s: 3.6.0
(was: 3.7.0)
> Enable
[
https://issues.apache.org/jira/browse/KAFKA-15160?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17764956#comment-17764956
]
Phuc Hong Tran edited comment on KAFKA-15160 at 9/14/23 3:32 AM:
-
[
https://issues.apache.org/jira/browse/KAFKA-15160?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17764956#comment-17764956
]
Phuc Hong Tran commented on KAFKA-15160:
[~vikashmishra0808] How did you setup ConsumerRecord
showuon commented on PR #14381:
URL: https://github.com/apache/kafka/pull/14381#issuecomment-1718627959
@satishd , not sure if you want to have another look. If no, I'll merge it.
Thanks.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log
showuon commented on PR #14381:
URL: https://github.com/apache/kafka/pull/14381#issuecomment-1718627359
Failed tests are unrelated:
```
Build / JDK 8 and Scala 2.12 /
kafka.network.SocketServerTest.testSaslReauthenticationFailureNoKip152SaslAuthenticate()
Build / JDK 8 and
showuon commented on PR #14347:
URL: https://github.com/apache/kafka/pull/14347#issuecomment-1718626560
Backported to 3.6 branch.
--
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
showuon merged PR #14347:
URL: https://github.com/apache/kafka/pull/14347
--
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:
showuon commented on PR #14347:
URL: https://github.com/apache/kafka/pull/14347#issuecomment-1718624599
Failed tests are unrelated:
```
Build / JDK 20 and Scala 2.13 /
[
https://issues.apache.org/jira/browse/KAFKA-15466?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Colin McCabe updated KAFKA-15466:
-
Description: Add KIP-919 support for kafka-features.sh, metadata-quorum.sh,
and
[
https://issues.apache.org/jira/browse/KAFKA-15466?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Colin McCabe updated KAFKA-15466:
-
Summary: Add KIP-919 support for some admin APIs (was: Add KIP-919 support
to
Colin McCabe created KAFKA-15466:
Summary: Add KIP-919 support to kafka-features.sh,
kafka-metadata-quorum.sh, kafka-cluster.sh
Key: KAFKA-15466
URL: https://issues.apache.org/jira/browse/KAFKA-15466
philipnee commented on code in PR #14386:
URL: https://github.com/apache/kafka/pull/14386#discussion_r1325175686
##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java:
##
@@ -381,41 +377,41 @@ public String toString() {
}
/**
kirktrue commented on code in PR #14359:
URL: https://github.com/apache/kafka/pull/14359#discussion_r1325174944
##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/NodeStatusDetector.java:
##
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation
kirktrue commented on code in PR #14359:
URL: https://github.com/apache/kafka/pull/14359#discussion_r1325174579
##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java:
##
@@ -91,23 +105,70 @@ public void onFailure(RuntimeException e) {
kirktrue commented on code in PR #14359:
URL: https://github.com/apache/kafka/pull/14359#discussion_r1325171010
##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java:
##
@@ -63,6 +68,31 @@ public NetworkClientDelegate(
junrao commented on code in PR #14386:
URL: https://github.com/apache/kafka/pull/14386#discussion_r1325040397
##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java:
##
@@ -91,7 +89,7 @@ public CommitRequestManager(
}
/**
-
philipnee commented on code in PR #14364:
URL: https://github.com/apache/kafka/pull/14364#discussion_r1325138384
##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/AssignorSelection.java:
##
@@ -79,6 +79,9 @@ public int hashCode() {
@Override
satishd commented on code in PR #14382:
URL: https://github.com/apache/kafka/pull/14382#discussion_r1324263153
##
docs/ops.html:
##
@@ -3859,6 +3859,80 @@ Finalizing the migration
# Other configs ...
+
+6.11 Tiered Storage
+
+Tiered Storage
overview
+
+Kafka data is
satishd commented on code in PR #14382:
URL: https://github.com/apache/kafka/pull/14382#discussion_r1324263153
##
docs/ops.html:
##
@@ -3859,6 +3859,80 @@ Finalizing the migration
# Other configs ...
+
+6.11 Tiered Storage
+
+Tiered Storage
overview
+
+Kafka data is
[
https://issues.apache.org/jira/browse/KAFKA-15459?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Justine Olshan resolved KAFKA-15459.
Resolution: Fixed
> Convert coordinator retriable errors to a known producer response
jolshan merged PR #14378:
URL: https://github.com/apache/kafka/pull/14378
--
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:
jolshan commented on PR #14378:
URL: https://github.com/apache/kafka/pull/14378#issuecomment-1718329367
Test failures look unrelated. I synced with Jason offline and he said we can
proceed with the merge.
--
This is an automated message from the Apache Git Service.
To respond to the
jeffkbkim opened a new pull request, #14387:
URL: https://github.com/apache/kafka/pull/14387
*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
kirktrue commented on code in PR #14359:
URL: https://github.com/apache/kafka/pull/14359#discussion_r1325061757
##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractFetch.java:
##
@@ -256,138 +258,23 @@ protected FetchRequest.Builder
kirktrue commented on code in PR #14359:
URL: https://github.com/apache/kafka/pull/14359#discussion_r1325058583
##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchUtils.java:
##
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF)
kirktrue commented on code in PR #14359:
URL: https://github.com/apache/kafka/pull/14359#discussion_r1325056350
##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchBuffer.java:
##
@@ -0,0 +1,143 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF)
[
https://issues.apache.org/jira/browse/KAFKA-15465?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17764867#comment-17764867
]
Ahmed HIBOT commented on KAFKA-15465:
-
[~omnia_h_ibrahim], Can you please take a look and confirm my
[
https://issues.apache.org/jira/browse/KAFKA-15465?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Ahmed HIBOT updated KAFKA-15465:
Description:
h1. Replication steps
* Setup a source kafka cluster (alias SOURCE) which doesn't
kirktrue commented on code in PR #14359:
URL: https://github.com/apache/kafka/pull/14359#discussion_r1325038436
##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchBuffer.java:
##
@@ -0,0 +1,143 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF)
Ahmed HIBOT created KAFKA-15465:
---
Summary: MM2 not working when its internal topics are pre-created
on a cluster that disallows topic creation
Key: KAFKA-15465
URL: https://issues.apache.org/jira/browse/KAFKA-15465
mumrah commented on code in PR #14376:
URL: https://github.com/apache/kafka/pull/14376#discussion_r1324939918
##
metadata/src/main/java/org/apache/kafka/metadata/ListenerInfo.java:
##
@@ -0,0 +1,376 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+
philipnee opened a new pull request, #14386:
URL: https://github.com/apache/kafka/pull/14386
TopicMetadataRequestManager is responsible for sending topic metadata
requests. The manager manages API requests and build the request accordingly.
All topic metadata requests are chained, if
philipnee closed pull request #14362: KAFKA-14960: TopicMetadata request manager
URL: https://github.com/apache/kafka/pull/14362
--
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.
lianetm opened a new pull request, #14385:
URL: https://github.com/apache/kafka/pull/14385
Support for using committed offsets to update fetch positions.
This PR includes:
- moving`refreshCommittedOffsets` function out of the existing
`ConsumerCoordinator` so it can be reused (no
[
https://issues.apache.org/jira/browse/KAFKA-15464?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Jakub Scholz reassigned KAFKA-15464:
Assignee: Jakub Scholz
> Allow dynamic reloading of certificates with different DN /
Jakub Scholz created KAFKA-15464:
Summary: Allow dynamic reloading of certificates with different DN
/ SANs
Key: KAFKA-15464
URL: https://issues.apache.org/jira/browse/KAFKA-15464
Project: Kafka
[
https://issues.apache.org/jira/browse/KAFKA-15452?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17764839#comment-17764839
]
Raghu Baddam commented on KAFKA-15452:
--
[~hachikuji] /[~mimaison] /[~mjsax]: Do you mind validating
zhaohaidao commented on PR #14271:
URL: https://github.com/apache/kafka/pull/14271#issuecomment-1718110509
> @zhaohaidao Thanks for all the work you are doing on this. I really
appreciate it. I have created a few jiras for the next steps:
>
> *
splett2 commented on code in PR #14053:
URL: https://github.com/apache/kafka/pull/14053#discussion_r1324902142
##
core/src/main/scala/kafka/cluster/Partition.scala:
##
@@ -864,13 +866,19 @@ class Partition(val topicPartition: TopicPartition,
// No need to calculate low
zhaohaidao commented on PR #14271:
URL: https://github.com/apache/kafka/pull/14271#issuecomment-1718105046
> @zhaohaidao Thanks for the update. I made another pass on it and I left a
bunch of small comments. We should be able to merge it when they are addressed.
Could you also rebase the
zhaohaidao commented on code in PR #14271:
URL: https://github.com/apache/kafka/pull/14271#discussion_r1324884926
##
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##
@@ -431,14 +434,17 @@ public void rollback() {
zhaohaidao commented on code in PR #14271:
URL: https://github.com/apache/kafka/pull/14271#discussion_r1324884926
##
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##
@@ -431,14 +434,17 @@ public void rollback() {
jolshan commented on code in PR #14378:
URL: https://github.com/apache/kafka/pull/14378#discussion_r1324872270
##
core/src/main/scala/kafka/server/ReplicaManager.scala:
##
@@ -753,39 +753,63 @@ class ReplicaManager(val config: KafkaConfig,
val localProduceResults =
dajac commented on code in PR #14378:
URL: https://github.com/apache/kafka/pull/14378#discussion_r1324871934
##
core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala:
##
@@ -2569,6 +2571,70 @@ class ReplicaManagerTest {
assertEquals((Errors.NONE, node0),
zhaohaidao commented on code in PR #14271:
URL: https://github.com/apache/kafka/pull/14271#discussion_r1324870944
##
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java:
##
@@ -599,6 +608,164 @@ public void
jolshan commented on code in PR #14378:
URL: https://github.com/apache/kafka/pull/14378#discussion_r1324870442
##
core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala:
##
@@ -2569,6 +2571,70 @@ class ReplicaManagerTest {
assertEquals((Errors.NONE, node0),
dajac commented on code in PR #14378:
URL: https://github.com/apache/kafka/pull/14378#discussion_r1324868208
##
core/src/main/scala/kafka/server/ReplicaManager.scala:
##
@@ -753,39 +753,63 @@ class ReplicaManager(val config: KafkaConfig,
val localProduceResults =
dajac commented on code in PR #14378:
URL: https://github.com/apache/kafka/pull/14378#discussion_r1324869043
##
core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala:
##
@@ -2569,6 +2571,70 @@ class ReplicaManagerTest {
assertEquals((Errors.NONE, node0),
dajac commented on code in PR #14378:
URL: https://github.com/apache/kafka/pull/14378#discussion_r1324868507
##
core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala:
##
@@ -2569,6 +2571,70 @@ class ReplicaManagerTest {
assertEquals((Errors.NONE, node0),
dajac commented on code in PR #14378:
URL: https://github.com/apache/kafka/pull/14378#discussion_r1324868208
##
core/src/main/scala/kafka/server/ReplicaManager.scala:
##
@@ -753,39 +753,63 @@ class ReplicaManager(val config: KafkaConfig,
val localProduceResults =
zhaohaidao commented on code in PR #14271:
URL: https://github.com/apache/kafka/pull/14271#discussion_r1324869392
##
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##
@@ -8574,6 +8584,93 @@ public void
dajac commented on code in PR #14378:
URL: https://github.com/apache/kafka/pull/14378#discussion_r1324869297
##
core/src/main/scala/kafka/server/ReplicaManager.scala:
##
@@ -753,39 +753,56 @@ class ReplicaManager(val config: KafkaConfig,
val localProduceResults =
zhaohaidao commented on code in PR #14271:
URL: https://github.com/apache/kafka/pull/14271#discussion_r1324869129
##
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##
@@ -8574,6 +8584,93 @@ public void
splett2 commented on code in PR #14053:
URL: https://github.com/apache/kafka/pull/14053#discussion_r1324845475
##
core/src/main/scala/kafka/cluster/Partition.scala:
##
@@ -137,7 +137,8 @@ object Partition {
delayedOperations = delayedOperations,
metadataCache =
[
https://issues.apache.org/jira/browse/KAFKA-15463?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Matthias J. Sax updated KAFKA-15463:
Priority: Major (was: Blocker)
> StreamsException: Accessing from an unknown node
>
mjsax commented on code in PR #14322:
URL: https://github.com/apache/kafka/pull/14322#discussion_r1324811185
##
docs/design.html:
##
@@ -136,8 +136,10 @@
-Kafka supports this with an efficient batching format. A batch of messages
can be clumped together compressed and
davetroiano commented on code in PR #14322:
URL: https://github.com/apache/kafka/pull/14322#discussion_r1324799715
##
docs/design.html:
##
@@ -136,8 +136,10 @@
-Kafka supports this with an efficient batching format. A batch of messages
can be clumped together compressed
jolshan commented on code in PR #14378:
URL: https://github.com/apache/kafka/pull/14378#discussion_r1324792282
##
core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala:
##
@@ -2569,6 +2571,70 @@ class ReplicaManagerTest {
assertEquals((Errors.NONE, node0),
zhaohaidao commented on code in PR #14271:
URL: https://github.com/apache/kafka/pull/14271#discussion_r1324791489
##
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java:
##
@@ -599,6 +608,164 @@ public void
junrao commented on code in PR #14359:
URL: https://github.com/apache/kafka/pull/14359#discussion_r1323771865
##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java:
##
@@ -63,6 +68,31 @@ public NetworkClientDelegate(
OmniaGM commented on PR #13204:
URL: https://github.com/apache/kafka/pull/13204#issuecomment-1717905593
I found out why the tests were failing. It turns out that we are hitting a
similar problem to this gradle
[issue#847](https://github.com/gradle/gradle/issues/847) that causing
OmniaGM commented on PR #13201:
URL: https://github.com/apache/kafka/pull/13201#issuecomment-1717877198
> Hi @OmniaGM, thanks. I left some comments.
>
> Additionally, this error message is different from previous implementation:
>
> ```shell
> ### OLD
> $
jolshan commented on code in PR #14378:
URL: https://github.com/apache/kafka/pull/14378#discussion_r1324699341
##
core/src/main/scala/kafka/server/ReplicaManager.scala:
##
@@ -753,39 +753,63 @@ class ReplicaManager(val config: KafkaConfig,
val localProduceResults =
jolshan commented on code in PR #14378:
URL: https://github.com/apache/kafka/pull/14378#discussion_r1324697600
##
core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala:
##
@@ -2569,6 +2571,70 @@ class ReplicaManagerTest {
assertEquals((Errors.NONE, node0),
jolshan commented on code in PR #14378:
URL: https://github.com/apache/kafka/pull/14378#discussion_r1324696082
##
core/src/main/scala/kafka/server/ReplicaManager.scala:
##
@@ -753,39 +753,63 @@ class ReplicaManager(val config: KafkaConfig,
val localProduceResults =
jolshan commented on code in PR #14378:
URL: https://github.com/apache/kafka/pull/14378#discussion_r1324692053
##
core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala:
##
@@ -2569,6 +2571,70 @@ class ReplicaManagerTest {
assertEquals((Errors.NONE, node0),
[
https://issues.apache.org/jira/browse/KAFKA-15163?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Jun Rao resolved KAFKA-15163.
-
Fix Version/s: 3.7.0
Resolution: Fixed
This is covered in
AndrewJSchofield commented on code in PR #14384:
URL: https://github.com/apache/kafka/pull/14384#discussion_r1324590647
##
clients/src/main/java/org/apache/kafka/common/PartitionInfo.java:
##
@@ -60,12 +69,19 @@ public int partition() {
}
/**
- * The node id of
[
https://issues.apache.org/jira/browse/KAFKA-15115?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Jun Rao resolved KAFKA-15115.
-
Fix Version/s: 3.7.0
Resolution: Fixed
merged the PR to trunk
> Implement resetPositions
junrao merged PR #14346:
URL: https://github.com/apache/kafka/pull/14346
--
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:
tylerbertrand commented on code in PR #14344:
URL: https://github.com/apache/kafka/pull/14344#discussion_r1324625070
##
build.gradle:
##
@@ -685,6 +685,7 @@ subprojects {
}
checkstyle {
+configDirectory = rootProject.layout.projectDirectory.dir("checkstyle")
Review
tylerbertrand commented on code in PR #14344:
URL: https://github.com/apache/kafka/pull/14344#discussion_r1324618554
##
build.gradle:
##
@@ -685,6 +685,7 @@ subprojects {
}
checkstyle {
+configDirectory = rootProject.layout.projectDirectory.dir("checkstyle")
tylerbertrand commented on code in PR #14344:
URL: https://github.com/apache/kafka/pull/14344#discussion_r1324618554
##
build.gradle:
##
@@ -685,6 +685,7 @@ subprojects {
}
checkstyle {
+configDirectory = rootProject.layout.projectDirectory.dir("checkstyle")
satishd commented on PR #14382:
URL: https://github.com/apache/kafka/pull/14382#issuecomment-1717677707
>That sounds good. But since we don't have much time left for v3.6.0 RC
build, we can add them after release. Doc updates can directly update in
kafka-site repo. I'll add them in a
[
https://issues.apache.org/jira/browse/KAFKA-15160?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17764677#comment-17764677
]
Phuc Hong Tran commented on KAFKA-15160:
[~vikashmishra0808] Sorry for the delay, I'm
lianetm commented on PR #14346:
URL: https://github.com/apache/kafka/pull/14346#issuecomment-1717579385
There were 6 unrelated test failures in the last build:
Build / JDK 20 and Scala 2.13 / testTaskRequestWithOldStartMsGetsUpdated() –
cadonna commented on code in PR #14317:
URL: https://github.com/apache/kafka/pull/14317#discussion_r1324450860
##
streams/src/test/java/org/apache/kafka/streams/state/internals/metrics/RocksDBBlockCacheMetricsTest.java:
##
@@ -0,0 +1,133 @@
+/*
+ * Licensed to the Apache
showuon commented on PR #14382:
URL: https://github.com/apache/kafka/pull/14382#issuecomment-1717567763
> Will the RemoteLogManagerConfig and TBRLMMConfig will be auto-exported in
the doc config section?
Yes, that's been handled in another
dajac commented on code in PR #14378:
URL: https://github.com/apache/kafka/pull/14378#discussion_r1324437789
##
core/src/main/scala/kafka/server/ReplicaManager.scala:
##
@@ -753,39 +753,63 @@ class ReplicaManager(val config: KafkaConfig,
val localProduceResults =
showuon commented on code in PR #14382:
URL: https://github.com/apache/kafka/pull/14382#discussion_r1324438818
##
docs/ops.html:
##
@@ -3859,6 +3859,80 @@ Finalizing the migration
# Other configs ...
+
+6.11 Tiered Storage
+
+Tiered Storage
overview
+
+Kafka data is
1 - 100 of 156 matches
Mail list logo