Re: [VOTE] KIP-87 - Add Compaction Tombstone Flag

2016-12-19 Thread Michael Pearce
Agreed. As I said there is still a use case just I wouldn't be pushing for it, the. Need for it reduces for me. Sent using OWA for iPhone From: radai Sent: Tuesday, December 20, 2016 3:51:33 AM To: dev@kafka.apache.org

[jira] [Resolved] (KAFKA-4447) Controller resigned but it also acts as a controller for a long time

2016-12-19 Thread Json Tu (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4447?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Json Tu resolved KAFKA-4447. Resolution: Fixed > Controller resigned but it also acts as a controller for a long time >

Jenkins build is back to normal : kafka-trunk-jdk7 #1768

2016-12-19 Thread Apache Jenkins Server
See

Jenkins build is back to normal : kafka-trunk-jdk8 #1114

2016-12-19 Thread Apache Jenkins Server
See

Re: [VOTE] KIP-87 - Add Compaction Tombstone Flag

2016-12-19 Thread radai
i didnt mean to sound as insisting. what i actually mean is it would still be a valid issue but of much less concern. On Mon, Dec 19, 2016 at 7:50 PM, radai wrote: > this kip fixes a "bug" (quirk?) that arises when people implement headers > "in V" (in the payload

Re: [VOTE] KIP-87 - Add Compaction Tombstone Flag

2016-12-19 Thread radai
this kip fixes a "bug" (quirk?) that arises when people implement headers "in V" (in the payload part of a message). if you have proper headers you obviously dont need to to stick them in V and so wont run into this, but its still a valid issue. On Mon, Dec 19, 2016 at 3:06 PM, Jay Kreps

Re: [VOTE] 0.10.1.1 RC1

2016-12-19 Thread Vahid S Hashemian
Hi Guozhang, I also verified the quickstart on Ubuntu and Mac. +1 on those. On Windows OS there are a couple of issues for which the following PRs exist: - https://github.com/apache/kafka/pull/2146 (already merged to trunk) - https://github.com/apache/kafka/pull/2238 (open) These issues are

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

2016-12-19 Thread radai
1. on write amplification: i dont see x6 the writes, at worst i see x2 the writes - once to the "tx log", then read and again to the destination partition. if you have some != 1 replication factor than both the 1st and the 2nd writes get replicated, but it is still a relative factor of x2. what am

Build failed in Jenkins: kafka-trunk-jdk7 #1767

2016-12-19 Thread Apache Jenkins Server
See Changes: [ismael] MINOR: Fix exception handling in case of file record truncation during -- [...truncated 7927 lines...] kafka.log.LogTest > testReadWithTooSmallMaxLength STARTED

[jira] [Commented] (KAFKA-4560) Min / Max Partitions Fetch Records params

2016-12-19 Thread Stephane Maarek (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4560?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15763036#comment-15763036 ] Stephane Maarek commented on KAFKA-4560: Basically if you set the max bytes, each partitions will

[jira] [Commented] (KAFKA-4560) Min / Max Partitions Fetch Records params

2016-12-19 Thread huxi (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4560?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15763027#comment-15763027 ] huxi commented on KAFKA-4560: - Not sure if I catch your meaning. You want a config controlling consumer to

[jira] [Updated] (KAFKA-4447) Controller resigned but it also acts as a controller for a long time

2016-12-19 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4447?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jiangjie Qin updated KAFKA-4447: Fix Version/s: 0.10.2.0 > Controller resigned but it also acts as a controller for a long time >

[jira] [Commented] (KAFKA-4447) Controller resigned but it also acts as a controller for a long time

2016-12-19 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4447?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15763005#comment-15763005 ] ASF GitHub Bot commented on KAFKA-4447: --- Github user asfgit closed the pull request at:

[GitHub] kafka pull request #2191: KAFKA-4447: Controller resigned but it also acts a...

2016-12-19 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2191 --- 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

[jira] [Commented] (KAFKA-4560) Min / Max Partitions Fetch Records params

2016-12-19 Thread Stephane Maarek (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4560?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15762979#comment-15762979 ] Stephane Maarek commented on KAFKA-4560: Hi [~huxi_2b], this setting has the following

[jira] [Commented] (KAFKA-4560) Min / Max Partitions Fetch Records params

2016-12-19 Thread huxi (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4560?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15762931#comment-15762931 ] huxi commented on KAFKA-4560: - There is a consumer config named "max.poll.records" that controls the maximum

Build failed in Jenkins: kafka-trunk-jdk8 #1113

2016-12-19 Thread Apache Jenkins Server
See Changes: [ismael] KAFKA-4166; Fix transient MM failure caused by slow old consumer [ismael] KAFKA-4526; Disable throttling test until it can be fixed correctly. -- [...truncated 7936

[GitHub] kafka pull request #2275: Fix exception handling in case of file record trun...

2016-12-19 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2275 --- 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

[jira] [Created] (KAFKA-4560) Min / Max Partitions Fetch Records params

2016-12-19 Thread Stephane Maarek (JIRA)
Stephane Maarek created KAFKA-4560: -- Summary: Min / Max Partitions Fetch Records params Key: KAFKA-4560 URL: https://issues.apache.org/jira/browse/KAFKA-4560 Project: Kafka Issue Type:

[GitHub] kafka pull request #2278: KAFKA-4526 - Disable throttling test until it can ...

2016-12-19 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2278 --- 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

[jira] [Commented] (KAFKA-4526) Transient failure in ThrottlingTest.test_throttled_reassignment

2016-12-19 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4526?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15762663#comment-15762663 ] ASF GitHub Bot commented on KAFKA-4526: --- Github user asfgit closed the pull request at:

[jira] [Resolved] (KAFKA-4526) Transient failure in ThrottlingTest.test_throttled_reassignment

2016-12-19 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4526?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma resolved KAFKA-4526. Resolution: Fixed Issue resolved by pull request 2278 [https://github.com/apache/kafka/pull/2278]

[jira] [Updated] (KAFKA-4166) TestMirrorMakerService.test_bounce transient system test failure

2016-12-19 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4166?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-4166: --- Status: Patch Available (was: Open) > TestMirrorMakerService.test_bounce transient system test

[jira] [Commented] (KAFKA-4166) TestMirrorMakerService.test_bounce transient system test failure

2016-12-19 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4166?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15762654#comment-15762654 ] ASF GitHub Bot commented on KAFKA-4166: --- Github user asfgit closed the pull request at:

[jira] [Updated] (KAFKA-4166) TestMirrorMakerService.test_bounce transient system test failure

2016-12-19 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4166?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-4166: --- Fix Version/s: 0.10.2.0 > TestMirrorMakerService.test_bounce transient system test failure >

[GitHub] kafka pull request #2279: KAFKA-4166: Fix transient MM failure caused by slo...

2016-12-19 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2279 --- 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

Jenkins build is back to normal : kafka-trunk-jdk7 #1765

2016-12-19 Thread Apache Jenkins Server
See

[jira] [Created] (KAFKA-4559) Add a site search bar on the Web site

2016-12-19 Thread Guozhang Wang (JIRA)
Guozhang Wang created KAFKA-4559: Summary: Add a site search bar on the Web site Key: KAFKA-4559 URL: https://issues.apache.org/jira/browse/KAFKA-4559 Project: Kafka Issue Type: Bug

[jira] [Assigned] (KAFKA-3808) Transient failure in ReplicaVerificationToolTest

2016-12-19 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3808?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma reassigned KAFKA-3808: -- Assignee: Ismael Juma > Transient failure in ReplicaVerificationToolTest >

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

2016-12-19 Thread Guozhang Wang
One more thing about the double journal proposal: when discussing about this method back at LinkedIn, another raised issue besides double writing was that it will void the offset ordering and enforce people to accept "transaction ordering", that is, consumer will not see messages from the same

[jira] [Created] (KAFKA-4558) throttling_test fails if the producer starts too fast.

2016-12-19 Thread Apurva Mehta (JIRA)
Apurva Mehta created KAFKA-4558: --- Summary: throttling_test fails if the producer starts too fast. Key: KAFKA-4558 URL: https://issues.apache.org/jira/browse/KAFKA-4558 Project: Kafka Issue

Re: [VOTE] KIP-87 - Add Compaction Tombstone Flag

2016-12-19 Thread Jay Kreps
Makes sense! -Jay On Mon, Dec 19, 2016 at 2:40 PM, Michael Pearce wrote: > Wow just read that def over tired. Hopefully it makes sense. Or you get > the gist at least. > > > From: Michael Pearce > Sent:

Build failed in Jenkins: kafka-trunk-jdk8 #1112

2016-12-19 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-4534: StreamPartitionAssignor only ever updates the [jason] KAFKA-4553; Improve round robin assignment in Connect to avoid uneven -- [...truncated 17463 lines...]

[jira] [Commented] (KAFKA-4166) TestMirrorMakerService.test_bounce transient system test failure

2016-12-19 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4166?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15762540#comment-15762540 ] ASF GitHub Bot commented on KAFKA-4166: --- GitHub user hachikuji opened a pull request:

Re: [VOTE] 0.10.1.1 RC1

2016-12-19 Thread Jun Rao
Hi, Guozhang, Thanks for preparing the release. Verified quickstart. +1 Jun On Thu, Dec 15, 2016 at 1:29 PM, Guozhang Wang wrote: > Hello Kafka users, developers and client-developers, > > This is the second, and hopefully the last candidate for the release of > Apache

[GitHub] kafka pull request #2279: KAFKA-4166: Fix transient MM failure caused by slo...

2016-12-19 Thread hachikuji
GitHub user hachikuji opened a pull request: https://github.com/apache/kafka/pull/2279 KAFKA-4166: Fix transient MM failure caused by slow old consumer shutdown You can merge this pull request into a Git repository by running: $ git pull https://github.com/hachikuji/kafka

Re: [VOTE] KIP-87 - Add Compaction Tombstone Flag

2016-12-19 Thread Michael Pearce
Wow just read that def over tired. Hopefully it makes sense. Or you get the gist at least. From: Michael Pearce Sent: Monday, December 19, 2016 9:19:02 PM To: dev@kafka.apache.org Subject: Re: [VOTE] KIP-87 - Add Compaction

[jira] [Commented] (KAFKA-4166) TestMirrorMakerService.test_bounce transient system test failure

2016-12-19 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4166?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15762516#comment-15762516 ] Jason Gustafson commented on KAFKA-4166: Seems like MM is taking a little too long to stop when

Re: [VOTE] KIP-100 - Relax Type constraints in Kafka Streams API

2016-12-19 Thread Guozhang Wang
+1. On Sat, Dec 17, 2016 at 3:27 AM, Ismael Juma wrote: > Thanks Xavier. +1 (binding) > > Ismael > > On Fri, Dec 16, 2016 at 8:15 PM, Xavier Léauté > wrote: > > > Ismael made a good point so I updated KIP-100 and expanded its scope to > > include

Re: [DISCUSS] KIP-102 - Add close with timeout for consumers

2016-12-19 Thread Guozhang Wang
+1 on this idea as well. Streams has also added a similar feature itself partly because consumer does not support it directly (other part of the reason is that like brokers, streams also have some exception handling logic which could lead to deadlock with careless System.exit). For consumer

[jira] [Commented] (KAFKA-4526) Transient failure in ThrottlingTest.test_throttled_reassignment

2016-12-19 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4526?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15762495#comment-15762495 ] ASF GitHub Bot commented on KAFKA-4526: --- GitHub user apurvam opened a pull request:

[GitHub] kafka pull request #2278: KAFKA-4526 - Disable throttling test until it can ...

2016-12-19 Thread apurvam
GitHub user apurvam opened a pull request: https://github.com/apache/kafka/pull/2278 KAFKA-4526 - Disable throttling test until it can be fixed correctly. At present, the test is fragile in the sense that the console consumer has to start and be initialized before the verifiable

Build failed in Jenkins: kafka-trunk-jdk7 #1764

2016-12-19 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-4534: StreamPartitionAssignor only ever updates the -- [...truncated 17439 lines...] org.apache.kafka.streams.KafkaStreamsTest >

Re: [VOTE] 0.10.1.1 RC1

2016-12-19 Thread Gwen Shapira
+1 (binding) Validated signatures Ran tests Built from source distro Tested binaries using the quickstart guide Gwen On Thu, Dec 15, 2016 at 1:29 PM, Guozhang Wang wrote: > Hello Kafka users, developers and client-developers, > > This is the second, and hopefully the last

Re: [DISCUSS] KIP-95: Incremental Batch Processing for Kafka Streams

2016-12-19 Thread Guozhang Wang
Avi, I have granted you the permissions under apache id (avi). Guozhang On Thu, Dec 15, 2016 at 3:40 PM, Matthias J. Sax wrote: > What is you wiki ID? We can grant you permission. > > -Matthias > > On 12/15/16 3:27 PM, Avi Flax wrote: > > > >> On Dec 13, 2016, at 21:02,

[GitHub] kafka pull request #2277: KAFKA-4527: task status was being updated before a...

2016-12-19 Thread shikhar
GitHub user shikhar opened a pull request: https://github.com/apache/kafka/pull/2277 KAFKA-4527: task status was being updated before actual pause/resume h/t @ewencp for pointing out the issue You can merge this pull request into a Git repository by running: $ git pull

[jira] [Commented] (KAFKA-4527) Transient failure of ConnectDistributedTest.test_pause_and_resume_sink where paused connector produces messages

2016-12-19 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4527?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15762372#comment-15762372 ] ASF GitHub Bot commented on KAFKA-4527: --- GitHub user shikhar opened a pull request:

[GitHub] kafka pull request #2276: MINOR: Add more exception information in Processor...

2016-12-19 Thread guozhangwang
GitHub user guozhangwang opened a pull request: https://github.com/apache/kafka/pull/2276 MINOR: Add more exception information in ProcessorStateManager You can merge this pull request into a Git repository by running: $ git pull https://github.com/guozhangwang/kafka

[jira] [Commented] (KAFKA-4526) Transient failure in ThrottlingTest.test_throttled_reassignment

2016-12-19 Thread Apurva Mehta (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4526?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15762361#comment-15762361 ] Apurva Mehta commented on KAFKA-4526: - I had a look at the logs from one of the failures, and here is

Re: [DISCUSS] KIP 88: OffsetFetch Protocol Update

2016-12-19 Thread Vahid S Hashemian
Happy Monday, Jason, thanks for further explaining the issue. I have updated the KIP and reflected the recent discussions in there: https://cwiki.apache.org/confluence/display/KAFKA/KIP-88%3A+OffsetFetch+Protocol+Update You can also see the modifications to the KIP compared to the approved

Re: [VOTE] KIP-87 - Add Compaction Tombstone Flag

2016-12-19 Thread Michael Pearce
Hi Jay, Agreed this stemmed as offshoot from KIP-82. Which our main driver for was to be able to have some headers for a null value as such for our routing, audit, tracing and a few other bits which currently we are forced to do with a message wrapper, if we all agreed on KIP-82 that we need

Re: [VOTE] KIP-87 - Add Compaction Tombstone Flag

2016-12-19 Thread Jay Kreps
Hey Michael, Here is the compatibility concern I have: 1. You have a consumer app that relies on value == null to indicate a delete (current semantics). 2. You upgrade Kafka and your clients. 3. Some producer starts using the tombstone field in combination with non-null. I share

[jira] [Updated] (KAFKA-4553) Connect's round robin assignment produces undesirable distribution of connectors/tasks

2016-12-19 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4553?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson updated KAFKA-4553: --- Resolution: Fixed Fix Version/s: 0.10.2.0 Status: Resolved (was: Patch

[jira] [Commented] (KAFKA-4553) Connect's round robin assignment produces undesirable distribution of connectors/tasks

2016-12-19 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4553?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15762247#comment-15762247 ] ASF GitHub Bot commented on KAFKA-4553: --- Github user asfgit closed the pull request at:

[GitHub] kafka pull request #2272: KAFKA-4553: Improve round robin assignment in Conn...

2016-12-19 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2272 --- 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

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

2016-12-19 Thread Sriram Subramanian
small correction in my third point - 3. IO and memory constraints - We would want a solution that *does not take* 2x the number of writes. On Mon, Dec 19, 2016 at 12:37 PM, Sriram Subramanian wrote: > Radai, > > I think it is important to understand the key requirements that

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

2016-12-19 Thread Sriram Subramanian
Radai, I think it is important to understand the key requirements that we don’t want to compromise. We can then understand the tradeoffs of the different approaches. We did in fact start with the double journal approach couple of years back. I will highlight the must have requirements first and

[jira] [Commented] (KAFKA-4534) StreamPartitionAssignor only ever updates the partitionsByHostState and metadataWithInternalTopics once.

2016-12-19 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4534?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15762205#comment-15762205 ] ASF GitHub Bot commented on KAFKA-4534: --- Github user asfgit closed the pull request at:

[GitHub] kafka pull request #2256: KAFKA-4534: StreamPartitionAssignor only ever upda...

2016-12-19 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2256 --- 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

[jira] [Updated] (KAFKA-4534) StreamPartitionAssignor only ever updates the partitionsByHostState and metadataWithInternalTopics once.

2016-12-19 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4534?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-4534: - Resolution: Fixed Status: Resolved (was: Patch Available) Issue resolved by pull request

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

2016-12-19 Thread Jay Kreps
Hey Radai, I'm not sure if I fully understand what you are proposing, but I interpreted it to be similar to a proposal we worked through back at LinkedIn. The proposal was to commit to a central txlog topic, and then recopy to the destination topic upon transaction commit. The observation on that

Re: Kafka controlled shutdown hangs when there are large number of topics in the cluster

2016-12-19 Thread Gwen Shapira
Can you try setting auto.leader.rebalance.enable=false in your configuration (for all brokers) and see if it solves this problem? We've had some reports regarding this feature interfering with controlled shutdown. On Mon, Dec 19, 2016 at 5:02 AM, Robin, Martin (Nokia - IN/Bangalore)

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

2016-12-19 Thread radai
regarding efficiency: I'd like to distinguish between server efficiency (resource utilization of the broker machine alone) and overall network efficiency (resource utilization on brokers, producers and consumers, including network traffic). my proposal is not as resource-efficient on the broker

Re: [DISCUSS] KIP-102 - Add close with timeout for consumers

2016-12-19 Thread Rajini Sivaram
Thank you for the reviews. @Becket @Ewen, Agree that making all blocking calls have a timeout will be trickier and hence the scope of this KIP is limited to close(). @Jay Yes, this should definitely go into release notes, will make sure it is added. I will add some integration tests with broker

Kafka controlled shutdown hangs when there are large number of topics in the cluster

2016-12-19 Thread Robin, Martin (Nokia - IN/Bangalore)
Hi We have 9 broker instances in a kafka cluster spread across 3 linux machines. The 1st machine has 4 broker instances. 2nd machine has 4 broker instances and 3rd one has 1 broker instance. There are around 101 topics created in the cluster We start the broker as follows All 4

[jira] [Commented] (KAFKA-4557) ConcurrentModificationException in KafkaProducer event loop

2016-12-19 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4557?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15761434#comment-15761434 ] Ismael Juma commented on KAFKA-4557: Thanks for the report. Not clear how this is happening since we

[jira] [Updated] (KAFKA-4557) ConcurrentModificationException in KafkaProducer event loop

2016-12-19 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4557?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-4557: --- Fix Version/s: 0.10.2.0 > ConcurrentModificationException in KafkaProducer event loop >

[jira] [Updated] (KAFKA-4557) ConcurrentModificationException in KafkaProducer event loop

2016-12-19 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4557?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-4557: --- Labels: reliability (was: ) > ConcurrentModificationException in KafkaProducer event loop >

[jira] [Updated] (KAFKA-4557) ConcurrentModificationException in KafkaProducer event loop

2016-12-19 Thread Sergey Alaev (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4557?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sergey Alaev updated KAFKA-4557: Description: Under heavy load, Kafka producer can stop publishing events. Logs below.

[jira] [Created] (KAFKA-4557) ConcurrentModificationException in KafkaProducer event loop

2016-12-19 Thread Sergey Alaev (JIRA)
Sergey Alaev created KAFKA-4557: --- Summary: ConcurrentModificationException in KafkaProducer event loop Key: KAFKA-4557 URL: https://issues.apache.org/jira/browse/KAFKA-4557 Project: Kafka

[jira] [Updated] (KAFKA-4540) Suspended tasks that are not assigned to the StreamThread need to be closed before new active and standby tasks are created

2016-12-19 Thread Damian Guy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4540?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Damian Guy updated KAFKA-4540: -- Status: Patch Available (was: Open) > Suspended tasks that are not assigned to the StreamThread need

[GitHub] kafka pull request #2274: Implement topic config for internal topics

2016-12-19 Thread sjmittal
Github user sjmittal closed the pull request at: https://github.com/apache/kafka/pull/2274 --- 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

[jira] [Updated] (KAFKA-4556) unordered messages when multiple topics are combined in single topic through stream

2016-12-19 Thread Savdeep Singh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4556?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Savdeep Singh updated KAFKA-4556: - Affects Version/s: 0.10.0.1 Component/s: streams producer

[jira] [Updated] (KAFKA-4556) unordered messages when multiple topics are combined in single topic through stream

2016-12-19 Thread Savdeep Singh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4556?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Savdeep Singh updated KAFKA-4556: - Description: When binding builder with multiple topics, single resultant topic has unordered set

[jira] [Updated] (KAFKA-4556) unordered messages when multiple topics are combined in single topic through stream

2016-12-19 Thread Savdeep Singh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4556?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Savdeep Singh updated KAFKA-4556: - Description: When binding builder with multiple topics, single resultant topic has unordered set

[jira] [Updated] (KAFKA-4556) unordered messages when multiple topics are combined in single topic through stream

2016-12-19 Thread Savdeep Singh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4556?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Savdeep Singh updated KAFKA-4556: - Description: When binding builder with multiple topics, single resultant topic has unordered set

[jira] [Updated] (KAFKA-4556) unordered messages when multiple topics are combined in single topic through stream

2016-12-19 Thread Savdeep Singh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4556?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Savdeep Singh updated KAFKA-4556: - Description: When binding builder with multiple topics, single resultant topic has unordered set

[GitHub] kafka pull request #2275: Fix exception handling in case of file record trun...

2016-12-19 Thread kamilszymanski
GitHub user kamilszymanski opened a pull request: https://github.com/apache/kafka/pull/2275 Fix exception handling in case of file record truncation during write In case of file record truncation during write due to improper types usage (`AtomicInteger` in place of `int`)

[jira] [Created] (KAFKA-4556) unordered messages when multiple topics are combined in single topic through stream

2016-12-19 Thread Savdeep Singh (JIRA)
Savdeep Singh created KAFKA-4556: Summary: unordered messages when multiple topics are combined in single topic through stream Key: KAFKA-4556 URL: https://issues.apache.org/jira/browse/KAFKA-4556