I have also created the pull request for the behaviour change.
https://github.com/apache/kafka/pull/13455
Thanks again,
Dan
On Thu, Mar 23, 2023 at 11:40 PM Dan S wrote:
> I have created the jira, the PR will follow within the next few days.
> https://issues.apache.org/jira/browse/KAFKA
I have created the jira, the PR will follow within the next few days.
https://issues.apache.org/jira/browse/KAFKA-14841
Thanks again
On Thu, Mar 23, 2023 at 7:33 PM Philip Nee wrote:
> Awesome!
>
> On Thu, Mar 23, 2023 at 12:31 PM Dan S wrote:
>
> > I'll pick it up, than
I'll pick it up, thanks!
On Thu, Mar 23, 2023, 19:27 Philip Nee wrote:
> Hey Dan,
>
> Your analysis looks right. I do see TODO item there to implement the
> rebalance callback. Would you like to create a jira issue and work on that?
>
> Thanks,
> P
>
> On Thu
s indeed
desired behavior.
On Thu, Mar 23, 2023, 18:17 Philip Nee wrote:
> Hey Dan,
>
> Thanks for looking into this. Are you talking about MockConsumer? If you
> invoke subscribe(Collection topics), it actually registers a Noop
> callback. Perhaps this is what you are seeing?
>
&
Hello all,
It seems to me based on reading the code, that the consumer rebalance
listener that is passed into the mock consumer when subscribing to a topic
is not actually called when a rebalance is simulated. My understanding is
that the consumer rebalance listener is called from the consumer
coo
> fixed recently, so hopefully stability will improve a bit. See KAFKA-14433
> for details.
>
> best,
> Colin
>
> On Thu, Nov 24, 2022, at 12:48, John Roesler wrote:
> > Hi Dan,
> >
> > I’m not sure if there’s a consistently used tag, but I’ve gotten good
> >
Thanks for the reply John! Is there a jira tag or view or something that
can be used to find all the failing tests and maybe even try to fix them
(even if fix just means extending a timeout)?
On Thu, Nov 24, 2022, 16:03 John Roesler wrote:
> Hi Dan,
>
> Thanks for pointing this o
ine is not
the only one. I was wondering if there is a place where all the flaky tests
are being tracked, and if it makes sense to fix (or at least temporarily
disable) them so that confidence in new PRs could be increased.
Thanks,
Dan
open a PR first? Should I simply wait for discussion comments or
votes (I've gotten no votes yet)?
Thanks so much,
Dan
On Fri, Nov 11, 2022 at 1:30 AM Knowles Atchison Jr
wrote:
> This would be helpful. For our own client library wrappers we implemented
> this functionality for any t
added to the builder's withXYZ methods.
Please let me know if you have any comments, questions, or suggestions in
the discussion thread, or vote here!
https://cwiki.apache.org/confluence/display/KAFKA/KIP-886
%3A+Add+Client+Producer+and+Consumer+Builders
Thanks,
Dan
added to the builder's withXYZ methods.
Please let me know if you have any comments, questions, or suggestions!
https://cwiki.apache.org/confluence/display/KAFKA/KIP-886%3A+Add+Client+Producer+and+Consumer+Builders
Thanks,
Dan
,
Dan
Hello,
I would like to contribute to kafka,
my wiki id, jira id, and github username are all "scanteianu"
Thanks,
Dan
m).
https://github.com/apache/kafka/pull/12753
Thank you,
Dan
something wrong/missing something, or is there a bug, and if so,
what should I do (file a jira, add a fix to the pr, open a new pr?). What
is the desired behaviour? Is it to get message at offset 11 in both cases?
Thanks,
Dan
something wrong/missing something, or is there a bug, and if so,
what should I do (file a jira, add a fix to the pr, open a new pr?). What
is the desired behaviour? Is it to get message at offset 11 in both cases?
Thanks,
Dan
dan norwood created KAFKA-8414:
--
Summary:
org.apache.kafka.common.metrics.MetricsTest.testConcurrentReadUpdateReport hang
Key: KAFKA-8414
URL: https://issues.apache.org/jira/browse/KAFKA-8414
Project
After 80 hours, the results of the voting:
+3 binding
+1 non-binding
(+1 from me too :)
I'll update this wiki to show this KIP as approved. 🥳
thanks
dan
On Thu, May 9, 2019 at 8:11 AM Rajini Sivaram
wrote:
> Hi Dan,
>
> Thanks for the KIP, +1 (binding)
>
> Regards,
>
I would like to start voting for
https://cwiki.apache.org/confluence/display/KAFKA/KIP-465%3A+Add+Consolidated+Connector+Endpoint+to+Connect+REST+API
thanks
dan
,
"task": 0
}
],
"type": "source"
},
"status": {
"name": "blah",
"connector": {
"state": "RUNNING",
"worker_id": "10.2
The intent of this KIP is to add a consolidate endpoint to connect that
gives consumers of the api a one stop shop for all their connector info
needs.
https://cwiki.apache.org/confluence/display/KAFKA/KIP-465%3A+Add+Consolidated+Connector+Endpoint+to+Connect+REST+API
thanks,
dan
dan norwood created KAFKA-8309:
--
Summary: KIP-465: Add Consolidated Connector Endpoint to Connect
REST API
Key: KAFKA-8309
URL: https://issues.apache.org/jira/browse/KAFKA-8309
Project: Kafka
dan norwood created KAFKA-7713:
--
Summary: producer io-wait-ratio > 1
Key: KAFKA-7713
URL: https://issues.apache.org/jira/browse/KAFKA-7713
Project: Kafka
Issue Type: Bug
Affects Versi
Dan created KAFKA-6547:
--
Summary: group offset reset and begin_offset ignored/no effect
Key: KAFKA-6547
URL: https://issues.apache.org/jira/browse/KAFKA-6547
Project: Kafka
Issue Type: Bug
Hi,
I'm running an app on Kafka Streams 1.0.0, and in the past day a lot of
nodes are failing and I see this in the log.
These appear to be failures when attempting to update the changelog. Any
ideas on what I should do to work around this? Should I configure separate
retry and timeouts for the
o get more
retries for changelog publishing?
Thanks!
Dan
dan norwood created KAFKA-6418:
--
Summary: adminclient throws timeoutexception when there is a
SchemaException
Key: KAFKA-6418
URL: https://issues.apache.org/jira/browse/KAFKA-6418
Project: Kafka
need to add
all this info for all describeTopics() calls. not sure of the
ramifications/overhead here.
dan
On Wed, Dec 13, 2017 at 10:10 AM, Colin McCabe wrote:
> On Wed, Dec 13, 2017, at 10:00, dan wrote:
> > > Why not just return
> > > org.apache.kafka.clients.ad
eplica mappings it would create (i have no idea
if this is actually possible)
dan
On Wed, Dec 13, 2017 at 9:55 AM, Colin McCabe wrote:
> On Tue, Dec 12, 2017, at 19:02, Ewen Cheslack-Postava wrote:
> > re: API versions, I actually wasn't sure if we needed it or not. I'm fine
> &
Jorge,
Would you consider also adding `listGroupOffsets` to this KIP?
thanks
dan
On Tue, Dec 12, 2017 at 9:39 AM, Jason Gustafson wrote:
> Hi Colin,
>
> They do share the same namespace. We have a "protocol type" field in the
> JoinGroup request to make sure that all m
Colin/Ewen,
i will add changes to bump the API version.
any preferences on the return type for the new method? tbh it seems like
returning a NewTopic could make sense because the ConfigResource for a
TOPIC type does not let me encode `numPartitions`
thanks
dan
On Mon, Dec 11, 2017 at 7:22 PM
even better for them. since the person maintaining the actual cluster
has put thought in to this config. and as the maintainer keeps working on
making the cluster run better they can change and tune things on the
cluster level as needed.
dan
On Wed, Dec 6, 2017 at 11:51 AM, Dong Lin wrote:
>
KIP,
another option would be to have the `CreateTopicsOptions.validateOnly=true`
version return data, but seems more invasive/questionable.
dan
On Wed, Dec 6, 2017 at 5:10 AM, Rajini Sivaram
wrote:
> Hi Dan,
>
> Thank you for the KIP. KIP-226 (https://cwiki.apache.org/
> confluence/d
s for topics. right
now a user is left to either decide configs a priori and lose this
functionality, or guess/check what they need to set and end in a possibly
bad situation until they can get their *live* topic configured.
dan
On Mon, Dec 4, 2017 at 2:50 PM, Dong Lin wrote:
> Hey Dan,
>
>
updated again :)
by having users always set all configs you lose the ability to use the
broker defaults as intended, since topic configs are overlaid. example in
the kip doc.
dan
On Mon, Dec 4, 2017 at 11:47 AM, Dong Lin wrote:
> Hey Dan,
>
> Thanks for the update. I just want to
Dong,
i added a section on current state and workarounds along with my arguments
for why they are less than optimal to the wiki. but the jist of it is you
can end up with messages in your topic in an incorrect/invalid state if you
do this.
thanks,
dan
On Mon, Dec 4, 2017 at 10:53 AM, Dong Lin
updated to be more explicit
On Mon, Dec 4, 2017 at 9:55 AM, Ted Yu wrote:
> In the example, resources is derived from ConfigResource.
> It would be clearer if you show the derivation.
>
> Cheers
>
> On Mon, Dec 4, 2017 at 9:51 AM, dan wrote:
>
> > I added sample us
I added sample usage as well as a WIP commit to the KIP.
dan
On Mon, Dec 4, 2017 at 9:36 AM, Ted Yu wrote:
> Can you add some code snippet showing how ConfigResource delivers topic
> defaults with the proposed change ?
>
>
> Cheers
>
> On Mon, Dec 4, 2017 at 9:25 AM, dan
I would like to start a discussion about KIP-234
https://cwiki.apache.org/confluence/display/KAFKA/KIP-234%3A+add+support+for+getting+topic+defaults+from+AdminClient
thanks
dan
dan norwood created KAFKA-6309:
--
Summary: add support for getting topic defaults from AdminClient
Key: KAFKA-6309
URL: https://issues.apache.org/jira/browse/KAFKA-6309
Project: Kafka
Issue Type
dan norwood created KAFKA-5979:
--
Summary: kip-120 changed internal streams topic naming
Key: KAFKA-5979
URL: https://issues.apache.org/jira/browse/KAFKA-5979
Project: Kafka
Issue Type: Bug
dan norwood created KAFKA-5658:
--
Summary: adminclient will stop working after some amount of time
Key: KAFKA-5658
URL: https://issues.apache.org/jira/browse/KAFKA-5658
Project: Kafka
Issue Type
Dan created KAFKA-5587:
--
Summary: Processor got uncaught exception: NullPointerException
Key: KAFKA-5587
URL: https://issues.apache.org/jira/browse/KAFKA-5587
Project: Kafka
Issue Type: Bug
[
https://issues.apache.org/jira/browse/KAFKA-5275?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16016718#comment-16016718
]
dan norwood commented on KAFKA-5275:
it would be nice to have a def
[
https://issues.apache.org/jira/browse/KAFKA-3984?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16010436#comment-16010436
]
Dan commented on KAFKA-3984:
We encountered the same problem in 0.10.1.1. Is there any
here we are, only 72hrs later and we have reached agreement.
+4 binding from Guozhang, Ismael, Ram, and Ewen.
+2 non binding from Bharat and Konstantine.
thanks everyone who voted/commented.
dan
On Wed, May 10, 2017 at 11:38 AM, Guozhang Wang wrote:
> +1
>
> On Wed, May 10, 2017 at
[
https://issues.apache.org/jira/browse/KAFKA-5213?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16005651#comment-16005651
]
dan norwood commented on KAFKA-5213:
awesome we found a real problem.
i did run
[
https://issues.apache.org/jira/browse/KAFKA-5213?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16005525#comment-16005525
]
dan norwood commented on KAFKA-5213:
turns out i was running trunk client aga
[
https://issues.apache.org/jira/browse/KAFKA-5213?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
dan norwood updated KAFKA-5213:
---
Description:
i have a streams app that was working recently while pointing at trunk. this
morning i
dan norwood created KAFKA-5213:
--
Summary: IllegalStateException in ensureOpenForRecordAppend
Key: KAFKA-5213
URL: https://issues.apache.org/jira/browse/KAFKA-5213
Project: Kafka
Issue Type: Bug
+1 (non binding)
thanks Konstantine :thumbsup:
On Mon, May 8, 2017 at 3:24 PM, Guozhang Wang wrote:
> +1
>
> On Mon, May 8, 2017 at 1:36 PM, Stephane Maarek <
> steph...@simplemachines.com.au> wrote:
>
> > +1
> > Thanks heaps I can’t wait!
> >
> >
> > On 9/5/17, 4:48 am, "Konstantine Karantasis
oting thread will stay active for a minimum of 72 hours.
thanks
dan
thanks for the feedback, it all sounds good. i have made the changes to the
pr and the kip.
dan
On Fri, May 5, 2017 at 9:29 AM, Konstantine Karantasis <
konstant...@confluent.io> wrote:
> Thank you for the KIP. It's a nice improvement.
>
> Two small suggestions:
>
> 1)
another kip.
dan
On Thu, May 4, 2017 at 2:36 PM, Colin McCabe wrote:
> On Thu, May 4, 2017, at 13:46, Magnus Edenhill wrote:
> > Hey Colin,
> >
> > good KIP!
> >
> > Some comments:
> >
> > 1a. For operation, permission_type and resource_type: is there
https://github.com/apache/kafka/pull/2960
thanks
dan
[
https://issues.apache.org/jira/browse/KAFKA-4343?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
dan norwood reassigned KAFKA-4343:
--
Assignee: dan norwood
> Connect REST API should expose whether each connector is a source
Dan created KAFKA-5115:
--
Summary: Use bootstrap.servers to refresh metadata
Key: KAFKA-5115
URL: https://issues.apache.org/jira/browse/KAFKA-5115
Project: Kafka
Issue Type: Improvement
Affects
[
https://issues.apache.org/jira/browse/KAFKA-4979?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15971068#comment-15971068
]
Dan commented on KAFKA-4979:
Thank you for the reply! I created the same request and got
[
https://issues.apache.org/jira/browse/KAFKA-4979?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15968803#comment-15968803
]
Dan commented on KAFKA-4979:
How is the JoinGroupRequest Handcrafted?
> Hand
[
https://issues.apache.org/jira/browse/KAFKA-4987?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15951490#comment-15951490
]
dan norwood commented on KAFKA-4987:
this was a custom client based on 0.10
[
https://issues.apache.org/jira/browse/KAFKA-4987?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
dan norwood updated KAFKA-4987:
---
Affects Version/s: (was: 0.10.2.0)
> Topic creation allows invalid config values on runn
[
https://issues.apache.org/jira/browse/KAFKA-4987?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
dan norwood updated KAFKA-4987:
---
Affects Version/s: 0.10.0.1
> Topic creation allows invalid config values on running brok
dan norwood created KAFKA-4987:
--
Summary: Topic creation allows invalid config values on running
brokers
Key: KAFKA-4987
URL: https://issues.apache.org/jira/browse/KAFKA-4987
Project: Kafka
[
https://issues.apache.org/jira/browse/KAFKA-4834?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15899306#comment-15899306
]
Dan commented on KAFKA-4834:
No. I found in another broker's log that it changed
[
https://issues.apache.org/jira/browse/KAFKA-4845?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15899299#comment-15899299
]
Dan commented on KAFKA-4845:
I checked 0.10.2.0 again and it was indeed fixed. Thanks a
[
https://issues.apache.org/jira/browse/KAFKA-4845?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Dan updated KAFKA-4845:
---
Status: Open (was: Patch Available)
> KafkaConsumer.seekToEnd cannot take effect when integrating with sp
[
https://issues.apache.org/jira/browse/KAFKA-4845?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Dan updated KAFKA-4845:
---
Status: Patch Available (was: Open)
> KafkaConsumer.seekToEnd cannot take effect when integrating with sp
Dan created KAFKA-4845:
--
Summary: KafkaConsumer.seekToEnd cannot take effect when
integrating with spark streaming
Key: KAFKA-4845
URL: https://issues.apache.org/jira/browse/KAFKA-4845
Project: Kafka
[
https://issues.apache.org/jira/browse/KAFKA-4834?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15896740#comment-15896740
]
Dan commented on KAFKA-4834:
By invoking AdminUtils.deleteTopic(zkUtils, topic).
It se
Dan created KAFKA-4834:
--
Summary: Kafka cannot delete topic with ReplicaStateMachine went
wrong
Key: KAFKA-4834
URL: https://issues.apache.org/jira/browse/KAFKA-4834
Project: Kafka
Issue Type: Bug
dan norwood created KAFKA-4717:
--
Summary: connect jars are missing LICENSE/NOTICE files
Key: KAFKA-4717
URL: https://issues.apache.org/jira/browse/KAFKA-4717
Project: Kafka
Issue Type: Bug
nnect where there is a separate endpoint
<https://github.com/apache/kafka/blob/trunk/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResource.java#L49-L58>
to attempt to validate a connect configuration without actually creating
the connector.
dan norwood created KAFKA-3491:
--
Summary: Issue with consumer close() in finally block with
'enable.auto.commit=true'
Key: KAFKA-3491
URL: https://issues.apache.org/jira/browse/KAFKA-3491
Proj
[
https://issues.apache.org/jira/browse/KAFKA-1419?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14150693#comment-14150693
]
Dan Di Spaltro commented on KAFKA-1419:
---
[~herriojr] It looks like the patch
[
https://issues.apache.org/jira/browse/KAFKA-1367?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14144108#comment-14144108
]
Dan Hoffman commented on KAFKA-1367:
I'd also add that having the broker b
e guarantees.
>
> You can download the release from: http://kafka.apache.org/downloads.html
>
> We welcome your help and feedback. For more information on how to
> report problems, and to get involved, visit the project website at
> http://kafka.apache.org/
>
>
--
Dan Di Spaltro
[
https://issues.apache.org/jira/browse/KAFKA-881?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13786925#comment-13786925
]
Dan F commented on KAFKA-881:
-
Sure enough, that's a race condition. Practically spe
[
https://issues.apache.org/jira/browse/KAFKA-881?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13786084#comment-13786084
]
Dan F commented on KAFKA-881:
-
Any word on whether this will be accepted into 0.7 and 0.8
[
https://issues.apache.org/jira/browse/KAFKA-881?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13749430#comment-13749430
]
Dan F commented on KAFKA-881:
-
1. No, I believe that is the bug. lastAppendTime does
[
https://issues.apache.org/jira/browse/KAFKA-881?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13728571#comment-13728571
]
Dan F commented on KAFKA-881:
-
Sam: Thanks for that. It would be comforting to know if/whe
[
https://issues.apache.org/jira/browse/KAFKA-881?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13725974#comment-13725974
]
Dan F commented on KAFKA-881:
-
I am not necessarily opposed, but I don't unders
[
https://issues.apache.org/jira/browse/KAFKA-881?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13721956#comment-13721956
]
Dan F edited comment on KAFKA-881 at 7/28/13 2:42 PM:
--
Sure. I
[
https://issues.apache.org/jira/browse/KAFKA-881?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13721956#comment-13721956
]
Dan F edited comment on KAFKA-881 at 7/28/13 2:41 PM:
--
Sure. I
[
https://issues.apache.org/jira/browse/KAFKA-881?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Dan F updated KAFKA-881:
Attachment: kafka-roll.again.patch
> Kafka broker not respecting log.roll.ho
[
https://issues.apache.org/jira/browse/KAFKA-881?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Dan F updated KAFKA-881:
Status: Open (was: Patch Available)
> Kafka broker not respecting log.roll.ho
[
https://issues.apache.org/jira/browse/KAFKA-881?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13721956#comment-13721956
]
Dan F commented on KAFKA-881:
-
Sure. I did:
I did the below, then submitted k
[
https://issues.apache.org/jira/browse/KAFKA-881?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Dan F updated KAFKA-881:
Status: Patch Available (was: Open)
> Kafka broker not respecting log.roll.ho
Dan Swanson created KAFKA-975:
-
Summary: Leader not local for partition when partition is leader
Key: KAFKA-975
URL: https://issues.apache.org/jira/browse/KAFKA-975
Project: Kafka
Issue Type
[
https://issues.apache.org/jira/browse/KAFKA-975?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13709845#comment-13709845
]
Dan Swanson commented on KAFKA-975:
---
I have also tried using IP instead of host name
[
https://issues.apache.org/jira/browse/KAFKA-975?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Dan Swanson updated KAFKA-975:
--
Summary: Leader not local for partition when partition is leader
[
https://issues.apache.org/jira/browse/KAFKA-881?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13652447#comment-13652447
]
Dan F commented on KAFKA-881:
-
We are going to roll this out. Is anyone willing to look at
[
https://issues.apache.org/jira/browse/KAFKA-881?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13651487#comment-13651487
]
Dan F commented on KAFKA-881:
-
Testing on my own server looks pretty good:
- start se
[
https://issues.apache.org/jira/browse/KAFKA-881?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Dan F updated KAFKA-881:
Attachment: kafka_roll.patch
I attach a patch to 0.7.2. It passes the unit tests. I added a unit test for a
2nd
[
https://issues.apache.org/jira/browse/KAFKA-881?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13651225#comment-13651225
]
Dan F commented on KAFKA-881:
-
There would be major issues for us switching versions.
[
https://issues.apache.org/jira/browse/KAFKA-881?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13647912#comment-13647912
]
Dan F commented on KAFKA-881:
-
Someone pointed out a particularly easy fix: don't re
[
https://issues.apache.org/jira/browse/KAFKA-881?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13646405#comment-13646405
]
Dan F commented on KAFKA-881:
-
Note: it would be useful for us to have the files roll ove
Dan F created KAFKA-881:
---
Summary: Kafka broker not respecting log.roll.hours
Key: KAFKA-881
URL: https://issues.apache.org/jira/browse/KAFKA-881
Project: Kafka
Issue Type: Bug
Components
97 matches
Mail list logo