Re: [DISCUSS] KIP-1018: Introduce max remote fetch timeout config

2024-04-27 Thread Kamal Chandraprakash
Hi Federico,

Thanks for the suggestion! Updated the config name to "
remote.fetch.max.wait.ms".

Christo,

Good point. We don't have the remote-read latency metrics to measure the
performance of the remote read requests. I'll update the KIP to emit this
metric.

--
Kamal


On Sat, Apr 27, 2024 at 4:03 PM Federico Valeri 
wrote:

> Hi Kamal, it looks like all TS configurations starts with "remote."
> prefix, so I was wondering if we should name it
> "remote.fetch.max.wait.ms".
>
> On Fri, Apr 26, 2024 at 7:07 PM Kamal Chandraprakash
>  wrote:
> >
> > Hi all,
> >
> > If there are no more comments, I'll start a vote thread by tomorrow.
> > Please review the KIP.
> >
> > Thanks,
> > Kamal
> >
> > On Sat, Mar 30, 2024 at 11:08 PM Kamal Chandraprakash <
> > kamal.chandraprak...@gmail.com> wrote:
> >
> > > Hi all,
> > >
> > > Bumping the thread. Please review this KIP. Thanks!
> > >
> > > On Thu, Feb 1, 2024 at 9:11 PM Kamal Chandraprakash <
> > > kamal.chandraprak...@gmail.com> wrote:
> > >
> > >> Hi Jorge,
> > >>
> > >> Thanks for the review! Added your suggestions to the KIP. PTAL.
> > >>
> > >> The `fetch.max.wait.ms` config will be also applicable for topics
> > >> enabled with remote storage.
> > >> Updated the description to:
> > >>
> > >> ```
> > >> The maximum amount of time the server will block before answering the
> > >> fetch request
> > >> when it is reading near to the tail of the partition (high-watermark)
> and
> > >> there isn't
> > >> sufficient data to immediately satisfy the requirement given by
> > >> fetch.min.bytes.
> > >> ```
> > >>
> > >> --
> > >> Kamal
> > >>
> > >> On Thu, Feb 1, 2024 at 12:12 AM Jorge Esteban Quilcate Otoya <
> > >> quilcate.jo...@gmail.com> wrote:
> > >>
> > >>> Hi Kamal,
> > >>>
> > >>> Thanks for this KIP! It should help to solve one of the main issues
> with
> > >>> tiered storage at the moment that is dealing with individual consumer
> > >>> configurations to avoid flooding logs with interrupted exceptions.
> > >>>
> > >>> One of the topics discussed in [1][2] was on the semantics of `
> > >>> fetch.max.wait.ms` and how it's affected by remote storage. Should
> we
> > >>> consider within this KIP the update of `fetch.max.wail.ms` docs to
> > >>> clarify
> > >>> it only applies to local storage?
> > >>>
> > >>> Otherwise, LGTM -- looking forward to see this KIP adopted.
> > >>>
> > >>> [1] https://issues.apache.org/jira/browse/KAFKA-15776
> > >>> [2]
> https://github.com/apache/kafka/pull/14778#issuecomment-1820588080
> > >>>
> > >>> On Tue, 30 Jan 2024 at 01:01, Kamal Chandraprakash <
> > >>> kamal.chandraprak...@gmail.com> wrote:
> > >>>
> > >>> > Hi all,
> > >>> >
> > >>> > I have opened a KIP-1018
> > >>> > <
> > >>> >
> > >>>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-1018%3A+Introduce+max+remote+fetch+timeout+config+for+DelayedRemoteFetch+requests
> > >>> > >
> > >>> > to introduce dynamic max-remote-fetch-timeout broker config to give
> > >>> more
> > >>> > control to the operator.
> > >>> >
> > >>> >
> > >>> >
> > >>>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-1018%3A+Introduce+max+remote+fetch+timeout+config+for+DelayedRemoteFetch+requests
> > >>> >
> > >>> > Let me know if you have any feedback or suggestions.
> > >>> >
> > >>> > --
> > >>> > Kamal
> > >>> >
> > >>>
> > >>
>


