[jira] [Commented] (KAFKA-4576) Log segments close to max size break on fetch

2017-01-03 Thread Onur Karaman (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4576?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15794567#comment-15794567 ] Onur Karaman commented on KAFKA-4576: - I think you're right, [~huxi_2b]. It looks like

Re: [DISCUSS] KIP-99: Add Global Tables to Kafka Streams

2017-01-03 Thread Damian Guy
Thanks Gouzhang - i'll remove the joins. I agree we need to refactor TopologyBuilder, but I think we'll need another KIP for that. Thanks, Damian On Fri, 30 Dec 2016 at 01:32 Guozhang Wang wrote: > 1/2: Sounds good, let's remove the joins within KGlobalTable for now. > > 3. I see, makes sense.

[GitHub] kafka pull request #2297: MINOR: Code refactoring in scala classes.

2017-01-03 Thread himani1
GitHub user himani1 opened a pull request: https://github.com/apache/kafka/pull/2297 MINOR: Code refactoring in scala classes. You can merge this pull request into a Git repository by running: $ git pull https://github.com/himani1/kafka refactored_code Alternatively you can r

[jira] [Created] (KAFKA-4579) Schema Registry client should have a proper caching mechanism for schema IDs

2017-01-03 Thread Justin Manchester (JIRA)
Justin Manchester created KAFKA-4579: Summary: Schema Registry client should have a proper caching mechanism for schema IDs Key: KAFKA-4579 URL: https://issues.apache.org/jira/browse/KAFKA-4579 Pr

[jira] [Updated] (KAFKA-4576) Log segments close to max size break on fetch

2017-01-03 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4576?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-4576: --- Fix Version/s: 0.10.2.0 > Log segments close to max size break on fetch >

[jira] [Commented] (KAFKA-4576) Log segments close to max size break on fetch

2017-01-03 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4576?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15794645#comment-15794645 ] Ismael Juma commented on KAFKA-4576: Great catch [~huxi_2b]. The code in trunk has cha

[jira] [Updated] (KAFKA-4576) Log segments close to max size break on fetch

2017-01-03 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4576?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-4576: --- Assignee: huxi > Log segments close to max size break on fetch > -

[jira] [Updated] (KAFKA-4576) Log segments close to max size break on fetch

2017-01-03 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4576?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-4576: --- Priority: Critical (was: Major) > Log segments close to max size break on fetch > ---

[jira] [Created] (KAFKA-4580) Change one SASL system test to use new JAAS config property

2017-01-03 Thread Rajini Sivaram (JIRA)
Rajini Sivaram created KAFKA-4580: - Summary: Change one SASL system test to use new JAAS config property Key: KAFKA-4580 URL: https://issues.apache.org/jira/browse/KAFKA-4580 Project: Kafka

[jira] [Created] (KAFKA-4581) Fail gracefully if multiple login modules are specified in sasl.jaas.config

2017-01-03 Thread Rajini Sivaram (JIRA)
Rajini Sivaram created KAFKA-4581: - Summary: Fail gracefully if multiple login modules are specified in sasl.jaas.config Key: KAFKA-4581 URL: https://issues.apache.org/jira/browse/KAFKA-4581 Project:

[jira] [Assigned] (KAFKA-4528) Failure in kafka.producer.ProducerTest.testAsyncSendCanCorrectlyFailWithTimeout

2017-01-03 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4528?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma reassigned KAFKA-4528: -- Assignee: Ismael Juma > Failure in > kafka.producer.ProducerTest.testAsyncSendCanCorrectlyFail

[jira] [Updated] (KAFKA-4528) Failure in kafka.producer.ProducerTest.testAsyncSendCanCorrectlyFailWithTimeout

2017-01-03 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4528?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-4528: --- Fix Version/s: 0.10.2.0 > Failure in > kafka.producer.ProducerTest.testAsyncSendCanCorrectlyFailWithT

[GitHub] kafka pull request #2298: KAFKA-4528: Fix failure in ProducerTest.testAsyncS...

2017-01-03 Thread ijuma
GitHub user ijuma opened a pull request: https://github.com/apache/kafka/pull/2298 KAFKA-4528: Fix failure in ProducerTest.testAsyncSendCanCorrectlyFailWithTimeout I was able to reproduce the failure in less than 10 runs before the change. With the change, the test passed 70 ti

[jira] [Commented] (KAFKA-4528) Failure in kafka.producer.ProducerTest.testAsyncSendCanCorrectlyFailWithTimeout

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

[GitHub] kafka pull request #2072: Kafka-4351: Topic regex behavioral change with Mir...

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

[GitHub] kafka pull request #2297: MINOR: Code refactoring in scala classes.

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

[GitHub] kafka pull request #2218: KAFKA-4480: Report an error in 'kafka-configs' com...

2017-01-03 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2218 --- 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-4480) kafka-configs will execute the removal of an invalid property and not report an error

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

