Re: [DISCUSS] KIP-116 - Add State Store Checkpoint Interval Configuration

2017-02-13 Thread Guozhang Wang
I think I'm OK to always enable checkpointing, but I'm not sure if we want to checkpoint on every commit. Since in the extreme case users can commit on completed processing each record. So I think it is still valuable to have a checkpoint internal config in this KIP, which can be ignored if EOS is

[jira] [Assigned] (KAFKA-4542) Add authentication based on delegation token.

2017-02-13 Thread Manikumar Reddy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4542?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar Reddy reassigned KAFKA-4542: -- Assignee: Manikumar Reddy > Add authentication based on delegation token. > ---

Re: [VOTE] KIP-48 Support for delegation tokens as an authentication mechanism

2017-02-13 Thread Manikumar
Thank you all for your votes and feedback. The vote has passed with 4 binding votes(Gwen, Jun, Grant, Harsha) and 2 non-binding votes(Roger, Dong Lin). I have updated the relevant wiki pages. Thanks Manikumar On Tue, Feb 14, 2017 at 12:02 AM, Dong Lin wrote: > +1 (non-binding) > > On Mon, Feb

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-13 Thread radai
1. making the client Closeable/AutoCloseable would allow try (Client = ...) {} without the need to finally close. 2. a "stream processing unit" (producer + consumer) currently holds 2 open sockets to every broker it interacts with, because producer and consumer dont share the network stack. if we

[jira] [Commented] (KAFKA-4762) Consumer throwing RecordTooLargeException even when messages are not that large