Re: [DISCUSS] KIP-1018: Introduce max remote fetch timeout config

2024-04-27 Thread Christo Lolov
Heya Kamal,

I quite like the proposal and would support it!

However, today I don't think we have a metric which shows the latency of
fetch requests which are served from remote, am I wrong?
I looked at both
https://github.com/clolov/kafka/blob/trunk/core/src/main/scala/kafka/network/RequestChannel.scala#L521-L527
and https://kafka.apache.org/documentation/#tiered_storage_monitoring.
If I am right, then I believe it would be very useful if this KIP also
introduces such a metric because the two are tightly coupled.

What do you think?

Best,
Christo

On Sat, 27 Apr 2024 at 11:33, Federico Valeri  wrote:

> Hi Kamal, it looks like all TS configurations starts with "remote."
> prefix, so I was wondering if we should name it
> "remote.fetch.max.wait.ms".
>
> On Fri, Apr 26, 2024 at 7:07 PM Kamal Chandraprakash
>  wrote:
> >
> > Hi all,
> >
> > If there are no more comments, I'll start a vote thread by tomorrow.
> > Please review the KIP.
> >
> > Thanks,
> > Kamal
> >
> > On Sat, Mar 30, 2024 at 11:08 PM Kamal Chandraprakash <
> > kamal.chandraprak...@gmail.com> wrote:
> >
> > > Hi all,
> > >
> > > Bumping the thread. Please review this KIP. Thanks!
> > >
> > > On Thu, Feb 1, 2024 at 9:11 PM Kamal Chandraprakash <
> > > kamal.chandraprak...@gmail.com> wrote:
> > >
> > >> Hi Jorge,
> > >>
> > >> Thanks for the review! Added your suggestions to the KIP. PTAL.
> > >>
> > >> The `fetch.max.wait.ms` config will be also applicable for topics
> > >> enabled with remote storage.
> > >> Updated the description to:
> > >>
> > >> ```
> > >> The maximum amount of time the server will block before answering the
> > >> fetch request
> > >> when it is reading near to the tail of the partition (high-watermark)
> and
> > >> there isn't
> > >> sufficient data to immediately satisfy the requirement given by
> > >> fetch.min.bytes.
> > >> ```
> > >>
> > >> --
> > >> Kamal
> > >>
> > >> On Thu, Feb 1, 2024 at 12:12 AM Jorge Esteban Quilcate Otoya <
> > >> quilcate.jo...@gmail.com> wrote:
> > >>
> > >>> Hi Kamal,
> > >>>
> > >>> Thanks for this KIP! It should help to solve one of the main issues
> with
> > >>> tiered storage at the moment that is dealing with individual consumer
> > >>> configurations to avoid flooding logs with interrupted exceptions.
> > >>>
> > >>> One of the topics discussed in [1][2] was on the semantics of `
> > >>> fetch.max.wait.ms` and how it's affected by remote storage. Should
> we
> > >>> consider within this KIP the update of `fetch.max.wail.ms` docs to
> > >>> clarify
> > >>> it only applies to local storage?
> > >>>
> > >>> Otherwise, LGTM -- looking forward to see this KIP adopted.
> > >>>
> > >>> [1] https://issues.apache.org/jira/browse/KAFKA-15776
> > >>> [2]
> https://github.com/apache/kafka/pull/14778#issuecomment-1820588080
> > >>>
> > >>> On Tue, 30 Jan 2024 at 01:01, Kamal Chandraprakash <
> > >>> kamal.chandraprak...@gmail.com> wrote:
> > >>>
> > >>> > Hi all,
> > >>> >
> > >>> > I have opened a KIP-1018
> > >>> > <
> > >>> >
> > >>>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-1018%3A+Introduce+max+remote+fetch+timeout+config+for+DelayedRemoteFetch+requests
> > >>> > >
> > >>> > to introduce dynamic max-remote-fetch-timeout broker config to give
> > >>> more
> > >>> > control to the operator.
> > >>> >
> > >>> >
> > >>> >
> > >>>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-1018%3A+Introduce+max+remote+fetch+timeout+config+for+DelayedRemoteFetch+requests
> > >>> >
> > >>> > Let me know if you have any feedback or suggestions.
> > >>> >
> > >>> > --
> > >>> > Kamal
> > >>> >
> > >>>
> > >>
>


