[jira] [Updated] (KAFKA-5165) Kafka Logs Cleanup Not happening, Huge File Growth - Windows

2017-05-03 Thread Manikandan P (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5165?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikandan P updated KAFKA-5165: Affects Version/s: 0.9.0.1 > Kafka Logs Cleanup Not happening, Huge File Growth - Windows >

Re: [DISCUSS] KIP 141 - ProducerRecordBuilder Interface

2017-05-03 Thread Ewen Cheslack-Postava
Stephane, VOTES are really on-demand based on the author, but obviously it's good to come to some level of consensus in the DISCUSS thread before initiating a vote. I think the request for comments/votes on your 3 options is a reasonable way to gauge current opinions. For myself, I think either 1

[jira] [Commented] (KAFKA-4923) Add Exactly-Once Semantics to Streams

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

[GitHub] kafka pull request #2974: KAFKA-4923: Add Exaclty-Once Semantics to Streams ...

2017-05-03 Thread mjsax
GitHub user mjsax opened a pull request: https://github.com/apache/kafka/pull/2974 KAFKA-4923: Add Exaclty-Once Semantics to Streams (testing) - add broker compatibility system tests You can merge this pull request into a Git repository by running: $ git pull https://github.co

[jira] [Commented] (KAFKA-5171) TC should not accept empty string transactional id

2017-05-03 Thread Umesh Chaudhary (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5171?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15996180#comment-15996180 ] Umesh Chaudhary commented on KAFKA-5171: [~guozhang] , Please review the PR. > T

[GitHub] kafka pull request #2973: Kafka 5171:TC should not accept empty string trans...

2017-05-03 Thread umesh9794
GitHub user umesh9794 opened a pull request: https://github.com/apache/kafka/pull/2973 Kafka 5171:TC should not accept empty string transactional id This is an initial PR. Changed the unit tests accordingly as per the expectation from TC. You can merge this pull request into a Git

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

2017-05-03 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-5045: Clarify on KTable APIs for queryable stores -- [...truncated 2.12 MB...]

Re: Exiting a streams app at end of stream?

2017-05-03 Thread Matthias J. Sax
Hi, there is KIP for this idea, but nobody or working on it actively atm: https://cwiki.apache.org/confluence/display/KAFKA/KIP-95%3A+Incremental+Batch+Processing+for+Kafka+Streams Right now, you can only check the committed offsets and terminate the app accordingly. -Matthias On 5/3/17 5:32

[jira] [Commented] (KAFKA-5172) CachingSessionStore doesn't fetchPrevious correctly.

2017-05-03 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5172?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15996038#comment-15996038 ] ASF GitHub Bot commented on KAFKA-5172: --- GitHub user KyleWinkelman opened a pull req

[GitHub] kafka pull request #2972: [KAFKA-5172] Fix fetchPrevious to find the correct...

2017-05-03 Thread KyleWinkelman
GitHub user KyleWinkelman opened a pull request: https://github.com/apache/kafka/pull/2972 [KAFKA-5172] Fix fetchPrevious to find the correct session. Change fetchPrevious to use findSessions with the proper key and timestamps rather than using fetch. You can merge this pull reques

[jira] [Created] (KAFKA-5172) CachingSessionStore doesn't fetchPrevious correctly.

2017-05-03 Thread Kyle Winkelman (JIRA)
Kyle Winkelman created KAFKA-5172: - Summary: CachingSessionStore doesn't fetchPrevious correctly. Key: KAFKA-5172 URL: https://issues.apache.org/jira/browse/KAFKA-5172 Project: Kafka Issue Ty

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

2017-05-03 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-5045: Clarify on KTable APIs for queryable stores -- [...truncated 1.65 MB...] kafka.integration.SaslPlaintextTopicMetadataTest > testAlive

[GitHub] kafka pull request #2971: [Minor] Remove use of aggSerde in RocksDBSessionSt...

2017-05-03 Thread KyleWinkelman
GitHub user KyleWinkelman opened a pull request: https://github.com/apache/kafka/pull/2971 [Minor] Remove use of aggSerde in RocksDBSessionStore. RocksDBSessionStore wasn't properly using the default aggSerde if no Serde was supplied. You can merge this pull request into a Git repo

[GitHub] kafka pull request #2970: Kafka-5160; KIP-98 Broker side support for TxnOffs...

2017-05-03 Thread apurvam
GitHub user apurvam opened a pull request: https://github.com/apache/kafka/pull/2970 Kafka-5160; KIP-98 Broker side support for TxnOffsetCommitRequest This patch adds support for the `TxnOffsetCommitRequest` added in KIP-98. Desired handling for this request is [described here](htt

[jira] [Created] (KAFKA-5171) TC should not accept empty string transactional id

2017-05-03 Thread Guozhang Wang (JIRA)
Guozhang Wang created KAFKA-5171: Summary: TC should not accept empty string transactional id Key: KAFKA-5171 URL: https://issues.apache.org/jira/browse/KAFKA-5171 Project: Kafka Issue Type:

Re: [DISCUSS] KIP-146: Classloading Isolation in Connect

2017-05-03 Thread Stephane Maarek
Glad you find the feedback useful ! Definitely all the ideas should be split in reasonable length KIPs. I just want to make sure the ideas are not lost. I won’t create the subsequent KIPs because I’m not good enough to implement the changes, but happy to keep on providing feedback alongside the

Re: [DISCUSS] KIP 141 - ProducerRecordBuilder Interface

2017-05-03 Thread Stephane Maarek
How do votes works? I feel there are 3 options right here, and I’d like a pre vote before a real vote? 1) Adding constructors. Could get messy over time, especially with headers coming into play, and future possible improvement to the message format 2) Adding a builder / nicer looking API (like

[GitHub] kafka pull request #2969: doc typo

2017-05-03 Thread smferguson
GitHub user smferguson opened a pull request: https://github.com/apache/kafka/pull/2969 doc typo You can merge this pull request into a Git repository by running: $ git pull https://github.com/smferguson/kafka doc_typo Alternatively you can review and apply these changes as t

[jira] [Created] (KAFKA-5170) KafkaAdminClientIntegration test should wait until metadata is propagated to all brokers

2017-05-03 Thread Colin P. McCabe (JIRA)
Colin P. McCabe created KAFKA-5170: -- Summary: KafkaAdminClientIntegration test should wait until metadata is propagated to all brokers Key: KAFKA-5170 URL: https://issues.apache.org/jira/browse/KAFKA-5170

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

2017-05-03 Thread Apache Jenkins Server
See Changes: [wangguoz] MINOR: Fix error logged if not enough alive brokers for transactions [wangguoz] KAFKA-5144: renamed variables in MinTimestampTracker and added comments [wangguoz] KAFKA-5055: Fix Kafka

[jira] [Updated] (KAFKA-5150) LZ4 decompression is 4-5x slower than Snappy on small batches / messages

2017-05-03 Thread JIRA
[ https://issues.apache.org/jira/browse/KAFKA-5150?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Xavier Léauté updated KAFKA-5150: - Status: Patch Available (was: Open) > LZ4 decompression is 4-5x slower than Snappy on small batch

[jira] [Commented] (KAFKA-5045) KTable materialization and improved semantics

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

[GitHub] kafka pull request #2832: KAFKA-5045: KTable cleanup

2017-05-03 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2832 --- 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] [Resolved] (KAFKA-5045) KTable materialization and improved semantics

2017-05-03 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5045?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-5045. -- Resolution: Fixed Issue resolved by pull request 2832 [https://github.com/apache/kafka/pull/2832

[jira] [Reopened] (KAFKA-2837) FAILING TEST: kafka.api.ProducerBounceTest > testBrokerFailure

2017-05-03 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2837?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang reopened KAFKA-2837: -- Saw this test failure again: {code} kafka.api.ProducerBounceTest > testBrokerFailure FAILED ja

[jira] [Commented] (KAFKA-4801) Transient test failure (part 2): ConsumerBounceTest.testConsumptionWithBrokerFailures

2017-05-03 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4801?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15995872#comment-15995872 ] Guozhang Wang commented on KAFKA-4801: -- [~hachikuji] Saw this happen again but with a

[jira] [Commented] (KAFKA-4801) Transient test failure (part 2): ConsumerBounceTest.testConsumptionWithBrokerFailures

2017-05-03 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4801?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15995863#comment-15995863 ] Jason Gustafson commented on KAFKA-4801: After my patch was merged, this seems to

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

2017-05-03 Thread Apache Jenkins Server
See Changes: [wangguoz] MINOR: Fix error logged if not enough alive brokers for transactions [wangguoz] KAFKA-5144: renamed variables in MinTimestampTracker and added comments [wangguoz] KAFKA-5055: Fix Kafka

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

2017-05-03 Thread Apache Jenkins Server
See Changes: [rajinisivaram] KAFKA-4703: Test with two SASL_SSL listeners with different JAAS -- [...truncated 1.65 MB...] kafka.integration.SaslPlaintextTopicMetadataTest

[jira] [Commented] (KAFKA-5167) streams task gets stuck after re-balance due to LockException

2017-05-03 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5167?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15995777#comment-15995777 ] Matthias J. Sax commented on KAFKA-5167: [~Narendra Kumar] Thanks for reporting th

[jira] [Commented] (KAFKA-5114) Clarify meaning of logs in Introduction: Topics and Logs

2017-05-03 Thread Michael Ernest (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5114?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15995767#comment-15995767 ] Michael Ernest commented on KAFKA-5114: --- What if a partition were described as a str

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

2017-05-03 Thread Apache Jenkins Server
See Changes: [rajinisivaram] KAFKA-4703: Test with two SASL_SSL listeners with different JAAS -- [...truncated 1.64 MB...] kafka.server.DynamicConfigChangeTest > testDefa

[jira] [Updated] (KAFKA-5150) LZ4 decompression is 4-5x slower than Snappy on small batches / messages

2017-05-03 Thread JIRA
[ https://issues.apache.org/jira/browse/KAFKA-5150?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Xavier Léauté updated KAFKA-5150: - Fix Version/s: 0.11.0.0 > LZ4 decompression is 4-5x slower than Snappy on small batches / messages

[jira] [Commented] (KAFKA-5144) MinTimestampTracker uses confusing variable names

2017-05-03 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5144?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15995746#comment-15995746 ] Eno Thereska commented on KAFKA-5144: - [~mihbor] thanks. I have a PR that cleans up th

[jira] [Commented] (KAFKA-5169) KafkaConsumer.close should be idempotent

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

[jira] [Updated] (KAFKA-5169) KafkaConsumer.close should be idempotent

2017-05-03 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5169?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-5169: --- Fix Version/s: 0.11.0.0 Affects Version/s: 0.10.2.1 0.10.2.0

[GitHub] kafka pull request #2968: KAFKA-5169: KafkaConsumer.close should be idempote...

2017-05-03 Thread mjsax
GitHub user mjsax opened a pull request: https://github.com/apache/kafka/pull/2968 KAFKA-5169: KafkaConsumer.close should be idempotent You can merge this pull request into a Git repository by running: $ git pull https://github.com/mjsax/kafka kafka-5169-consumer-close Altern

[jira] [Created] (KAFKA-5169) KafkaConsumer.close should be idempotent

2017-05-03 Thread Matthias J. Sax (JIRA)
Matthias J. Sax created KAFKA-5169: -- Summary: KafkaConsumer.close should be idempotent Key: KAFKA-5169 URL: https://issues.apache.org/jira/browse/KAFKA-5169 Project: Kafka Issue Type: Bug

[jira] [Commented] (KAFKA-5055) Kafka Streams skipped-records-rate sensor producing nonzero values even when FailOnInvalidTimestamp is used as extractor

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

[GitHub] kafka pull request #2949: KAFKA-5055: Kafka Streams skipped-records-rate sen...

2017-05-03 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2949 --- 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] [Updated] (KAFKA-5055) Kafka Streams skipped-records-rate sensor producing nonzero values even when FailOnInvalidTimestamp is used as extractor

2017-05-03 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5055?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-5055: - Resolution: Fixed Fix Version/s: 0.11.0.0 Status: Resolved (was: Patch Available

[jira] [Commented] (KAFKA-5150) LZ4 decompression is 4-5x slower than Snappy on small batches / messages

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

[GitHub] kafka pull request #2967: KAFKA-5150 reduce lz4 decompression overhead

2017-05-03 Thread xvrl
GitHub user xvrl opened a pull request: https://github.com/apache/kafka/pull/2967 KAFKA-5150 reduce lz4 decompression overhead - reuse decompression buffers, keeping one per thread - switch lz4 input stream to operate directly on ByteBuffers - more tests with both compressibl

[jira] [Commented] (KAFKA-5144) MinTimestampTracker uses confusing variable names

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

[GitHub] kafka pull request #2948: KAFKA-5144 renamed and added comments to make it c...

2017-05-03 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2948 --- 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] [Updated] (KAFKA-5144) MinTimestampTracker uses confusing variable names

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

[jira] [Assigned] (KAFKA-5114) Clarify meaning of logs in Introduction: Topics and Logs

2017-05-03 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5114?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang reassigned KAFKA-5114: Assignee: (was: Michael Ernest) > Clarify meaning of logs in Introduction: Topics and L

[jira] [Assigned] (KAFKA-5114) Clarify meaning of logs in Introduction: Topics and Logs

2017-05-03 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5114?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang reassigned KAFKA-5114: Assignee: Michael Ernest > Clarify meaning of logs in Introduction: Topics and Logs > -

Re: Request to add to the contributor list

2017-05-03 Thread Guozhang Wang
Hi Amit, I have added you to the contributor list and assigned 4996 to you. Cheers. Will also review your PR soon. Guozhang On Tue, May 2, 2017 at 5:43 AM, Amit Daga wrote: > Hello, > > Wondering if you were able to look into it? > > Thanks, > Amit Daga > > On Sun, Apr 30, 2017 at 1:29 PM,

[jira] [Assigned] (KAFKA-4996) Fix findbugs multithreaded correctness warnings for streams

2017-05-03 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4996?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang reassigned KAFKA-4996: Assignee: Amit Daga > Fix findbugs multithreaded correctness warnings for streams > ---

[GitHub] kafka pull request #2954: MINOR: Fix error logged if not enough alive broker...

2017-05-03 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2954 --- 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] [Reopened] (KAFKA-4583) KafkaConsumerTest.testGracefulClose transient failure

2017-05-03 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4583?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang reopened KAFKA-4583: -- Saw this failure again but with a different stack trace: https://builds.apache.org/job/kafka-pr-jdk

[jira] [Commented] (KAFKA-4703) Test with two SASL_SSL listeners with different JAAS contexts

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

[GitHub] kafka pull request #2506: KAFKA-4703 Test with two SASL_SSL listeners with d...

2017-05-03 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2506 --- 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] [Updated] (KAFKA-4703) Test with two SASL_SSL listeners with different JAAS contexts

2017-05-03 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4703?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram updated KAFKA-4703: -- Resolution: Fixed Fix Version/s: 0.11.0.0 Status: Resolved (was: Patch Availab

Re: Topic Creation programatically

2017-05-03 Thread Colin McCabe
Hi Zishan, The best practice will depend on the situation. In most cases, creating the topic using the CLI creation command prior to running your code is easier than creating it programmatically. There will be a new AdminClient API for creating topics programmatically in the 0.11 release; prior

[jira] [Updated] (KAFKA-5168) Cleanup delayed produce purgatory during partition emmigration

2017-05-03 Thread Damian Guy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5168?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Damian Guy updated KAFKA-5168: -- Status: Patch Available (was: Open) > Cleanup delayed produce purgatory during partition emmigration >

[jira] [Commented] (KAFKA-4996) Fix findbugs multithreaded correctness warnings for streams

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

[GitHub] kafka pull request #2966: KAFKA-4996: Fix findbugs multithreaded correctness...

2017-05-03 Thread amitdaga
GitHub user amitdaga opened a pull request: https://github.com/apache/kafka/pull/2966 KAFKA-4996: Fix findbugs multithreaded correctness warnings for streams You can merge this pull request into a Git repository by running: $ git pull https://github.com/amitdaga/kafka findbugs

Re: [DISCUSS] KIP-143: Controller Health Metrics

2017-05-03 Thread Joel Koshy
On Wed, May 3, 2017 at 10:54 AM, Onur Karaman wrote: > Regarding the ControllerState and the potential for overlap, I think it > depends on our definition of controller state. While KAFKA-5028 allows only > a single ControllerEvent to be processed at a time, it still allows > interleavings for lo

[jira] [Commented] (KAFKA-5168) Cleanup delayed produce purgatory during partition emmigration

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

[GitHub] kafka pull request #2965: KAFKA-5168: Cleanup delayed produce purgatory duri...

2017-05-03 Thread dguy
GitHub user dguy opened a pull request: https://github.com/apache/kafka/pull/2965 KAFKA-5168: Cleanup delayed produce purgatory during partition emmigration remove operations from the replica manager's producer purgatory on transaction emmigration You can merge this pull request in

[jira] [Updated] (KAFKA-5059) Implement Transactional Coordinator

2017-05-03 Thread Damian Guy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5059?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Damian Guy updated KAFKA-5059: -- Fix Version/s: 0.11.0.0 > Implement Transactional Coordinator > --- > >

[jira] [Created] (KAFKA-5168) Cleanup delayed produce purgatory during partition emmigration

2017-05-03 Thread Damian Guy (JIRA)
Damian Guy created KAFKA-5168: - Summary: Cleanup delayed produce purgatory during partition emmigration Key: KAFKA-5168 URL: https://issues.apache.org/jira/browse/KAFKA-5168 Project: Kafka Issu

[jira] [Commented] (KAFKA-5154) Kafka Streams throws NPE during rebalance

2017-05-03 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5154?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15995380#comment-15995380 ] Matthias J. Sax commented on KAFKA-5154: Thanks. We tried to reproduce, too. But n

Re: [DISCUSS] KIP-146: Classloading Isolation in Connect

2017-05-03 Thread Konstantine Karantasis
Thank you Stephane, your comments bring interesting and useful subjects to the discussion. I'm adding my replies below Ewen's comments. On Tue, May 2, 2017 at 10:15 PM, Ewen Cheslack-Postava wrote: > On Tue, May 2, 2017 at 10:01 PM, Stephane Maarek < > steph...@simplemachines.com.au> wrote: >

Re: [DISCUSS] KIP-146: Classloading Isolation in Connect

2017-05-03 Thread Konstantine Karantasis
Thanks Ewen. I'm replying inline as well. On Tue, May 2, 2017 at 11:24 AM, Ewen Cheslack-Postava wrote: > Thanks for the KIP. > > A few responses inline, followed by additional comments. > > On Mon, May 1, 2017 at 9:50 PM, Konstantine Karantasis < > konstant...@confluent.io> wrote: > > > Gwen, R

Re: [DISCUSS] KIP-143: Controller Health Metrics

2017-05-03 Thread Onur Karaman
Regarding the ControllerState and the potential for overlap, I think it depends on our definition of controller state. While KAFKA-5028 allows only a single ControllerEvent to be processed at a time, it still allows interleavings for long-lasting actions like partition reassignment and topic deleti

Re: [DISCUSS] KIP-137: Enhance TopicCommand --describe to show topics marked for deletion

2017-05-03 Thread Mickael Maison
Yes it's what I was thinking when writing this up, JSON output would be nice. I'll be happy to have a look at it. I'm guessing that would require another KIP ? On Wed, May 3, 2017 at 8:26 AM, Ismael Juma wrote: > Yeah, structured output for the CLI tools would be great. 3 digit number > JIRA, nic

[jira] [Commented] (KAFKA-5004) poll() timeout not enforced when connecting to 0.10.0 broker

2017-05-03 Thread Colin P. McCabe (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5004?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15995257#comment-15995257 ] Colin P. McCabe commented on KAFKA-5004: Thanks for filing this, [~mjsax]. I thin

[jira] [Updated] (KAFKA-5167) streams task gets stuck after re-balance due to LockException

2017-05-03 Thread Narendra Kumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5167?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Narendra Kumar updated KAFKA-5167: -- Attachment: logs.txt Description: During rebalance processor node's close() method gets cal

[jira] [Created] (KAFKA-5167) streams task gets stuck after re-balance due to LockException

2017-05-03 Thread Narendra Kumar (JIRA)
Narendra Kumar created KAFKA-5167: - Summary: streams task gets stuck after re-balance due to LockException Key: KAFKA-5167 URL: https://issues.apache.org/jira/browse/KAFKA-5167 Project: Kafka

Re: Questions about Development Practices

2017-05-03 Thread Colin McCabe
On Sun, Apr 2, 2017, at 13:28, Daan Rennings wrote: > Dear developers, > > When going over your way of working adopted with Apache Kafka, I was > wondering about the following: > > *1) Visualizing Technical Debt* > Based on my findings (with FindBugs, CheckStyle and JaCoCo), I concluded > that Ka

[jira] [Created] (KAFKA-5166) Add option "dry run" to Streams application reset tool

2017-05-03 Thread Matthias J. Sax (JIRA)
Matthias J. Sax created KAFKA-5166: -- Summary: Add option "dry run" to Streams application reset tool Key: KAFKA-5166 URL: https://issues.apache.org/jira/browse/KAFKA-5166 Project: Kafka Issu

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

2017-05-03 Thread Apache Jenkins Server
See

[jira] [Commented] (KAFKA-4985) kafka-acls should resolve dns names and accept ip ranges

2017-05-03 Thread Colin P. McCabe (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4985?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15995159#comment-15995159 ] Colin P. McCabe commented on KAFKA-4985: Hmm. The problem with resolving hostname

Re: [DISCUSS] KIP-143: Controller Health Metrics

2017-05-03 Thread Ismael Juma
Thanks for the feedback Tom, Joel and Jun. I updated the KIP in the following way: 1. Removed ControlledShutdownRateAndTimeMs 2. Added QueueSize and QueueTimeMs 3. Renamed FailedIsrUpdateRate to FailedIsrUpdatesPerSec for consistency with other metrics in the Partition class 4. Mentioned that Yam

[jira] [Commented] (KAFKA-5155) Messages can be deleted prematurely when some producers use timestamps and some not

2017-05-03 Thread JIRA
[ https://issues.apache.org/jira/browse/KAFKA-5155?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15994831#comment-15994831 ] Petr Plavjaník commented on KAFKA-5155: --- Hi [~huxi_2b] and [~mihbor], this defect i

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

2017-05-03 Thread Apache Jenkins Server
See Changes: [ismael] KAFA-4378: Fix Scala 2.12 "eta-expansion of zero-argument method" -- [...truncated 3.51 MB...] kafka.utils.CommandLineUtilsTest > testParseSingleArg

Exiting a streams app at end of stream?

2017-05-03 Thread Thomas Becker
We have had a number of situations where we need to migrate data in a Kafka topic to a new topic that is keyed differently. Stream processing is a good fit for this use-case with one exception: there is no easy way to know when your "migration job" is finished. Has any thought been given to adding

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

2017-05-03 Thread Apache Jenkins Server
See Changes: [ismael] KAFKA-3754; Add GC log retention policy to limit size of log -- [...truncated 837.81 KB...] kafka.coordinator.transaction.TransactionCoordinatorTest

[jira] [Commented] (KAFKA-5154) Kafka Streams throws NPE during rebalance

2017-05-03 Thread Lukas Gemela (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5154?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15994646#comment-15994646 ] Lukas Gemela commented on KAFKA-5154: - Ok I've made it running with kafka streams tag

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

2017-05-03 Thread Apache Jenkins Server
See Changes: [ismael] KAFA-4378: Fix Scala 2.12 "eta-expansion of zero-argument method" -- [...truncated 841.97 KB...] kafka.coordinator.transaction.TransactionCoordinator

[jira] [Created] (KAFKA-5165) Kafka Logs Cleanup Not happening, Huge File Growth - Windows

2017-05-03 Thread Manikandan P (JIRA)
Manikandan P created KAFKA-5165: --- Summary: Kafka Logs Cleanup Not happening, Huge File Growth - Windows Key: KAFKA-5165 URL: https://issues.apache.org/jira/browse/KAFKA-5165 Project: Kafka Iss

[jira] [Commented] (KAFKA-3754) Kafka default -Xloggc settings should include GC log rotation flags

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

[GitHub] kafka pull request #1431: KAFKA-3754 Add GC log retention policy to limit si...

2017-05-03 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/1431 --- 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] [Resolved] (KAFKA-3754) Kafka default -Xloggc settings should include GC log rotation flags

2017-05-03 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3754?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma resolved KAFKA-3754. Resolution: Fixed Fix Version/s: 0.11.0.0 Issue resolved by pull request 1431 [https://github

[jira] [Updated] (KAFKA-3754) Kafka default -Xloggc settings should include GC log rotation flags

2017-05-03 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3754?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-3754: --- Description: By default kafka-run-class.sh defines it's GC settings like so: KAFKA_GC_LOG_OPTS="-Xlog

[jira] [Commented] (KAFKA-5155) Messages can be deleted prematurely when some producers use timestamps and some not

2017-05-03 Thread Michal Borowiecki (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5155?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15994575#comment-15994575 ] Michal Borowiecki commented on KAFKA-5155: -- Hi @huxi, Personally, I feel the simi

[jira] [Comment Edited] (KAFKA-5155) Messages can be deleted prematurely when some producers use timestamps and some not

2017-05-03 Thread Michal Borowiecki (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5155?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15994575#comment-15994575 ] Michal Borowiecki edited comment on KAFKA-5155 at 5/3/17 9:44 AM: --

[jira] [Commented] (KAFKA-4421) Update release process so that Scala 2.12 artifacts are published

2017-05-03 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4421?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15994544#comment-15994544 ] Ismael Juma commented on KAFKA-4421: The release script that Ewen wrote handles this a

[jira] [Resolved] (KAFKA-4378) Address 2.12 eta-expansion warnings

2017-05-03 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4378?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma resolved KAFKA-4378. Resolution: Fixed Fix Version/s: 0.11.0.0 > Address 2.12 eta-expansion warnings > ---

[GitHub] kafka pull request #2098: KAFA-4378: resolve eta-expansion of zero-argument ...

2017-05-03 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2098 --- 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

Re: [DISCUSS] KIP 145 - Expose Record Headers in Kafka Connect

2017-05-03 Thread Michael Pearce
Hi Ewen, As code I think helps, as I don’t think I explained what I meant very well. I have pushed what I was thinking to the branch/pr. https://github.com/apache/kafka/pull/2942 The key bits added on top here are: new ConnectHeader that holds the header key (as string) and then header value ob

Re: [DISCUSS]: KIP-149: Enabling key access in ValueTransformer, ValueMapper, and ValueJoiner

2017-05-03 Thread Jeyhun Karimov
Hi Mathieu, Thanks for feedback. I followed similar approach and updated PR and KIP accordingly. I tried to guard the key in Processors sending a copy of an actual key. Because I am doing deep copy of an object, I think memory can be bottleneck in some use-cases. Cheers, Jeyhun On Tue, May 2, 20

[jira] [Commented] (KAFKA-5154) Kafka Streams throws NPE during rebalance

2017-05-03 Thread Lukas Gemela (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5154?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15994475#comment-15994475 ] Lukas Gemela commented on KAFKA-5154: - [~mjsax] should not be a problem, I can make it

RE: Kafka Connect and Partitions

2017-05-03 Thread david.franklin
Hi Randall, Many thanks for your and Gwen's help with this - it's very reassuring that help is at hand in such circumstances :) All the best, David -Original Message- From: Randall Hauch [mailto:rha...@gmail.com] Sent: 02 May 2017 21:01 To: dev@kafka.apache.org Subject: Re: Kafka Connect

Re: [DISCUSS] KIP-137: Enhance TopicCommand --describe to show topics marked for deletion

2017-05-03 Thread Ismael Juma
Yeah, structured output for the CLI tools would be great. 3 digit number JIRA, nice. :) Ismael On Wed, May 3, 2017 at 7:05 AM, Ewen Cheslack-Postava wrote: > Since everything is whitespace delimited anyway, I don't think we should > worry about the compatibility issue. We don't guarantee this u

Re: [VOTE] KIP-86: Configurable SASL callback handlers

2017-05-03 Thread Rajini Sivaram
Can we have some more reviews or votes for this KIP to include in 0.11.0.0? It is not a breaking change and the code is ready for integration, so it will be good to get it in if possible. Ismael/Jun, since you had reviewed the KIP earlier, can you let me know if I can do anything more to get your

Re: [VOTE] KIP-137: Enhance TopicCommand --describe to show topics marked for deletion

2017-05-03 Thread Rajini Sivaram
Thanks for the KIP, Mickael. +1 (binding) Regards, Rajini On Wed, May 3, 2017 at 7:06 AM, Ewen Cheslack-Postava wrote: > +1 (binding) > > Thanks for helping improve the CLI tools! > > -Ewen > > On Tue, May 2, 2017 at 8:25 AM, Vahid S Hashemian < > vahidhashem...@us.ibm.com > > wrote: > > > +1

  1   2   >