2017-02-13 Thread huxi (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4762?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15864954#comment-15864954 ] huxi commented on KAFKA-4762: - Logs show that you are using 0.10.x where max.partition.fetch.b

Re: [VOTE] KIP-111 Kafka should preserve the Principal generated by the PrincipalBuilder while processing the request received on socket channel, on the broker.

2017-02-13 Thread radai
if i understand correctly, kafka-acls.sh spins up an instance of (the custom, in our case) Authorizer, and calls things like addAcls(acls: Set[Acl], resource: Resource) on it, which are defined in the interface, hence expected to be "extensible". (side note: if Authorizer and PrincipalBuilder are

Re: [DISCUSS] KIP-112: Handle disk failure for JBOD

2017-02-13 Thread Dong Lin
And the test plan has also been updated to simulate disk failure by changing log directory permission to 000. On Mon, Feb 13, 2017 at 5:50 PM, Dong Lin wrote: > Hi Jun, > > Thanks for the reply. These comments are very helpful. Let me answer them > inline. > > > On Mon, Feb 13, 2017 at 3:25 PM,

Re: [DISCUSS] KIP-112: Handle disk failure for JBOD

2017-02-13 Thread Dong Lin
Hi Jun, Thanks for the reply. These comments are very helpful. Let me answer them inline. On Mon, Feb 13, 2017 at 3:25 PM, Jun Rao wrote: > Hi, Dong, > > Thanks for the reply. A few more replies and new comments below. > > On Fri, Feb 10, 2017 at 4:27 PM, Dong Lin wrote: > > > Hi Jun, > > > >

Re: [VOTE] 0.10.2.0 RC1

2017-02-13 Thread Guozhang Wang
Hi Ian, Thanks for the responses. Another suggestion I had is about your session.timeout.ms config value (currently set to 10 seconds): since Nina meaned that a processing of a single record could be as long as minutes, that session timeout would be insufficient as no heartbeats will be sent by th

Re: [VOTE] 0.10.2.0 RC1

2017-02-13 Thread Matthias J. Sax
Ian, an important hint: it is highly recommended to change "state.dir" configuration parameter from "/tmp/kafka-streams" to a different directory. It might be, that /tmp gets deleted and thus you loose all your cached data. While this is no an issues with regard to data loss (as all data is relia

Re: Need suggestion on sending XML files via kafka

2017-02-13 Thread Ratha v
This Sample program may help you? http://vvratha.blogspot.com.au/2016/07/sample-kafka-producer-and-consumer.html On 14 February 2017 at 06:36, Prashanth Venkatesan < prashanth.181...@gmail.com> wrote: > Hi Team, > > I just started using Kafka. I have a usecase to send XML file or Document > objec

Re: [DISCUSS] KIP-112: Handle disk failure for JBOD

2017-02-13 Thread Jun Rao
Hi, Dong, Thanks for the reply. A few more replies and new comments below. On Fri, Feb 10, 2017 at 4:27 PM, Dong Lin wrote: > Hi Jun, > > Thanks for the detailed comments. Please see answers inline: > > On Fri, Feb 10, 2017 at 3:08 PM, Jun Rao wrote: > > > Hi, Dong, > > > > Thanks for the upda

Re: [VOTE] KIP-98: Exactly Once Delivery and Transactional Messaging

2017-02-13 Thread Apurva Mehta
Hi Jun, Thanks for the comments. 200.1, 200.2, 200.3 were all oversights which have been fixed. 201.1: This has been added to the BeginTxnRequest now. If the transaction.timeout.ms value is greater than max.transaction.timeout.ms, then the BeginTxnRequest will fail with a `InvalidTransactionTime

[jira] [Created] (KAFKA-4763) Handle disk failure for JBOD (KIP-112)

2017-02-13 Thread Dong Lin (JIRA)
Dong Lin created KAFKA-4763: --- Summary: Handle disk failure for JBOD (KIP-112) Key: KAFKA-4763 URL: https://issues.apache.org/jira/browse/KAFKA-4763 Project: Kafka Issue Type: Improvement

[jira] [Updated] (KAFKA-4761) NullPointerException if batch.size=0 for producer config

2017-02-13 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4761?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson updated KAFKA-4761: --- Resolution: Fixed Status: Resolved (was: Patch Available) Issue resolved by pull requ

Re: [VOTE] 0.10.2.0 RC1

2017-02-13 Thread Ian Duffy
Hi Guozhang, Thank you for your assistance on this. > About the stack trace pasted before: is it tailing some warning logs like "Could not create task ... Will retry" Yes, we see the following: 17/02/13 21:49:55 WARN internals.StreamThread: Could not create task 0_93. Will retry. org.apache.kaf

[GitHub] kafka pull request #2545: KAFKA-4761: Fix producer regression handling small...

2017-02-13 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2545 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enab

[jira] [Commented] (KAFKA-4761) NullPointerException if batch.size=0 for producer config

2017-02-13 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4761?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15864550#comment-15864550 ] ASF GitHub Bot commented on KAFKA-4761: --- Github user asfgit closed the pull request

Re: [VOTE] KIP-98: Exactly Once Delivery and Transactional Messaging

2017-02-13 Thread Rajini Sivaram
+1 (non-binding) On Mon, Feb 13, 2017 at 9:25 PM, Bill Bejeck wrote: > Sorry for the late response, +1 > > On Mon, Feb 13, 2017 at 4:21 PM, Jun Rao wrote: > > > Hi, Guozhang, > > > > Thanks for the proposal. I made a pass of the wiki and had the following > > comments. > > > > 200. Message form

[GitHub] kafka pull request #2540: KAFKA-4756: The auto-generated broker id should be...

2017-02-13 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2540 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enab

[jira] [Commented] (KAFKA-4756) The auto-generated broker id should be passed to MetricReporter.configure

2017-02-13 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4756?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15864515#comment-15864515 ] ASF GitHub Bot commented on KAFKA-4756: --- Github user asfgit closed the pull request

[jira] [Resolved] (KAFKA-4756) The auto-generated broker id should be passed to MetricReporter.configure

2017-02-13 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4756?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao resolved KAFKA-4756. Resolution: Fixed Fix Version/s: 0.10.2.0 Issue resolved by pull request 2540 [https://github.com/apa

Re: [VOTE] KIP-111 Kafka should preserve the Principal generated by the PrincipalBuilder while processing the request received on socket channel, on the broker.

2017-02-13 Thread Jun Rao
Hi, Mayuresh, I seems to me that there are two common use cases of authorizer. (1) Use the default SimpleAuthorizer and the kafka-acl to do authorization. (2) Use a customized authorizer and an external tool for authorization. Do you think there is a use case for a customized authorizer and kafka-

Re: [VOTE] KIP-98: Exactly Once Delivery and Transactional Messaging

2017-02-13 Thread Bill Bejeck
Sorry for the late response, +1 On Mon, Feb 13, 2017 at 4:21 PM, Jun Rao wrote: > Hi, Guozhang, > > Thanks for the proposal. I made a pass of the wiki and had the following > comments. > > 200. Message format: > 200.1 MaxTimestampDelta: Does that need to be delta since it's always a > fixed size

[jira] [Updated] (KAFKA-4762) Consumer throwing RecordTooLargeException even when messages are not that large

2017-02-13 Thread Vipul Singh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4762?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Vipul Singh updated KAFKA-4762: --- Description: We were just recently hit by a weird error. Before going in any further, explaining of o

Re: [VOTE] KIP-98: Exactly Once Delivery and Transactional Messaging

2017-02-13 Thread Jun Rao
Hi, Guozhang, Thanks for the proposal. I made a pass of the wiki and had the following comments. 200. Message format: 200.1 MaxTimestampDelta: Does that need to be delta since it's always a fixed size in64? 200.2 The wiki says "At the end we still maintains a message-level CRC". Is that still val

[jira] [Updated] (KAFKA-4762) Consumer throwing RecordTooLargeException even when messages are not that large

2017-02-13 Thread Vipul Singh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4762?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Vipul Singh updated KAFKA-4762: --- Description: We were just recently hit by a weird error. Before going in any further, explaining of o

Re: Need suggestion on sending XML files via kafka

2017-02-13 Thread Prashanth Venkatesan
Hi Team, I just started using Kafka. I have a usecase to send XML file or Document object via Kafka topic using Java. Can you enlight me with the guidance steps to achieve it?? Please apologize and ignore if I am posting to inappropriate mail address. Thanks Prashanth +91-9677103475 India

[jira] [Updated] (KAFKA-3264) Mark the old Scala consumer and related classes as deprecated

2017-02-13 Thread Vahid Hashemian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3264?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Vahid Hashemian updated KAFKA-3264: --- Status: Patch Available (was: Open) > Mark the old Scala consumer and related classes as depr

[jira] [Updated] (KAFKA-2857) ConsumerGroupCommand throws GroupCoordinatorNotAvailableException when describing a non-existent group before the offset topic is created

2017-02-13 Thread Vahid Hashemian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2857?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Vahid Hashemian updated KAFKA-2857: --- Status: Patch Available (was: Open) > ConsumerGroupCommand throws GroupCoordinatorNotAvailabl

Re: [VOTE] 0.10.2.0 RC1

2017-02-13 Thread Guozhang Wang
@Ian, Nina Thanks for the detailed description of your apps. A couple of follow-up questions I have to help us further investigate your issue: 1. About the stack trace pasted before: is it tailing some warning logs like "Could not create task ... Will retry" (i.e. it is part of that warning log)

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-13 Thread Colin McCabe
On Sun, Feb 12, 2017, at 09:21, Jay Kreps wrote: > Hey Colin, > > Thanks for the hard work on this. I know going back and forth on APIs is > kind of frustrating but we're at the point where these things live long > enough and are used by enough people that it is worth the pain. I'm sure > it'll co

[jira] [Created] (KAFKA-4762) Consumer throwing RecordTooLargeException even when messages are not that large

2017-02-13 Thread Vipul Singh (JIRA)
Vipul Singh created KAFKA-4762: -- Summary: Consumer throwing RecordTooLargeException even when messages are not that large Key: KAFKA-4762 URL: https://issues.apache.org/jira/browse/KAFKA-4762 Project: Ka

Re: [VOTE] 0.10.2.0 RC1

2017-02-13 Thread Guozhang Wang
Thanks for reporting the JIRA Swen. Jason has a patch ready under KAFKA-4761 and I have reviewed it. You could try it out and see if it has fixed your issue. After this is merged in, we will need another RC. Guozhang On Mon, Feb 13, 2017 at 9:52 AM, Moczarski, Swen < smoczar...@ebay-kleinanzei

[jira] [Updated] (KAFKA-4761) NullPointerException if batch.size=0 for producer config

2017-02-13 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4761?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson updated KAFKA-4761: --- Priority: Blocker (was: Minor) > NullPointerException if batch.size=0 for producer config > -

[jira] [Updated] (KAFKA-4761) NullPointerException if batch.size=0 for producer config

2017-02-13 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4761?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson updated KAFKA-4761: --- Fix Version/s: 0.10.2.0 > NullPointerException if batch.size=0 for producer config > -

[jira] [Updated] (KAFKA-4761) NullPointerException if batch.size=0 for producer config

2017-02-13 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4761?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson updated KAFKA-4761: --- Status: Patch Available (was: In Progress) https://github.com/apache/kafka/pull/2545/files >

[jira] [Commented] (KAFKA-4671) Fix Streams window retention policy

2017-02-13 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4671?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15864247#comment-15864247 ] ASF GitHub Bot commented on KAFKA-4671: --- GitHub user hachikuji opened a pull request

[GitHub] kafka pull request #2545: KAFKA-4671: Fix producer regression handling small...

2017-02-13 Thread hachikuji
GitHub user hachikuji opened a pull request: https://github.com/apache/kafka/pull/2545 KAFKA-4671: Fix producer regression handling small or zero batch size You can merge this pull request into a Git repository by running: $ git pull https://github.com/hachikuji/kafka KAFKA-47

[jira] [Comment Edited] (KAFKA-1379) Partition reassignment resets clock for time-based retention

2017-02-13 Thread Andrew Olson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15864128#comment-15864128 ] Andrew Olson edited comment on KAFKA-1379 at 2/13/17 6:43 PM: --

Re: [VOTE] KIP-111 Kafka should preserve the Principal generated by the PrincipalBuilder while processing the request received on socket channel, on the broker.

2017-02-13 Thread Mayuresh Gharat
Hi Jun, Thanks for the review and comments. Please find the replies inline : This is so that in the future, we can extend to types like group. ---> Yep, I did think the same. But since the SocketServer was always creating User type, it wasn't actually used. If we go ahead with changes in this KIP

Re: [VOTE] KIP-48 Support for delegation tokens as an authentication mechanism

2017-02-13 Thread Dong Lin
+1 (non-binding) On Mon, Feb 13, 2017 at 10:21 AM, Harsha Chintalapani wrote: > +1. > -Harsha > > On Fri, Feb 10, 2017 at 11:12 PM Manikumar > wrote: > > > Yes, owners and the renewers can always describe their own tokens. > Updated > > the KIP. > > > > On Sat, Feb 11, 2017 at 3:12 AM, Jun Rao

Re: [VOTE] KIP-48 Support for delegation tokens as an authentication mechanism

2017-02-13 Thread Harsha Chintalapani
+1. -Harsha On Fri, Feb 10, 2017 at 11:12 PM Manikumar wrote: > Yes, owners and the renewers can always describe their own tokens. Updated > the KIP. > > On Sat, Feb 11, 2017 at 3:12 AM, Jun Rao wrote: > > > Hi, Mani, > > > > Thanks for the update. Just a minor comment below. Otherwise, +1 from

[jira] [Commented] (KAFKA-1379) Partition reassignment resets clock for time-based retention

2017-02-13 Thread Andrew Olson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1379?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15864128#comment-15864128 ] Andrew Olson commented on KAFKA-1379: - [~hachikuji] Jason, could you confirm if this b

[jira] [Work started] (KAFKA-4761) NullPointerException if batch.size=0 for producer config

2017-02-13 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4761?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-4761 started by Jason Gustafson. -- > NullPointerException if batch.size=0 for producer config >

Re: [VOTE] 0.10.2.0 RC1

2017-02-13 Thread Moczarski, Swen
+0 (non-binding) Thanks for compiling a new release candidate. I get an NullPointerException when setting batch.size=0 on producer config. This worked before with 0.10.1.1. See https://issues.apache.org/jira/browse/KAFKA-4761 Regards, Swen Am 2/10/17, 5:51 PM schrieb "Ewen Cheslack-Postava" :

[jira] [Assigned] (KAFKA-4761) NullPointerException if batch.size=0 for producer config

2017-02-13 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4761?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson reassigned KAFKA-4761: -- Assignee: Jason Gustafson > NullPointerException if batch.size=0 for producer config >

Re: Correct prefetching of data to KTable-like structure on application startup

2017-02-13 Thread Matthias J. Sax
Jan, brokers with version 0.10.1 or higher allow to set both topic cleanup policies in combination: https://cwiki.apache.org/confluence/display/KAFKA/KIP-71%3A+Enable+log+compaction+and+deletion+to+co-exist However, this will only delete data in you changelog topic but not in your RocksDB -- if

Re: KafkaStream Run on 0.9.0.1

2017-02-13 Thread Matthias J. Sax
No. If you want to use Kafka Streams, you need brokers with the same or higher version number. Thus, you need at least 0.10 brokers. This restriction will be relaxed with upcoming 0.10.2 release that is the first release being broker backward compatible. Thus, you can run a 0.10.2 Streams app als

[jira] [Assigned] (KAFKA-4494) Significant startup delays in KStreams app

2017-02-13 Thread Damian Guy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4494?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Damian Guy reassigned KAFKA-4494: - Assignee: Damian Guy > Significant startup delays in KStreams app > -

[jira] [Work started] (KAFKA-4494) Significant startup delays in KStreams app

2017-02-13 Thread Damian Guy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4494?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-4494 started by Damian Guy. - > Significant startup delays in KStreams app >

[jira] [Created] (KAFKA-4761) NullPointerException if batch.size=0 for producer config

2017-02-13 Thread Swen Moczarski (JIRA)
Swen Moczarski created KAFKA-4761: - Summary: NullPointerException if batch.size=0 for producer config Key: KAFKA-4761 URL: https://issues.apache.org/jira/browse/KAFKA-4761 Project: Kafka Issu

KafkaStream Run on 0.9.0.1

2017-02-13 Thread Abhinav Patil
Hi, Is kafka stream run on 0.9.0.1 ? -- *Regards,* *Abhinav PATIL* Mobile: +91-902812760

Re: [VOTE] 0.10.2.0 RC1

2017-02-13 Thread Nina Hanzlikova
Hi, I am a colleague of Ian's. We use the following processing pipeline in stream app he mentions: https://github.com/zalando-incubator/pipeline-backbone The streams are built using: object Run extends App { // ... private val latch = new CountDownLatch(1) private val builder = { val

[jira] [Updated] (KAFKA-4760) Restarting one MirrorMaker causes a global pause in mirroring.

2017-02-13 Thread JIRA
[ https://issues.apache.org/jira/browse/KAFKA-4760?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Tomasz Gański updated KAFKA-4760: - Description: When 1 MirrorMaker is restarted in consumer group, the whole consumer group stop wor

[jira] [Updated] (KAFKA-4760) Restarting one MirrorMaker causes a global pause in mirroring.

2017-02-13 Thread JIRA
[ https://issues.apache.org/jira/browse/KAFKA-4760?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Tomasz Gański updated KAFKA-4760: - Description: When 1 MirrorMaker is restarted in consumer group, the whole consumer group stop wor

[jira] [Updated] (KAFKA-4760) Restarting one MirrorMaker causes a global pause in mirroring.

2017-02-13 Thread JIRA
[ https://issues.apache.org/jira/browse/KAFKA-4760?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Tomasz Gański updated KAFKA-4760: - Affects Version/s: 0.10.1.1 Component/s: tools > Restarting one MirrorMaker causes a gl

[jira] [Updated] (KAFKA-4760) Restarting one MirrorMaker causes a global pause in mirroring.

2017-02-13 Thread JIRA
[ https://issues.apache.org/jira/browse/KAFKA-4760?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Tomasz Gański updated KAFKA-4760: - Description: When 1 MirrorMaker is restarted in consumer group, the whole consumer group stop wor

[jira] [Updated] (KAFKA-4760) Restarting one MirrorMaker causes a global pause in mirroring.

2017-02-13 Thread JIRA
[ https://issues.apache.org/jira/browse/KAFKA-4760?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Tomasz Gański updated KAFKA-4760: - Description: When 1 MirrorMaker is restarted in consumer group, the whole consumer group stop wor

[jira] [Updated] (KAFKA-4760) Restarting one MirrorMaker causes a global pause in mirroring.

2017-02-13 Thread JIRA
[ https://issues.apache.org/jira/browse/KAFKA-4760?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Tomasz Gański updated KAFKA-4760: - Description: When 1 MirrorMaker is restarted in consumer group, the whole consumer group stop wor

[jira] [Updated] (KAFKA-4760) Restarting one MirrorMaker causes a global pause in mirroring.

2017-02-13 Thread JIRA
[ https://issues.apache.org/jira/browse/KAFKA-4760?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Tomasz Gański updated KAFKA-4760: - Description: When 1 MirrorMaker is restarted in consumer group, the whole consumer group stop wor

[jira] [Updated] (KAFKA-4760) Restarting one MirrorMaker causes a global pause in mirroring.

2017-02-13 Thread JIRA
[ https://issues.apache.org/jira/browse/KAFKA-4760?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Tomasz Gański updated KAFKA-4760: - Description: When 1 MirrorMaker is restarted in consumer group, the whole consumer group stop wor

[jira] [Created] (KAFKA-4760) Restarting one MirrorMaker causes a global pause in mirroring.

2017-02-13 Thread JIRA
Tomasz Gański created KAFKA-4760: Summary: Restarting one MirrorMaker causes a global pause in mirroring. Key: KAFKA-4760 URL: https://issues.apache.org/jira/browse/KAFKA-4760 Project: Kafka

[GitHub] kafka pull request #2544: KFKA-4340: update the upgrade.html

2017-02-13 Thread becketqin
GitHub user becketqin opened a pull request: https://github.com/apache/kafka/pull/2544 KFKA-4340: update the upgrade.html You can merge this pull request into a Git repository by running: $ git pull https://github.com/becketqin/kafka KAFKA-4340_follow_up Alternatively you can

Re: Correct prefetching of data to KTable-like structure on application startup

2017-02-13 Thread Jan Lukavský
Hi Michael, sorry for my late answer. Configuring the topic as you suggest is one option (and I will configure it that way), but I wanted to combine the two data retention mechanisms (if possible). I would like to use log compaction, so that I will always get at least the last message for giv

Re: [VOTE] 0.10.2.0 RC1

2017-02-13 Thread Eno Thereska
+1 (non binding) Checked streams. Verified that stream tests work and examples off confluentinc/examples/kafka-streams work. Thanks Eno > On 10 Feb 2017, at 16:51, Ewen Cheslack-Postava wrote: > > Hello Kafka users, developers and client-developers, > > This is RC1 for release of Apache Kafk