Jenkins build is still unstable: Kafka » Kafka Branch Builder » trunk #2853

2024-04-27 Thread Apache Jenkins Server
See 




[jira] [Resolved] (KAFKA-16560) Refactor/cleanup BrokerNode/ControllerNode/ClusterConfig

2024-04-27 Thread Chia-Ping Tsai (Jira)


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

Chia-Ping Tsai resolved KAFKA-16560.

Fix Version/s: 3.8.0
   Resolution: Fixed

> Refactor/cleanup BrokerNode/ControllerNode/ClusterConfig
> 
>
> Key: KAFKA-16560
> URL: https://issues.apache.org/jira/browse/KAFKA-16560
> Project: Kafka
>  Issue Type: Improvement
>Reporter: Chia-Ping Tsai
>Assignee: Kuan Po Tseng
>Priority: Minor
> Fix For: 3.8.0
>
>
> origin discussion: 
> https://github.com/apache/kafka/pull/15715#discussion_r1564660916
> It seems to me this jira should address following tasks.
> 1. make them immutable. We have adopted the builder pattern, so all changes 
> should be completed in the builder phase
> 2. make all `Builder#build()` not accept any arguments. Instead, we should 
> add new setters for those arguments.



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


[jira] [Created] (KAFKA-16636) Flaky test - testStickyTaskAssignorLargePartitionCount – org.apache.kafka.streams.processor.internals.StreamsAssignmentScaleTest

2024-04-27 Thread Igor Soarez (Jira)
Igor Soarez created KAFKA-16636:
---

 Summary: Flaky test - testStickyTaskAssignorLargePartitionCount – 
org.apache.kafka.streams.processor.internals.StreamsAssignmentScaleTest
 Key: KAFKA-16636
 URL: https://issues.apache.org/jira/browse/KAFKA-16636
 Project: Kafka
  Issue Type: Test
Reporter: Igor Soarez
 Attachments: log (1).txt

testStickyTaskAssignorLargePartitionCount – 
org.apache.kafka.streams.processor.internals.StreamsAssignmentScaleTest
{code:java}
java.lang.AssertionError: The first assignment took too long to complete at 
131680ms.   at 
org.apache.kafka.streams.processor.internals.StreamsAssignmentScaleTest.completeLargeAssignment(StreamsAssignmentScaleTest.java:220)
 at 
org.apache.kafka.streams.processor.internals.StreamsAssignmentScaleTest.testStickyTaskAssignorLargePartitionCount(StreamsAssignmentScaleTest.java:102)
 {code}
