kamalcph commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1295402786
##
core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java:
##
@@ -1003,6 +1015,134 @@ public RemoteLogMetadataManager
createRemoteLogMetadataManager() {
kamalcph commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1295394486
##
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##
@@ -618,6 +625,230 @@ public void run() {
}
}
+public void
[
https://issues.apache.org/jira/browse/KAFKA-15352?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Kamal Chandraprakash updated KAFKA-15352:
-
Parent: KAFKA-7739
Issue Type: Sub-task (was: Task)
> Ensure
Kamal Chandraprakash created KAFKA-15352:
Summary: Ensure consistency while deleting the remote log segments
Key: KAFKA-15352
URL: https://issues.apache.org/jira/browse/KAFKA-15352
Project:
[
https://issues.apache.org/jira/browse/KAFKA-15351?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Kamal Chandraprakash updated KAFKA-15351:
-
Description:
Case-1:
In the FETCH response, the leader-log-start-offset will
Kamal Chandraprakash created KAFKA-15351:
Summary: Update log-start-offset after leader election for topics
enabled with remote storage
Key: KAFKA-15351
URL:
[
https://issues.apache.org/jira/browse/KAFKA-15351?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Kamal Chandraprakash updated KAFKA-15351:
-
Parent: KAFKA-7739
Issue Type: Sub-task (was: Task)
> Update
satishd commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1295387916
##
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##
@@ -1033,6 +1360,35 @@ public void close() {
}
}
+private static class
[
https://issues.apache.org/jira/browse/KAFKA-15346?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Matthias J. Sax updated KAFKA-15346:
Component/s: streams
> Single-Key_single-timestamp IQs with versioned state stores
>
[
https://issues.apache.org/jira/browse/KAFKA-15347?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Matthias J. Sax updated KAFKA-15347:
Component/s: streams
> Single-Key_multi-timestamp IQs with versioned state stores
>
[
https://issues.apache.org/jira/browse/KAFKA-15348?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Matthias J. Sax updated KAFKA-15348:
Component/s: streams
> Range IQs with versioned state stores
>
pprovenzano commented on code in PR #14083:
URL: https://github.com/apache/kafka/pull/14083#discussion_r1295339433
##
core/src/main/scala/kafka/server/ControllerApis.scala:
##
@@ -842,6 +847,75 @@ class ControllerApis(val requestChannel: RequestChannel,
}
}
+ def
[
https://issues.apache.org/jira/browse/KAFKA-15329?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Luke Chen resolved KAFKA-15329.
---
Resolution: Fixed
> Make default `remote.log.metadata.manager.class.name` as topic based RLMM
>
showuon merged PR #14202:
URL: https://github.com/apache/kafka/pull/14202
--
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 #14202:
URL: https://github.com/apache/kafka/pull/14202#issuecomment-1679845654
Failed tests are unrelated:
```
Build / JDK 11 and Scala 2.13 /
kafka.server.DynamicBrokerReconfigurationTest.testThreadPoolResize()
Build / JDK 11 and Scala 2.13 /
pprovenzano commented on PR #14083:
URL: https://github.com/apache/kafka/pull/14083#issuecomment-1679832173
> I think this starts to look good. There are some parts I haven't reviewed
yet, I'll try to get them in the next few days.
Thank you for the review comments. I think I have
pprovenzano commented on code in PR #14083:
URL: https://github.com/apache/kafka/pull/14083#discussion_r1295289182
##
core/src/main/scala/kafka/server/DelegationTokenManager.scala:
##
@@ -186,57 +134,28 @@ class DelegationTokenManager(val config: KafkaConfig,
val
pprovenzano commented on code in PR #14083:
URL: https://github.com/apache/kafka/pull/14083#discussion_r1295288501
##
core/src/main/scala/kafka/server/metadata/DelegationTokenPublisher.scala:
##
@@ -0,0 +1,83 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under
pprovenzano commented on PR #14083:
URL: https://github.com/apache/kafka/pull/14083#issuecomment-1679827453
> @pprovenzano Thanks for the PR. left few comments
>
> can we also update delegation token docs if required (like any configs for
controller nodes etc..)
gharris1727 commented on code in PR #14220:
URL: https://github.com/apache/kafka/pull/14220#discussion_r1295265105
##
docs/upgrade.html:
##
@@ -43,6 +43,10 @@ Notable changes in 3
See
C0urante commented on code in PR #14220:
URL: https://github.com/apache/kafka/pull/14220#discussion_r1295254068
##
docs/upgrade.html:
##
@@ -43,6 +43,10 @@ Notable changes in 3
See
C0urante commented on PR #14220:
URL: https://github.com/apache/kafka/pull/14220#issuecomment-1679781965
@gharris1727 @mimaison as the other two committers who voted for KIP-949,
would you mind taking a look? Thanks!
--
This is an automated message from the Apache Git Service.
To respond
C0urante opened a new pull request, #14220:
URL: https://github.com/apache/kafka/pull/14220
[Jira](https://issues.apache.org/jira/browse/KAFKA-15102)
This is a docs-only follow-up for https://github.com/apache/kafka/pull/14082
that calls out the necessity for this property in our
C0urante merged PR #14082:
URL: https://github.com/apache/kafka/pull/14082
--
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:
mumrah commented on code in PR #14208:
URL: https://github.com/apache/kafka/pull/14208#discussion_r1295237042
##
metadata/src/main/java/org/apache/kafka/image/loader/LogDeltaManifest.java:
##
@@ -66,6 +119,10 @@ public LogDeltaManifest(
this.numBytes = numBytes;
}
pprovenzano commented on code in PR #14083:
URL: https://github.com/apache/kafka/pull/14083#discussion_r1295230735
##
metadata/src/test/java/org/apache/kafka/image/DelegationTokenImageTest.java:
##
Review Comment:
Yes and Done
--
This is an automated message from the
pprovenzano commented on code in PR #14083:
URL: https://github.com/apache/kafka/pull/14083#discussion_r1295230635
##
metadata/src/test/java/org/apache/kafka/image/MetadataImageTest.java:
##
Review Comment:
Fixed
--
This is an automated message from the Apache Git
[
https://issues.apache.org/jira/browse/KAFKA-15350?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Greg Harris resolved KAFKA-15350.
-
Resolution: Invalid
I believe this was an environmental problem, due to some stale artifacts. A
[
https://issues.apache.org/jira/browse/KAFKA-15350?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Greg Harris updated KAFKA-15350:
Issue Type: Bug (was: Improvement)
> MetadataLoaderMetrics has ClassNotFoundException in system
[
https://issues.apache.org/jira/browse/KAFKA-15336?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Greg Harris updated KAFKA-15336:
Issue Type: Improvement (was: Bug)
> Connect plugin Javadocs should mention serviceloader
[
https://issues.apache.org/jira/browse/KAFKA-15336?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Greg Harris updated KAFKA-15336:
Issue Type: Bug (was: Improvement)
> Connect plugin Javadocs should mention serviceloader
Greg Harris created KAFKA-15350:
---
Summary: MetadataLoaderMetrics has ClassNotFoundException in
system tests
Key: KAFKA-15350
URL: https://issues.apache.org/jira/browse/KAFKA-15350
Project: Kafka
rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1295198962
##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##
@@ -0,0 +1,399 @@
+/*
+ * Licensed to the
rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1295197016
##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##
@@ -0,0 +1,399 @@
+/*
+ * Licensed to the
lucasbru commented on PR #14216:
URL: https://github.com/apache/kafka/pull/14216#issuecomment-1679690414
https://jenkins.confluent.io/job/confluentinc/job/kafka-streams-benchmarks/job/master/653/parameters/
junrao merged PR #14179:
URL: https://github.com/apache/kafka/pull/14179
--
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:
Greg Harris created KAFKA-15349:
---
Summary: ducker-ak should fail fast when gradlew systemTestLibs
fails
Key: KAFKA-15349
URL: https://issues.apache.org/jira/browse/KAFKA-15349
Project: Kafka
pprovenzano commented on code in PR #14083:
URL: https://github.com/apache/kafka/pull/14083#discussion_r1295140057
##
metadata/src/test/java/org/apache/kafka/image/MetadataImageTest.java:
##
Review Comment:
It is because the compare is using the toString() which redacts
gharris1727 merged PR #14177:
URL: https://github.com/apache/kafka/pull/14177
--
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:
cmccabe commented on PR #14213:
URL: https://github.com/apache/kafka/pull/14213#issuecomment-1679620704
> The RecordsBatchReader implementation is also changed to include control
records. This makes it possible for the state machine learn about committed
control records. This additional
gharris1727 commented on PR #14177:
URL: https://github.com/apache/kafka/pull/14177#issuecomment-1679635472
CI failures appear unrelated, and the tests pass locally. The test being
fixed shows no failures.
--
This is an automated message from the Apache Git Service.
To respond to the
[
https://issues.apache.org/jira/browse/KAFKA-15343?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17754771#comment-17754771
]
Greg Harris commented on KAFKA-15343:
-
Hi [~prasanth] and thank you for reporting this issue! It is
jeqo opened a new pull request, #14219:
URL: https://github.com/apache/kafka/pull/14219
*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 testing
cmccabe commented on code in PR #14208:
URL: https://github.com/apache/kafka/pull/14208#discussion_r1295077023
##
metadata/src/main/java/org/apache/kafka/image/loader/LogDeltaManifest.java:
##
@@ -80,6 +137,7 @@ public LeaderAndEpoch leaderAndEpoch() {
return
cmccabe commented on PR #14208:
URL: https://github.com/apache/kafka/pull/14208#issuecomment-1679586281
Thanks for the PR, @mumrah . I think at some point we'll need to add
begin/end transactions to the set of things we "fuzz". Basically have a test
that just makes sure that the
jeqo closed pull request #14219: KIP-405: 2023-08-15
URL: https://github.com/apache/kafka/pull/14219
--
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:
cmccabe commented on code in PR #14208:
URL: https://github.com/apache/kafka/pull/14208#discussion_r1295076740
##
metadata/src/main/java/org/apache/kafka/image/loader/LogDeltaManifest.java:
##
@@ -27,6 +27,59 @@
* Contains information about a set of changes that were loaded
cmccabe commented on code in PR #14208:
URL: https://github.com/apache/kafka/pull/14208#discussion_r1295091800
##
metadata/src/main/java/org/apache/kafka/image/loader/MetadataBatchLoader.java:
##
@@ -0,0 +1,257 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under
lianetm commented on code in PR #14218:
URL: https://github.com/apache/kafka/pull/14218#discussion_r1295078074
##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerUtils.java:
##
@@ -134,44 +132,20 @@ public static FetchMetricsManager
cmccabe commented on code in PR #14208:
URL: https://github.com/apache/kafka/pull/14208#discussion_r1295084331
##
metadata/src/main/java/org/apache/kafka/image/loader/MetadataBatchLoader.java:
##
@@ -0,0 +1,257 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under
gharris1727 commented on PR #14194:
URL: https://github.com/apache/kafka/pull/14194#issuecomment-1679551865
This is a documentation-only change, and test failures appear unrelated. A
local `javadoc` build doesn't include any errors for these classes.
--
This is an automated message from
cmccabe commented on code in PR #14208:
URL: https://github.com/apache/kafka/pull/14208#discussion_r1295090413
##
metadata/src/main/java/org/apache/kafka/image/loader/MetadataLoader.java:
##
@@ -183,6 +183,8 @@ public MetadataLoader build() {
*/
private MetadataImage
cmccabe commented on code in PR #14208:
URL: https://github.com/apache/kafka/pull/14208#discussion_r1295084521
##
metadata/src/main/java/org/apache/kafka/image/loader/MetadataBatchLoader.java:
##
@@ -0,0 +1,257 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under
cmccabe commented on code in PR #14208:
URL: https://github.com/apache/kafka/pull/14208#discussion_r1295077023
##
metadata/src/main/java/org/apache/kafka/image/loader/LogDeltaManifest.java:
##
@@ -80,6 +137,7 @@ public LeaderAndEpoch leaderAndEpoch() {
return
cmccabe commented on code in PR #14208:
URL: https://github.com/apache/kafka/pull/14208#discussion_r1295081634
##
metadata/src/main/java/org/apache/kafka/image/loader/MetadataBatchLoader.java:
##
@@ -0,0 +1,257 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under
cmccabe commented on code in PR #14208:
URL: https://github.com/apache/kafka/pull/14208#discussion_r1295077448
##
metadata/src/main/java/org/apache/kafka/image/loader/LogDeltaManifest.java:
##
@@ -66,6 +119,10 @@ public LogDeltaManifest(
this.numBytes = numBytes;
cmccabe commented on code in PR #14208:
URL: https://github.com/apache/kafka/pull/14208#discussion_r1295074841
##
metadata/src/main/java/org/apache/kafka/controller/QuorumController.java:
##
@@ -1200,6 +1235,16 @@ public static List
generateActivationRecords(
cmccabe commented on code in PR #14208:
URL: https://github.com/apache/kafka/pull/14208#discussion_r1295072484
##
metadata/src/main/java/org/apache/kafka/controller/QuorumController.java:
##
@@ -892,48 +909,62 @@ class MigrationWriteOperation implements
gharris1727 merged PR #14194:
URL: https://github.com/apache/kafka/pull/14194
--
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:
cmccabe commented on code in PR #14208:
URL: https://github.com/apache/kafka/pull/14208#discussion_r1295073210
##
metadata/src/main/java/org/apache/kafka/controller/QuorumController.java:
##
@@ -954,11 +985,14 @@ public void
handleCommit(BatchReader reader) {
cmccabe commented on code in PR #14208:
URL: https://github.com/apache/kafka/pull/14208#discussion_r1295069992
##
metadata/src/main/java/org/apache/kafka/controller/QuorumController.java:
##
@@ -623,7 +624,14 @@ enum ControllerOperationFlag {
* even though the cluster
cmccabe commented on code in PR #14208:
URL: https://github.com/apache/kafka/pull/14208#discussion_r1295071752
##
metadata/src/main/java/org/apache/kafka/controller/QuorumController.java:
##
@@ -892,48 +909,62 @@ class MigrationWriteOperation implements
lianetm commented on code in PR #14218:
URL: https://github.com/apache/kafka/pull/14218#discussion_r1295049009
##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerUtils.java:
##
@@ -134,44 +132,20 @@ public static FetchMetricsManager
lianetm commented on code in PR #14218:
URL: https://github.com/apache/kafka/pull/14218#discussion_r1295036974
##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerUtils.java:
##
@@ -134,44 +132,20 @@ public static FetchMetricsManager
AndrewJSchofield commented on code in PR #14111:
URL: https://github.com/apache/kafka/pull/14111#discussion_r1295024473
##
clients/src/main/java/org/apache/kafka/clients/Metadata.java:
##
@@ -114,18 +125,39 @@ public synchronized Cluster fetch() {
/**
* Return the
cmccabe merged PR #14215:
URL: https://github.com/apache/kafka/pull/14215
--
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:
cmccabe closed pull request #14199: MINOR Fix the ZkMigrationState metric in
KafkaController
URL: https://github.com/apache/kafka/pull/14199
--
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
AndrewJSchofield commented on code in PR #14111:
URL: https://github.com/apache/kafka/pull/14111#discussion_r1295030179
##
clients/src/main/java/org/apache/kafka/clients/Metadata.java:
##
@@ -140,17 +172,34 @@ public long metadataExpireMs() {
}
/**
- * Request
cmccabe commented on PR #14199:
URL: https://github.com/apache/kafka/pull/14199#issuecomment-1679483573
LGTM, committed
--
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
ruslankrivoshein commented on PR #13562:
URL: https://github.com/apache/kafka/pull/13562#issuecomment-1679474019
@mimaison 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
divijvaidya commented on code in PR #14161:
URL: https://github.com/apache/kafka/pull/14161#discussion_r1294991302
##
core/src/test/scala/unit/kafka/server/KafkaServerTest.scala:
##
@@ -154,6 +155,96 @@ class KafkaServerTest extends QuorumTestHarness {
server.shutdown()
divijvaidya commented on code in PR #14161:
URL: https://github.com/apache/kafka/pull/14161#discussion_r1294983667
##
core/src/test/scala/unit/kafka/server/KafkaServerTest.scala:
##
@@ -154,6 +155,96 @@ class KafkaServerTest extends QuorumTestHarness {
server.shutdown()
divijvaidya commented on code in PR #14161:
URL: https://github.com/apache/kafka/pull/14161#discussion_r1294994048
##
core/src/main/scala/kafka/server/ConfigHandler.scala:
##
@@ -62,6 +62,12 @@ class TopicConfigHandler(private val logManager: LogManager,
kafkaConfig: KafkaC
philipnee commented on code in PR #14218:
URL: https://github.com/apache/kafka/pull/14218#discussion_r1294990279
##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerUtils.java:
##
@@ -134,44 +132,20 @@ public static FetchMetricsManager
divijvaidya commented on PR #14176:
URL: https://github.com/apache/kafka/pull/14176#issuecomment-1679415679
Unrelated test failures:
```
[Build / JDK 17 and Scala 2.13 /
divijvaidya merged PR #14176:
URL: https://github.com/apache/kafka/pull/14176
--
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:
divijvaidya commented on PR #14212:
URL: https://github.com/apache/kafka/pull/14212#issuecomment-1679398284
Thank you for adding this.
Please add what you wrote in the description as a comment in the test so
that reader and quickly understand what the test is supposed to check.
divijvaidya commented on code in PR #14212:
URL: https://github.com/apache/kafka/pull/14212#discussion_r1294964226
##
core/src/test/scala/unit/kafka/server/ReplicaFetcherTierStateMachineTest.scala:
##
@@ -85,6 +85,64 @@ class ReplicaFetcherTierStateMachineTest {
lianetm opened a new pull request, #14218:
URL: https://github.com/apache/kafka/pull/14218
This is a continuation of the previous
[PR#13990](https://github.com/apache/kafka/pull/13990)
This PR main refactoring relates to :
- serializers/deserializers used in clients - unified in a
nizhikov opened a new pull request, #14217:
URL: https://github.com/apache/kafka/pull/14217
This PR is part of #13247
It includes `ReassignPartitionsCommandArgsTest` rewritten in java.
### Committer Checklist (excluded from commit message)
- [ ] Verify design and implementation
nizhikov commented on PR #13247:
URL: https://github.com/apache/kafka/pull/13247#issuecomment-1679370921
@mimaison I create #14217 which include `ReassignPartitionsCommandArgsTest`
rewritten in java. It independent from the big PR and can be reviewed
separately. Can you, please, take a
pprovenzano commented on code in PR #14083:
URL: https://github.com/apache/kafka/pull/14083#discussion_r1294925543
##
metadata/src/main/resources/common/metadata/DelegationTokenRecord.json:
##
@@ -22,8 +22,10 @@
"fields": [
{ "name": "Owner", "type": "string",
pprovenzano commented on code in PR #14083:
URL: https://github.com/apache/kafka/pull/14083#discussion_r1294918864
##
server-common/src/main/java/org/apache/kafka/server/common/MetadataVersion.java:
##
@@ -267,6 +270,10 @@ public boolean isLeaderEpochBumpRequiredOnIsrShrink() {
[
https://issues.apache.org/jira/browse/KAFKA-14133?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Yash Mayya updated KAFKA-14133:
---
Description:
{color:#de350b}There are tests which use both PowerMock and EasyMock. I have
put
[
https://issues.apache.org/jira/browse/KAFKA-14133?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Yash Mayya updated KAFKA-14133:
---
Description:
{color:#de350b}There are tests which use both PowerMock and EasyMock. I have
put
[
https://issues.apache.org/jira/browse/KAFKA-15344?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
David Mao updated KAFKA-15344:
--
Description:
We noticed an application received an OFFSET_OUT_OF_RANGE error following a
network
AndrewJSchofield commented on PR #14111:
URL: https://github.com/apache/kafka/pull/14111#issuecomment-1679338770
@junrao I don't know what's causing so many test failures. 78 of them were
due to "unexpected threads" in the broker. I'll take another look when there's
another build. I expect
junrao commented on code in PR #14111:
URL: https://github.com/apache/kafka/pull/14111#discussion_r1294877936
##
clients/src/main/java/org/apache/kafka/clients/Metadata.java:
##
@@ -267,11 +307,14 @@ public synchronized void update(int requestVersion,
MetadataResponse
junrao commented on code in PR #14111:
URL: https://github.com/apache/kafka/pull/14111#discussion_r1294883198
##
clients/src/main/java/org/apache/kafka/clients/Metadata.java:
##
@@ -140,17 +172,34 @@ public long metadataExpireMs() {
}
/**
- * Request an update
junrao commented on code in PR #14111:
URL: https://github.com/apache/kafka/pull/14111#discussion_r1294873505
##
clients/src/main/java/org/apache/kafka/clients/Metadata.java:
##
@@ -114,18 +125,39 @@ public synchronized Cluster fetch() {
/**
* Return the next time
[
https://issues.apache.org/jira/browse/KAFKA-14133?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Yash Mayya updated KAFKA-14133:
---
Description:
{color:#de350b}There are tests which use both PowerMock and EasyMock. I have
put
[
https://issues.apache.org/jira/browse/KAFKA-15344?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Matthias J. Sax updated KAFKA-15344:
Component/s: streams
> Kafka Streams should include the message leader epoch when
divijvaidya commented on code in PR #14176:
URL: https://github.com/apache/kafka/pull/14176#discussion_r1294874991
##
core/src/test/scala/integration/kafka/admin/RemoteTopicCrudTest.scala:
##
@@ -0,0 +1,331 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
[
https://issues.apache.org/jira/browse/KAFKA-15257?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Alieh Saeedi updated KAFKA-15257:
-
Description:
Query types to consider include:
* single-key latest-value lookup
* single-key
[
https://issues.apache.org/jira/browse/KAFKA-15347?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Alieh Saeedi reassigned KAFKA-15347:
Assignee: Alieh Saeedi
> Single-Key_multi-timestamp IQs with versioned state stores
>
Alieh Saeedi created KAFKA-15348:
Summary: Range IQs with versioned state stores
Key: KAFKA-15348
URL: https://issues.apache.org/jira/browse/KAFKA-15348
Project: Kafka
Issue Type: Sub-task
[
https://issues.apache.org/jira/browse/KAFKA-15348?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Alieh Saeedi reassigned KAFKA-15348:
Assignee: Alieh Saeedi
> Range IQs with versioned state stores
>
Alieh Saeedi created KAFKA-15347:
Summary: Single-Key_multi-timestamp IQs with versioned state stores
Key: KAFKA-15347
URL: https://issues.apache.org/jira/browse/KAFKA-15347
Project: Kafka
[
https://issues.apache.org/jira/browse/KAFKA-15346?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Alieh Saeedi updated KAFKA-15346:
-
Summary: Single-Key_single-timestamp IQs with versioned state stores (was:
[
https://issues.apache.org/jira/browse/KAFKA-15346?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Alieh Saeedi updated KAFKA-15346:
-
Description:
1 - 100 of 184 matches
Mail list logo