Re: [DISCUSS] KIP-185: Make exactly once in order delivery per partition the default producer setting

2017-08-14 Thread Apurva Mehta
Hi Becket, Thanks for your comments. I added a 'Background' section to the KIP which defines the terms we are using. It also has a note on replication and delivery guarantees. Please have a look:

[GitHub] kafka pull request #3667: [KAFKA-5606] Review consumer's RequestFuture usage...

2017-08-14 Thread jedichien
GitHub user jedichien opened a pull request: https://github.com/apache/kafka/pull/3667 [KAFKA-5606] Review consumer's RequestFuture usage pattern Replacing succeeded, failed and retry with a status method returning an enum with 'SUCCEEDED', 'FAILED', 'RETRY' and 'NOT_RETRY' You

Re: [DISCUSS] KIP-185: Make exactly once in order delivery per partition the default producer setting

2017-08-14 Thread Becket Qin
Hi Apurva, Regarding acks. I think acks=all is not an isolated independent configuration for durability. The reason we want to enable acks=all is to tolerate broker failures. But If that is the case, not setting min.isr to >=2 seems defeating that purpose. If we set min.isr=2, setting replication

Re: Way to check if custom SMT has been added to the classpath or even if it i working.

2017-08-14 Thread Konstantine Karantasis
Hi, connector-plugins endpoint does not list the transformations classes currently. However if you are using the latest Kafka version ( >= 0.11.0) one way to see if your transform is discovered during startup in the given classpath is to notice whether a log message such as the one below is

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

2017-08-14 Thread Apache Jenkins Server
See

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

2017-08-14 Thread Apache Jenkins Server
See Changes: [jason] HOTFIX: ConsoleConsumer using wrong old consumer config value for -- [...truncated 938.27 KB...] kafka.log.BrokerCompressionTest >

[GitHub] kafka pull request #3665: HOTFIX: ConsoleConsumer using wrong old consumer c...

2017-08-14 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3665 --- 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-185: Make exactly once in order delivery per partition the default producer setting

2017-08-14 Thread Guozhang Wang
Just want to clarify that regarding 1), I'm fine with changing it to `all` but just wanted to argue it is not necessarily correlate with the exactly-once semantics, but rather on persistence v.s. availability trade-offs, so I'd like to discuss them separately. Regarding 2), one minor concern I

[GitHub] kafka pull request #3666: KAFKA-5730: Consumer should invoke async commit ca...

2017-08-14 Thread hachikuji
GitHub user hachikuji opened a pull request: https://github.com/apache/kafka/pull/3666 KAFKA-5730: Consumer should invoke async commit callback before sync commit returns You can merge this pull request into a Git repository by running: $ git pull

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

2017-08-14 Thread Apache Jenkins Server
See Changes: [damian.guy] KAFKA-4643; Improve test coverage of StreamsKafkaClient -- [...truncated 2.57 MB...]

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

2017-08-14 Thread Apache Jenkins Server
See Changes: [damian.guy] KAFKA-4643; Improve test coverage of StreamsKafkaClient -- [...truncated 913.53 KB...] kafka.integration.AutoOffsetResetTest >

Re: [DISCUSS] KIP-185: Make exactly once in order delivery per partition the default producer setting

2017-08-14 Thread Apurva Mehta
Hello, I just want to summarize where we are in this discussion There are two major points of contention: should we have acks=1 or acsk=all by default? and how to cap max.in.flight.requests.per.connection? 1) acks=1 vs acks=all1 Here are the tradeoffs of each: If you have

Re: Kafka 0.11.0.0

2017-08-14 Thread Ismael Juma
Hi Srikanth, 0.11.0.0 is looking pretty good so far. One concern is: https://issues.apache.org/jira/browse/KAFKA-5600 There is no concrete plan for 0.11.0.1, but I'd expect a RC within a few weeks (2 to 4, probably). Ismael On Mon, Aug 14, 2017 at 5:15 AM, Srikanth Sampath

[GitHub] kafka pull request #3663: KAFKA-4643: Improve test coverage of StreamsKafkaC...

2017-08-14 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3663 --- 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] [Resolved] (KAFKA-4643) Improve test coverage of StreamsKafkaClient

2017-08-14 Thread Damian Guy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4643?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Damian Guy resolved KAFKA-4643. --- Resolution: Fixed Fix Version/s: 1.0.0 Issue resolved by pull request 3663

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

2017-08-14 Thread Apache Jenkins Server
See Changes: [ismael] KAFKA-5595; Ensure client connection ids are not reused too quickly -- [...truncated 914.17 KB...] kafka.integration.AutoOffsetResetTest >

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