[https://ci-builds.apache.org/blue/organizations/jenkins/Kafka%2Fkafka-pr/detail/PR-15816/1/tests/]

 

 



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


[jira] [Created] (KAFKA-16635) Flaky test "shouldThrottleOldSegments(String).quorum=kraft" – kafka.server.ReplicationQuotasTest

2024-04-27 Thread Igor Soarez (Jira)
Igor Soarez created KAFKA-16635:
---

 Summary: Flaky test 
"shouldThrottleOldSegments(String).quorum=kraft" – 
kafka.server.ReplicationQuotasTest
 Key: KAFKA-16635
 URL: https://issues.apache.org/jira/browse/KAFKA-16635
 Project: Kafka
  Issue Type: Test
Reporter: Igor Soarez


"shouldThrottleOldSegments(String).quorum=kraft" – 
kafka.server.ReplicationQuotasTest
{code:java}
org.opentest4j.AssertionFailedError: Throttled replication of 2203ms should be 
> 3600.0ms ==> expected:  but was:  at 
app//org.junit.jupiter.api.AssertionFailureBuilder.build(AssertionFailureBuilder.java:151)
   at 
app//org.junit.jupiter.api.AssertionFailureBuilder.buildAndThrow(AssertionFailureBuilder.java:132)
   at app//org.junit.jupiter.api.AssertTrue.failNotTrue(AssertTrue.java:63) 
   at app//org.junit.jupiter.api.AssertTrue.assertTrue(AssertTrue.java:36) at 
app//org.junit.jupiter.api.Assertions.assertTrue(Assertions.java:214)at 
app//kafka.server.ReplicationQuotasTest.shouldThrottleOldSegments(ReplicationQuotasTest.scala:260)
 {code}
https://ci-builds.apache.org/blue/organizations/jenkins/Kafka%2Fkafka-pr/detail/PR-15816/1/tests/



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


[jira] [Created] (KAFKA-16634) Flaky test - testFenceMultipleBrokers() – org.apache.kafka.controller.QuorumControllerTest

2024-04-27 Thread Igor Soarez (Jira)
Igor Soarez created KAFKA-16634:
---

 Summary: Flaky test - testFenceMultipleBrokers() – 
org.apache.kafka.controller.QuorumControllerTest
 Key: KAFKA-16634
 URL: https://issues.apache.org/jira/browse/KAFKA-16634
 Project: Kafka
  Issue Type: Test
Reporter: Igor Soarez
 Attachments: output.txt

testFenceMultipleBrokers() – org.apache.kafka.controller.QuorumControllerTest

Error:
{code:java}
java.util.concurrent.TimeoutException: testFenceMultipleBrokers() timed out 
after 40 seconds {code}
Test logs in attached output.txt

https://ci-builds.apache.org/blue/organizations/jenkins/Kafka%2Fkafka-pr/detail/PR-15816/1/tests/



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


[jira] [Created] (KAFKA-16633) Flaky test - testDescribeExistingGroupWithNoMembers(String, String).quorum=kraft+kip848.groupProtocol=consumer – org.apache.kafka.tools.consumer.group.DescribeConsumerGr

2024-04-27 Thread Igor Soarez (Jira)
Igor Soarez created KAFKA-16633:
---

 Summary: Flaky test - 
testDescribeExistingGroupWithNoMembers(String, 
String).quorum=kraft+kip848.groupProtocol=consumer – 
org.apache.kafka.tools.consumer.group.DescribeConsumerGroupTest
 Key: KAFKA-16633
 URL: https://issues.apache.org/jira/browse/KAFKA-16633
 Project: Kafka
  Issue Type: Test
Reporter: Igor Soarez


testDescribeExistingGroupWithNoMembers(String, 
String).quorum=kraft+kip848.groupProtocol=consumer – 
org.apache.kafka.tools.consumer.group.DescribeConsumerGroupTest
{code:java}
org.opentest4j.AssertionFailedError: Condition not met within timeout 15000. 
Expected no active member in describe group results with describe type 
--offsets ==> expected:  but was:  at 
app//org.junit.jupiter.api.AssertionFailureBuilder.build(AssertionFailureBuilder.java:151)
   at 
app//org.junit.jupiter.api.AssertionFailureBuilder.buildAndThrow(AssertionFailureBuilder.java:132)
   at app//org.junit.jupiter.api.AssertTrue.failNotTrue(AssertTrue.java:63) 
   at app//org.junit.jupiter.api.AssertTrue.assertTrue(AssertTrue.java:36) at 
app//org.junit.jupiter.api.Assertions.assertTrue(Assertions.java:214)at 
app//org.apache.kafka.test.TestUtils.lambda$waitForCondition$3(TestUtils.java:396)
   at 
app//org.apache.kafka.test.TestUtils.retryOnExceptionWithTimeout(TestUtils.java:444)
 at app//org.apache.kafka.test.TestUtils.waitForCondition(TestUtils.java:393)   
 at app//org.apache.kafka.test.TestUtils.waitForCondition(TestUtils.java:377)   
 at app//org.apache.kafka.test.TestUtils.waitForCondition(TestUtils.java:350)   
 at 
app//org.apache.kafka.tools.consumer.group.DescribeConsumerGroupTest.testDescribeExistingGroupWithNoMembers(DescribeConsumerGroupTest.java:430)
 {code}
 

https://ci-builds.apache.org/blue/organizations/jenkins/Kafka%2Fkafka-pr/detail/PR-15816/1/tests



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


[jira] [Created] (KAFKA-16632) Flaky test testDeleteOffsetsOfStableConsumerGroupWithTopicPartition [1] Type=Raft-Isolated, MetadataVersion=3.8-IV0, Security=PLAINTEXT – org.apache.kafka.tools.consumer

2024-04-27 Thread Igor Soarez (Jira)
Igor Soarez created KAFKA-16632:
---

 Summary: Flaky test 
testDeleteOffsetsOfStableConsumerGroupWithTopicPartition [1] 
Type=Raft-Isolated, MetadataVersion=3.8-IV0, Security=PLAINTEXT – 
org.apache.kafka.tools.consumer.group.DeleteOffsetsConsumerGroupCommandIntegrationTest
 Key: KAFKA-16632
 URL: https://issues.apache.org/jira/browse/KAFKA-16632
 Project: Kafka
  Issue Type: Test
Reporter: Igor Soarez


testDeleteOffsetsOfStableConsumerGroupWithTopicPartition [1] 
Type=Raft-Isolated, MetadataVersion=3.8-IV0, Security=PLAINTEXT – 
org.apache.kafka.tools.consumer.group.DeleteOffsetsConsumerGroupCommandIntegrationTest

 
{code:java}
org.opentest4j.AssertionFailedError: expected: not equal but was: <0>   at 
app//org.junit.jupiter.api.AssertionFailureBuilder.build(AssertionFailureBuilder.java:152)
   at 
app//org.junit.jupiter.api.AssertionFailureBuilder.buildAndThrow(AssertionFailureBuilder.java:132)
   at 
app//org.junit.jupiter.api.AssertNotEquals.failEqual(AssertNotEquals.java:277)  
 at 
app//org.junit.jupiter.api.AssertNotEquals.assertNotEquals(AssertNotEquals.java:94)
  at 
app//org.junit.jupiter.api.AssertNotEquals.assertNotEquals(AssertNotEquals.java:86)
  at 
app//org.junit.jupiter.api.Assertions.assertNotEquals(Assertions.java:1981)  at 
app//org.apache.kafka.tools.consumer.group.DeleteOffsetsConsumerGroupCommandIntegrationTest.withConsumerGroup(DeleteOffsetsConsumerGroupCommandIntegrationTest.java:213)
 at 
app//org.apache.kafka.tools.consumer.group.DeleteOffsetsConsumerGroupCommandIntegrationTest.testWithConsumerGroup(DeleteOffsetsConsumerGroupCommandIntegrationTest.java:179)
 at 
app//org.apache.kafka.tools.consumer.group.DeleteOffsetsConsumerGroupCommandIntegrationTest.testDeleteOffsetsOfStableConsumerGroupWithTopicPartition(DeleteOffsetsConsumerGroupCommandIntegrationTest.java:96)
{code}
https://ci-builds.apache.org/blue/organizations/jenkins/Kafka%2Fkafka-pr/detail/PR-15816/1/tests



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


[jira] [Created] (KAFKA-16631) Flaky test - testDeleteOffsetsOfStableConsumerGroupWithTopicOnly [1] Type=Raft-Isolated, MetadataVersion=3.8-IV0, Security=PLAINTEXT – org.apache.kafka.tools.consumer.gr

2024-04-27 Thread Igor Soarez (Jira)
Igor Soarez created KAFKA-16631:
---

 Summary: Flaky test - 
testDeleteOffsetsOfStableConsumerGroupWithTopicOnly [1] Type=Raft-Isolated, 
MetadataVersion=3.8-IV0, Security=PLAINTEXT – 
org.apache.kafka.tools.consumer.group.DeleteOffsetsConsumerGroupCommandIntegrationTest
 Key: KAFKA-16631
 URL: https://issues.apache.org/jira/browse/KAFKA-16631
 Project: Kafka
  Issue Type: Test
Reporter: Igor Soarez


testDeleteOffsetsOfStableConsumerGroupWithTopicOnly [1] Type=Raft-Isolated, 
MetadataVersion=3.8-IV0, Security=PLAINTEXT – 
org.apache.kafka.tools.consumer.group.DeleteOffsetsConsumerGroupCommandIntegrationTest

 
{code:java}
org.opentest4j.AssertionFailedError: expected: not equal but was: <0>   at 
app//org.junit.jupiter.api.AssertionFailureBuilder.build(AssertionFailureBuilder.java:152)
   at 
app//org.junit.jupiter.api.AssertionFailureBuilder.buildAndThrow(AssertionFailureBuilder.java:132)
   at 
app//org.junit.jupiter.api.AssertNotEquals.failEqual(AssertNotEquals.java:277)  
 at 
app//org.junit.jupiter.api.AssertNotEquals.assertNotEquals(AssertNotEquals.java:94)
  at 
app//org.junit.jupiter.api.AssertNotEquals.assertNotEquals(AssertNotEquals.java:86)
  at 
app//org.junit.jupiter.api.Assertions.assertNotEquals(Assertions.java:1981)  at 
app//org.apache.kafka.tools.consumer.group.DeleteOffsetsConsumerGroupCommandIntegrationTest.withConsumerGroup(DeleteOffsetsConsumerGroupCommandIntegrationTest.java:213)
 at 
app//org.apache.kafka.tools.consumer.group.DeleteOffsetsConsumerGroupCommandIntegrationTest.testWithConsumerGroup(DeleteOffsetsConsumerGroupCommandIntegrationTest.java:179)
 at 
app//org.apache.kafka.tools.consumer.group.DeleteOffsetsConsumerGroupCommandIntegrationTest.testDeleteOffsetsOfStableConsumerGroupWithTopicOnly(DeleteOffsetsConsumerGroupCommandIntegrationTest.java:105)
{code}
https://ci-builds.apache.org/blue/organizations/jenkins/Kafka%2Fkafka-pr/detail/PR-15816/1/tests



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


[jira] [Created] (KAFKA-16630) Flaky test "testPollReturnsRecords(GroupProtocol).groupProtocol=CLASSIC" – org.apache.kafka.clients.consumer.KafkaConsumerTest

2024-04-27 Thread Igor Soarez (Jira)
Igor Soarez created KAFKA-16630:
---

 Summary: Flaky test 
"testPollReturnsRecords(GroupProtocol).groupProtocol=CLASSIC" – 
org.apache.kafka.clients.consumer.KafkaConsumerTest
 Key: KAFKA-16630
 URL: https://issues.apache.org/jira/browse/KAFKA-16630
 Project: Kafka
  Issue Type: Test
Reporter: Igor Soarez


"testPollReturnsRecords(GroupProtocol).groupProtocol=CLASSIC" – 
org.apache.kafka.clients.consumer.KafkaConsumerTest

 
{code:java}
org.opentest4j.AssertionFailedError: expected: <0> but was: <5> at 
app//org.junit.jupiter.api.AssertionFailureBuilder.build(AssertionFailureBuilder.java:151)
   at 
app//org.junit.jupiter.api.AssertionFailureBuilder.buildAndThrow(AssertionFailureBuilder.java:132)
   at 
app//org.junit.jupiter.api.AssertEquals.failNotEqual(AssertEquals.java:197)  at 
app//org.junit.jupiter.api.AssertEquals.assertEquals(AssertEquals.java:150)  at 
app//org.junit.jupiter.api.AssertEquals.assertEquals(AssertEquals.java:145)  at 
app//org.junit.jupiter.api.Assertions.assertEquals(Assertions.java:531)  at 
app//org.apache.kafka.clients.consumer.KafkaConsumerTest.testPollReturnsRecords(KafkaConsumerTest.java:289)
 {code}
[https://ci-builds.apache.org/blue/organizations/jenkins/Kafka%2Fkafka-pr/detail/PR-15816/1/tests]

 



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


Re: [DISCUSS] KIP-1018: Introduce max remote fetch timeout config

2024-04-27 Thread Federico Valeri
Hi Kamal, it looks like all TS configurations starts with "remote."
prefix, so I was wondering if we should name it
"remote.fetch.max.wait.ms".

On Fri, Apr 26, 2024 at 7:07 PM Kamal Chandraprakash
 wrote:
>
> Hi all,
>
> If there are no more comments, I'll start a vote thread by tomorrow.
> Please review the KIP.
>
> Thanks,
> Kamal
>
> On Sat, Mar 30, 2024 at 11:08 PM Kamal Chandraprakash <
> kamal.chandraprak...@gmail.com> wrote:
>
> > Hi all,
> >
> > Bumping the thread. Please review this KIP. Thanks!
> >
> > On Thu, Feb 1, 2024 at 9:11 PM Kamal Chandraprakash <
> > kamal.chandraprak...@gmail.com> wrote:
> >
> >> Hi Jorge,
> >>
> >> Thanks for the review! Added your suggestions to the KIP. PTAL.
> >>
> >> The `fetch.max.wait.ms` config will be also applicable for topics
> >> enabled with remote storage.
> >> Updated the description to:
> >>
> >> ```
> >> The maximum amount of time the server will block before answering the
> >> fetch request
> >> when it is reading near to the tail of the partition (high-watermark) and
> >> there isn't
> >> sufficient data to immediately satisfy the requirement given by
> >> fetch.min.bytes.
> >> ```
> >>
> >> --
> >> Kamal
> >>
> >> On Thu, Feb 1, 2024 at 12:12 AM Jorge Esteban Quilcate Otoya <
> >> quilcate.jo...@gmail.com> wrote:
> >>
> >>> Hi Kamal,
> >>>
> >>> Thanks for this KIP! It should help to solve one of the main issues with
> >>> tiered storage at the moment that is dealing with individual consumer
> >>> configurations to avoid flooding logs with interrupted exceptions.
> >>>
> >>> One of the topics discussed in [1][2] was on the semantics of `
> >>> fetch.max.wait.ms` and how it's affected by remote storage. Should we
> >>> consider within this KIP the update of `fetch.max.wail.ms` docs to
> >>> clarify
> >>> it only applies to local storage?
> >>>
> >>> Otherwise, LGTM -- looking forward to see this KIP adopted.
> >>>
> >>> [1] https://issues.apache.org/jira/browse/KAFKA-15776
> >>> [2] https://github.com/apache/kafka/pull/14778#issuecomment-1820588080
> >>>
> >>> On Tue, 30 Jan 2024 at 01:01, Kamal Chandraprakash <
> >>> kamal.chandraprak...@gmail.com> wrote:
> >>>
> >>> > Hi all,
> >>> >
> >>> > I have opened a KIP-1018
> >>> > <
> >>> >
> >>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-1018%3A+Introduce+max+remote+fetch+timeout+config+for+DelayedRemoteFetch+requests
> >>> > >
> >>> > to introduce dynamic max-remote-fetch-timeout broker config to give
> >>> more
> >>> > control to the operator.
> >>> >
> >>> >
> >>> >
> >>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-1018%3A+Introduce+max+remote+fetch+timeout+config+for+DelayedRemoteFetch+requests
> >>> >
> >>> > Let me know if you have any feedback or suggestions.
> >>> >
> >>> > --
> >>> > Kamal
> >>> >
> >>>
> >>


Jenkins build is still unstable: Kafka » Kafka Branch Builder » trunk #2852

2024-04-27 Thread Apache Jenkins Server
See