Re: kafka.log:type=LogFlushStats,name=LogFlushRateAndTimeMs Not Reported

2017-01-03 Thread Abhishek Agarwal
I am running into the same. which version are you using? On Thu, Dec 1, 2016 at 11:55 PM, Madhuri Sasurkar wrote: > Hi All, > > I have set up my kafka cluster and enabled JMX. > I was going through all the mbeans that are exposed and I am not able to > find mbean "kafka.log:type=LogFlushStats,na

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

2017-01-03 Thread Apache Jenkins Server
See Changes: [ismael] MINOR: Scala code readability improvements -- [...truncated 17626 lines...] org.apache.kafka.streams.KafkaStreamsTest > shouldNotGetAllTasksWithStoreWhenNotRunning STARTE

[jira] [Commented] (KAFKA-4576) Log segments close to max size break on fetch

2017-01-03 Thread huxi (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4576?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15795040#comment-15795040 ] huxi commented on KAFKA-4576: - If we use a loop structure to collect all the 12 bytes, there i

[jira] [Created] (KAFKA-4582) KStream job fails in multi-thread mode

2017-01-03 Thread Manuel Romero (JIRA)
Manuel Romero created KAFKA-4582: Summary: KStream job fails in multi-thread mode Key: KAFKA-4582 URL: https://issues.apache.org/jira/browse/KAFKA-4582 Project: Kafka Issue Type: Bug

[jira] [Commented] (KAFKA-4576) Log segments close to max size break on fetch

2017-01-03 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4576?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15795130#comment-15795130 ] Ismael Juma commented on KAFKA-4576: In theory, we should eventually reach the end of

[jira] [Created] (KAFKA-4583) KafkaConsumerTest.consumerCloseTest transient failure

2017-01-03 Thread Ismael Juma (JIRA)
Ismael Juma created KAFKA-4583: -- Summary: KafkaConsumerTest.consumerCloseTest transient failure Key: KAFKA-4583 URL: https://issues.apache.org/jira/browse/KAFKA-4583 Project: Kafka Issue Type: S

[jira] [Updated] (KAFKA-4583) KafkaConsumerTest.testGracefulClose transient failure

2017-01-03 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4583?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-4583: --- Summary: KafkaConsumerTest.testGracefulClose transient failure (was: KafkaConsumerTest.consumerCloseT

[jira] [Commented] (KAFKA-4583) KafkaConsumerTest.testGracefulClose transient failure

2017-01-03 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4583?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15795212#comment-15795212 ] Ismael Juma commented on KAFKA-4583: [~rsivaram], maybe you could take a look at this

[jira] [Resolved] (KAFKA-2057) DelayedOperationTest.testRequestExpiry transient failure

2017-01-03 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2057?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma resolved KAFKA-2057. Resolution: Fixed https://github.com/apache/kafka/pull/1890 did eventually fix the new issue so clo

[jira] [Updated] (KAFKA-4528) Failure in kafka.producer.ProducerTest.testAsyncSendCanCorrectlyFailWithTimeout

2017-01-03 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4528?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-4528: --- Status: Patch Available (was: Open) > Failure in > kafka.producer.ProducerTest.testAsyncSendCanCorre

[jira] [Updated] (KAFKA-4569) Transient failure in org.apache.kafka.clients.consumer.KafkaConsumerTest.testWakeupWithFetchDataAvailable

2017-01-03 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4569?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-4569: --- Fix Version/s: 0.10.2.0 > Transient failure in > org.apache.kafka.clients.consumer.KafkaConsumerTest.

[jira] [Commented] (KAFKA-4550) current trunk unstable

2017-01-03 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4550?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15795221#comment-15795221 ] Ismael Juma commented on KAFKA-4550: I provided a patch for KAFKA-4528. I suggest clos

[jira] [Commented] (KAFKA-4569) Transient failure in org.apache.kafka.clients.consumer.KafkaConsumerTest.testWakeupWithFetchDataAvailable

2017-01-03 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4569?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15795223#comment-15795223 ] Ismael Juma commented on KAFKA-4569: This started failing recently, so it would be goo

[jira] [Commented] (KAFKA-4113) Allow KTable bootstrap

2017-01-03 Thread Mitch Seymour (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4113?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15795304#comment-15795304 ] Mitch Seymour commented on KAFKA-4113: -- I believe I am experiencing the same issue th

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

2017-01-03 Thread Jun Rao
Just to follow up on Radai's idea of pushing the buffering logic to the broker. It may be possible to do this efficiently if we assume aborted transactions are rare. The following is a draft proposal. For each partition, the broker maintains the last stable offset (LSO) as described in the document

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

2017-01-03 Thread Jun Rao
Thanks for the KIP. Request-level impersonation is going to make the client implementation more complicated. On the producer side, we batch messages per partition. Now, do we have to batch per partition, per user? The consumer client is designed to be single-threaded. So, I am not sure if request-

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

2017-01-03 Thread Jun Rao
Hi, Mani Thanks for the proposal. Looks good at the high level. A few comments below. 101. DelegationTokenRequest/DelegationTokenResponse 101.1 I am wondering if renewer should be principalType + name to match what we have in kafka-acls tool? 101.2 Do we need to return owner in DelegationTokenRe

[GitHub] kafka pull request #2083: KAFKA-4318 Migrate ProducerSendTest to the new con...

2017-01-03 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2083 --- 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-4318) Migrate ProducerSendTest to the new consumer

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

[jira] [Resolved] (KAFKA-4318) Migrate ProducerSendTest to the new consumer

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

[GitHub] kafka pull request #2281: KAFKA-4561: Ordering of operations in StreamThread...

2017-01-03 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2281 --- 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-4561) Ordering of operations in StreamThread.shutdownTasksAndState may void at-least-once guarantees

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

[jira] [Commented] (KAFKA-4561) Ordering of operations in StreamThread.shutdownTasksAndState may void at-least-once guarantees

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

[jira] [Updated] (KAFKA-2434) remove roundrobin identical topic constraint in consumer coordinator (old API)

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

[jira] [Commented] (KAFKA-2434) remove roundrobin identical topic constraint in consumer coordinator (old API)

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

[jira] [Commented] (KAFKA-2394) Use RollingFileAppender by default in log4j.properties

2017-01-03 Thread JIRA
[ https://issues.apache.org/jira/browse/KAFKA-2394?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15795530#comment-15795530 ] Gérald Quintana commented on KAFKA-2394: It would be great to be able to mix time

[GitHub] kafka pull request #145: KAFKA-2434: Remove identical topic subscription con...

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

[jira] [Resolved] (KAFKA-4582) KStream job fails in multi-thread mode

2017-01-03 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4582?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax resolved KAFKA-4582. Resolution: Duplicate > KStream job fails in multi-thread mode > --

[jira] [Commented] (KAFKA-4582) KStream job fails in multi-thread mode

2017-01-03 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4582?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15795551#comment-15795551 ] Matthias J. Sax commented on KAFKA-4582: [~manudb9] Thanks for reporting! This sho

[jira] [Updated] (KAFKA-4528) Failure in kafka.producer.ProducerTest.testAsyncSendCanCorrectlyFailWithTimeout

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

[GitHub] kafka pull request #2298: KAFKA-4528: Fix failure in ProducerTest.testAsyncS...

2017-01-03 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2298 --- 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-4528) Failure in kafka.producer.ProducerTest.testAsyncSendCanCorrectlyFailWithTimeout

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

[jira] [Reopened] (KAFKA-4222) Transient failure in QueryableStateIntegrationTest.queryOnRebalance

2017-01-03 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4222?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang reopened KAFKA-4222: -- > Transient failure in QueryableStateIntegrationTest.queryOnRebalance >

[jira] [Commented] (KAFKA-4550) current trunk unstable

2017-01-03 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4550?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15795573#comment-15795573 ] Guozhang Wang commented on KAFKA-4550: -- I'm re-opening https://issues.apache.org/jira

[jira] [Commented] (KAFKA-4222) Transient failure in QueryableStateIntegrationTest.queryOnRebalance

2017-01-03 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4222?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15795576#comment-15795576 ] Guozhang Wang commented on KAFKA-4222: -- Re-opening this issue as we have seen it happ

[jira] [Commented] (KAFKA-4497) log cleaner breaks on timeindex

2017-01-03 Thread Andrew Holford (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4497?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15795598#comment-15795598 ] Andrew Holford commented on KAFKA-4497: --- Hi, We have hit this issue in one of our t

[jira] [Updated] (KAFKA-4466) Add support to ducktape to run only a part of all tests

2017-01-03 Thread Raghav Kumar Gautam (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4466?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Raghav Kumar Gautam updated KAFKA-4466: --- Assignee: Ewen Cheslack-Postava > Add support to ducktape to run only a part of all te

[jira] [Resolved] (KAFKA-4466) Add support to ducktape to run only a part of all tests

2017-01-03 Thread Raghav Kumar Gautam (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4466?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Raghav Kumar Gautam resolved KAFKA-4466. Resolution: Fixed > Add support to ducktape to run only a part of all tests > --

[jira] [Commented] (KAFKA-4466) Add support to ducktape to run only a part of all tests

2017-01-03 Thread Raghav Kumar Gautam (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4466?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15795601#comment-15795601 ] Raghav Kumar Gautam commented on KAFKA-4466: Thanks [~ewencp] for adding the s

Re: [VOTE] KIP-99: Add Global Tables to Kafka Streams

2017-01-03 Thread Damian Guy
Thank Guozhang, i've updated the KIP based on the discussion. On Fri, 30 Dec 2016 at 01:33 Guozhang Wang wrote: > Besides my comments on the other DISCUSS email thread, I'm +1. > > > On Mon, Dec 12, 2016 at 9:32 AM, Bill Bejeck wrote: > > > +1 > > > > On Mon, Dec 12, 2016 at 12:29 PM, Matthias

[VOTE] KIP-101: Alter Replication Protocol to use Leader Generation rather than High Watermark for Truncation

2017-01-03 Thread Ben Stopford
Hi All Happy New Year! Hopefully everyone has had an opportunity to review this KIP now, if they wished to, so I'd like to call a vote. As a reminder the KIP proposes a change to the replication protocol to remove the potential for replicas to diverge. https://cwiki.apache.org/confluence/displa

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

2017-01-03 Thread Apache Jenkins Server
See Changes: [ismael] KAFKA-4318; Migrate ProducerSendTest to the new consumer -- [...truncated 32970 lines...] org.apache.kafka.streams.KafkaStreamsTest > shouldNotGetAllTasksWithStoreWhenNot

Build problems in Eclipse dev environment setup

2017-01-03 Thread Dhwani Katagade
Hi, I am trying to setup the eclipse dev environment on trunk. I have followed the instructions given here https://github.com/apache/kafka/blob/trunk/README.md https://cwiki.apache.org/confluence/display/KAFKA/Developer+Setup#DeveloperSetup-Eclipsesetup After I run ./gradlew eclipse in the com

[DISCUSS] KIP-106 - Change Default unclean.leader.election.enabled from True to False

2017-01-03 Thread Ben Stopford
Hi All Please find the below KIP which proposes changing the setting unclean.leader.election.enabled from true to false. The motivation for this change is that it catches out new Kafka users who don’t realise the default favours availability over data loss. This would mean clusters wishing to

[jira] [Updated] (KAFKA-4404) Add knowledge of sign to numeric schema types

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

[GitHub] kafka pull request #2296: KAFKA-4404: Add javadocs to document core Connect ...

2017-01-03 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2296 --- 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-4404) Add knowledge of sign to numeric schema types

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

[jira] [Updated] (KAFKA-4584) Fail the 'kafka-configs' command if the config to be removed does not exist

2017-01-03 Thread Vahid Hashemian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4584?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Vahid Hashemian updated KAFKA-4584: --- Description: With [KAFKA-4480|https://issues.apache.org/jira/browse/KAFKA-4480], the {{kafka-

[jira] [Created] (KAFKA-4584) Fail the 'kafka-configs' command if the config to be removed does not exist

2017-01-03 Thread Vahid Hashemian (JIRA)
Vahid Hashemian created KAFKA-4584: -- Summary: Fail the 'kafka-configs' command if the config to be removed does not exist Key: KAFKA-4584 URL: https://issues.apache.org/jira/browse/KAFKA-4584 Project

Re: [DISCUSS] KIP-66 Kafka Connect Transformers for messages

2017-01-03 Thread Shikhar Bhushan
Makes sense Ewen, I edited the KIP to include this criteria. I'd like to start a voting thread soon unless anyone has additional points for discussion. On Fri, Dec 30, 2016 at 12:14 PM Ewen Cheslack-Postava wrote: On Thu, Dec 15, 2016 at 7:41 PM, Shikhar Bhushan wrote: > There is no decision

Re: [DISCUSS] KIP-104: Granular Sensors for Streams

2017-01-03 Thread Guozhang Wang
Thanks for the proposed KIP. Overall LGTM. A few comments: 1. "at the granularity of each processor node, in the addition to the global rate" I think you also add one sensor at the granularity of tasks, "Skipped records sensor in StreamTask" right? 2. From PR 1446 it seems you have also added a

Re: [DISCUSS] KIP-106 - Change Default unclean.leader.election.enabled from True to False

2017-01-03 Thread Gwen Shapira
I strongly support this. I've seen many users and customers accidentally lose data because they didn't know this configuration exists. I always support defaulting to "Not lose data" and let the users who prefer high availability do some extra research. This will need to be very well documented - bo

Re: [VOTE] KIP-72 - Allow putting a bound on memory consumed by Incoming requests

2017-01-03 Thread radai
I've just re-validated the functionality works - broker throttles under stress instead of OOMs. at this point my branch ( https://github.com/radai-rosenblatt/kafka/tree/broker-memory-pool-with-muting) is "code complete" and somewhat tested and im waiting on the voting process to come to a conclusi

Re: [VOTE] KIP-99: Add Global Tables to Kafka Streams

2017-01-03 Thread Gwen Shapira
+1 I'm glad this super-common use-case will become more performant now. On Mon, Dec 12, 2016 at 3:45 AM, Damian Guy wrote: > Hi all, > > I'd like to start the vote for KIP-99: > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=67633649 > > There is a PR for it here: https://githu

Re: [DISCUSS] KIP-106 - Change Default unclean.leader.election.enabled from True to False

2017-01-03 Thread Ismael Juma
Thanks for the KIP, +1 from me. Ismael On 3 Jan 2017 6:54 pm, "Ben Stopford" wrote: > Hi All > > Please find the below KIP which proposes changing the setting > unclean.leader.election.enabled from true to false. The motivation for > this change is that it catches out new Kafka users who don’t

[GitHub] kafka pull request #2299: KAFKA-3209: KIP-66: single message transforms

2017-01-03 Thread shikhar
GitHub user shikhar opened a pull request: https://github.com/apache/kafka/pull/2299 KAFKA-3209: KIP-66: single message transforms *WIP* particularly around testing You can merge this pull request into a Git repository by running: $ git pull https://github.com/shikhar/kafka smt

Re: [DISCUSS] KIP-106 - Change Default unclean.leader.election.enabled from True to False

2017-01-03 Thread Tom Crayford
+1. We've been running it in production for a long time and it's the right default. On Tue, Jan 3, 2017 at 7:17 PM, Ismael Juma wrote: > Thanks for the KIP, +1 from me. > > Ismael > > On 3 Jan 2017 6:54 pm, "Ben Stopford" wrote: > > > Hi All > > > > Please find the below KIP which proposes chan

[jira] [Commented] (KAFKA-3209) Support single message transforms in Kafka Connect

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

Re: [DISCUSS] KIP-106 - Change Default unclean.leader.election.enabled from True to False

2017-01-03 Thread Neha Narkhede
Big +1. On Tue, Jan 3, 2017 at 11:22 AM Tom Crayford wrote: > +1. We've been running it in production for a long time and it's the right > default. > > On Tue, Jan 3, 2017 at 7:17 PM, Ismael Juma wrote: > > > Thanks for the KIP, +1 from me. > > > > Ismael > > > > On 3 Jan 2017 6:54 pm, "Ben Sto

Re: [VOTE] KIP-99: Add Global Tables to Kafka Streams

2017-01-03 Thread Ewen Cheslack-Postava
+1 -Ewen On Tue, Jan 3, 2017 at 11:16 AM, Gwen Shapira wrote: > +1 > > I'm glad this super-common use-case will become more performant now. > > On Mon, Dec 12, 2016 at 3:45 AM, Damian Guy wrote: > > Hi all, > > > > I'd like to start the vote for KIP-99: > > https://cwiki.apache.org/confluence/

Re: [DISCUSS] Dormant/Inactive KIPs

2017-01-03 Thread Neha Narkhede
+1. This is a good idea as we have a lot of KIPs in flight. On Tue, Dec 13, 2016 at 3:53 AM Ismael Juma wrote: > Hi all, > > A while back Grant proposed moving inactive/dormant KIPs to a separate > table in the wiki. I think this is a good idea as it will make it easier > for people to see the K

Re: [VOTE] KIP-99: Add Global Tables to Kafka Streams

2017-01-03 Thread Neha Narkhede
+1 (binding) On Tue, Jan 3, 2017 at 11:40 AM Ewen Cheslack-Postava wrote: > +1 > > -Ewen > > On Tue, Jan 3, 2017 at 11:16 AM, Gwen Shapira wrote: > > > +1 > > > > I'm glad this super-common use-case will become more performant now. > > > > On Mon, Dec 12, 2016 at 3:45 AM, Damian Guy > wrote: >

Re: [DISCUSS] KIP-106 - Change Default unclean.leader.election.enabled from True to False

2017-01-03 Thread Ian Wrigley
+1 from me too. When I talk to people in training classes, who are typically much newer to Kafka, they tend to be surprised (and scared/horrified) that the default is true. Much safer to set it to false and let people change it when they really understand the tradeoffs. Ian. --- Ian Wrigley Di

Re: [DISCUSS] KIP 88: OffsetFetch Protocol Update

2017-01-03 Thread Vahid S Hashemian
One more try to ask for feedback on this KIP (that had to go through some more changes after approval): https://cwiki.apache.org/confluence/display/KAFKA/KIP-88%3A+OffsetFetch+Protocol+Update If there is no concern I could start the vote, hoping it could make it to the 0.10.2.0 release. Thanks

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

2017-01-03 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-4561; Ordering of operations in StreamThread.shutdownTasksAndState [wangguoz] KAFKA-2434: Remove identical topic subscription constraint for [wangguoz] KAFKA-4528: Fix failure in --

[GitHub] kafka pull request #2300: KAFKA-4583: Fail the 'kafka-configs' command if th...

2017-01-03 Thread vahidhashemian
GitHub user vahidhashemian opened a pull request: https://github.com/apache/kafka/pull/2300 KAFKA-4583: Fail the 'kafka-configs' command if the config to be removed does not exist You can merge this pull request into a Git repository by running: $ git pull https://github.com/

[jira] [Commented] (KAFKA-4583) KafkaConsumerTest.testGracefulClose transient failure

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

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

2017-01-03 Thread Apache Jenkins Server
See Changes: [jason] KAFKA-4404; Add javadocs to document core Connect types, especially that -- [...truncated 2369 lines...] kafka.api.AdminClientTest > testDescribeConsumerGroup STARTED kafka

[jira] [Updated] (KAFKA-4584) Fail the 'kafka-configs' command if the config to be removed does not exist

2017-01-03 Thread Vahid Hashemian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4584?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Vahid Hashemian updated KAFKA-4584: --- Status: Patch Available (was: Open) > Fail the 'kafka-configs' command if the config to be re

Re: [DISCUSS] KIP-106 - Change Default unclean.leader.election.enabled from True to False

2017-01-03 Thread Sriram Subramanian
+1 On Tue, Jan 3, 2017 at 11:56 AM, Ian Wrigley wrote: > +1 from me too. When I talk to people in training classes, who are > typically much newer to Kafka, they tend to be surprised (and > scared/horrified) that the default is true. Much safer to set it to false > and let people change it when

Re: [VOTE] KIP-99: Add Global Tables to Kafka Streams

2017-01-03 Thread Sriram Subramanian
+1 On Tue, Jan 3, 2017 at 11:43 AM, Neha Narkhede wrote: > +1 (binding) > > On Tue, Jan 3, 2017 at 11:40 AM Ewen Cheslack-Postava > wrote: > > > +1 > > > > -Ewen > > > > On Tue, Jan 3, 2017 at 11:16 AM, Gwen Shapira wrote: > > > > > +1 > > > > > > I'm glad this super-common use-case will becom

[jira] [Assigned] (KAFKA-4572) Kafka connect for windows is missing

2017-01-03 Thread Vahid Hashemian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4572?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Vahid Hashemian reassigned KAFKA-4572: -- Assignee: Vahid Hashemian (was: Ewen Cheslack-Postava) > Kafka connect for windows is

Re: [DISCUSS] KIP 88: OffsetFetch Protocol Update

2017-01-03 Thread Ewen Cheslack-Postava
Vahid, This looks reasonable to me and fits well with the changes made for metadata requests. -Ewen On Tue, Jan 3, 2017 at 12:12 PM, Vahid S Hashemian < vahidhashem...@us.ibm.com> wrote: > One more try to ask for feedback on this KIP (that had to go through some > more changes after approval):

Re: [DISCUSS] KIP-106 - Change Default unclean.leader.election.enabled from True to False

2017-01-03 Thread Onur Karaman
+1 On Tue, Jan 3, 2017 at 1:12 PM, Sriram Subramanian wrote: > +1 > > On Tue, Jan 3, 2017 at 11:56 AM, Ian Wrigley wrote: > > > +1 from me too. When I talk to people in training classes, who are > > typically much newer to Kafka, they tend to be surprised (and > > scared/horrified) that the def

[jira] [Commented] (KAFKA-2394) Use RollingFileAppender by default in log4j.properties

2017-01-03 Thread Dustin Cote (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2394?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15796240#comment-15796240 ] Dustin Cote commented on KAFKA-2394: [~gquintana] I agree it would be better, but the

[jira] [Commented] (KAFKA-1368) Upgrade log4j

2017-01-03 Thread JIRA
[ https://issues.apache.org/jira/browse/KAFKA-1368?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15796263#comment-15796263 ] Gérald Quintana commented on KAFKA-1368: It would be great to be able to mix time

[jira] [Updated] (KAFKA-3856) Move inner classes accessible only functions in TopologyBuilder out of public APIs

2017-01-03 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3856?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-3856: --- Labels: api needs-kip (was: api) > Move inner classes accessible only functions in TopologyBu

[jira] [Created] (KAFKA-4585) Offset fetch and commit requests use the same permissions

2017-01-03 Thread Ewen Cheslack-Postava (JIRA)
Ewen Cheslack-Postava created KAFKA-4585: Summary: Offset fetch and commit requests use the same permissions Key: KAFKA-4585 URL: https://issues.apache.org/jira/browse/KAFKA-4585 Project: Kafka

[jira] [Commented] (KAFKA-4585) Offset fetch and commit requests use the same permissions

2017-01-03 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4585?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15796326#comment-15796326 ] Ewen Cheslack-Postava commented on KAFKA-4585: -- [~vahid] I noticed this issue

[jira] [Commented] (KAFKA-4585) Offset fetch and commit requests use the same permissions

2017-01-03 Thread Vahid Hashemian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4585?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15796353#comment-15796353 ] Vahid Hashemian commented on KAFKA-4585: [~ewencp] Sure, I'd be happy to take a lo

[jira] [Assigned] (KAFKA-4585) Offset fetch and commit requests use the same permissions

2017-01-03 Thread Vahid Hashemian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4585?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Vahid Hashemian reassigned KAFKA-4585: -- Assignee: Vahid Hashemian > Offset fetch and commit requests use the same permissions >

[jira] [Resolved] (KAFKA-4429) records-lag should be zero if FetchResponse is empty

2017-01-03 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4429?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-4429. Resolution: Fixed Fix Version/s: 0.10.2.0 Issue resolved by pull request 2155 [https:

  1   2   >