2017-08-14 Thread Apache Jenkins Server
See Changes: [ismael] KAFKA-5595; Ensure client connection ids are not reused too quickly -- [...truncated 913.45 KB...] kafka.server.epoch.LeaderEpochFileCacheTest >

[GitHub] kafka pull request #3665: HOTFIX: ConsoleConsumer using wrong old consumer c...

2017-08-14 Thread hachikuji
GitHub user hachikuji opened a pull request: https://github.com/apache/kafka/pull/3665 HOTFIX: ConsoleConsumer using wrong old consumer config value for auto.offset.reset You can merge this pull request into a Git repository by running: $ git pull

[GitHub] kafka pull request #3530: KAFKA-5595: Ensure client connection ids are not r...

2017-08-14 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3530 --- 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: Kafka 0.11.0.0

2017-08-14 Thread Thomas Crayford
0.11.0.0 is a brand new release, with a very large number of changes compared to the previous stable release (0.10.2.1). As a thing that stores data, I would not recommend you switch to it without a very large amount of testing and validation, probably involving running a shadow setup of your

[jira] [Resolved] (KAFKA-3984) Broker doesn't retry reconnecting to an expired Zookeeper connection

2017-08-14 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3984?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao resolved KAFKA-3984. Resolution: Duplicate Marking this as duplicate. The fix will be done in KAFKA-5473. > Broker doesn't

Build failed in Jenkins: kafka-0.11.0-jdk7 #270

2017-08-14 Thread Apache Jenkins Server
See Changes: [ismael] MINOR: Safer handling of requests prior to SASL authentication -- [...truncated 2.43 MB...]

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

2017-08-14 Thread Apache Jenkins Server
See Changes: [ismael] MINOR: Safer handling of requests prior to SASL authentication -- [...truncated 913.07 KB...] kafka.producer.AsyncProducerTest >

[GitHub] kafka pull request #3664: KAFKA-5733: ensure clean RocksDB directory before ...

2017-08-14 Thread bbejeck
GitHub user bbejeck opened a pull request: https://github.com/apache/kafka/pull/3664 KAFKA-5733: ensure clean RocksDB directory before setting prepareForB… …ulkload settings You can merge this pull request into a Git repository by running: $ git pull

[GitHub] kafka pull request #3558: MINOR: Safer handling of requests prior to SASL au...

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

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

2017-08-14 Thread Apache Jenkins Server
See Changes: [ismael] MINOR: Safer handling of requests prior to SASL authentication -- Started by an SCM change Started by an SCM change [EnvInject] - Loading node

[jira] [Created] (KAFKA-5733) System tests get exception RocksDBException: db has more levels than options.num_levels

2017-08-14 Thread Eno Thereska (JIRA)
Eno Thereska created KAFKA-5733: --- Summary: System tests get exception RocksDBException: db has more levels than options.num_levels Key: KAFKA-5733 URL: https://issues.apache.org/jira/browse/KAFKA-5733

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

2017-08-14 Thread Apache Jenkins Server
See Changes: [damian.guy] KAFKA-5673; refactor KeyValueStore hierarchy to make -- [...truncated 4.85 MB...]

[GitHub] kafka pull request #3659: KAFKA-4643: Improve test coverage of StreamsKafkaC...

2017-08-14 Thread adyach
Github user adyach closed the pull request at: https://github.com/apache/kafka/pull/3659 --- 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

[GitHub] kafka pull request #3663: KAFKA-4643: Improve test coverage of StreamsKafkaC...

2017-08-14 Thread adyach
GitHub user adyach opened a pull request: https://github.com/apache/kafka/pull/3663 KAFKA-4643: Improve test coverage of StreamsKafkaClient The commit brings improved test coverage for StreamsKafkaClientTest.java You can merge this pull request into a Git repository by running:

[GitHub] kafka pull request #3592: KAFKA-5673: refactor KeyValueStore hierarchy to ma...

2017-08-14 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3592 --- 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-113: Support replicas movement between log directories

2017-08-14 Thread Dong Lin
Hey all, After discussion with Becket in the https://github.com/apache/kafka/pull/3621, I have updated the KIP-113 to make the following minor changes to the protocol: 1) Renamed DescribeDirsRequest (and DescribeDirsResponse) to DescribeLogDirsRequest (and DescribeLogDirsResponse). This change

Re: [VOTE] KIP-175: Additional '--describe' views for ConsumerGroupCommand

2017-08-14 Thread Manikumar
+1 (non-binding) On Fri, Aug 11, 2017 at 8:09 PM, Mickael Maison wrote: > +1 non-binding, thanks Vahid > > On Wed, Aug 9, 2017 at 9:31 PM, Jason Gustafson > wrote: > > Thanks for the KIP. +1 > > > > On Thu, Jul 27, 2017 at 2:04 PM, Vahid S