[VOTE] KIP-276 Add StreamsConfig prefix for different consumers

2018-04-16 Thread Boyang Chen
Hey friends. I would like to start a vote on KIP 276: add StreamsConfig prefix for different consumers. KIP: here Pull request: here Jira:

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

2018-04-16 Thread Apache Jenkins Server
See Changes: [junrao] KAFKA-6650: Allowing transition to OfflineReplica state for replicas -- [...truncated 417.63 KB...] kafka.controller.ControllerEventManagerTest >

Build failed in Jenkins: kafka-trunk-jdk10 #24

2018-04-16 Thread Apache Jenkins Server
See Changes: [junrao] KAFKA-6650: Allowing transition to OfflineReplica state for replicas -- [...truncated 1.48 MB...] kafka.admin.ResetConsumerGroupOffsetTest >

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

2018-04-16 Thread Apache Jenkins Server
See

Re: [DISCUSS] KIP-283: Efficient Memory Usage for Down-Conversion

2018-04-16 Thread Jun Rao
Hi, Dhruvil, Thanks for the KIP. Looks good me to overall. Just one comment below. "To prevent this from happening, we will not delay down-conversion of the first partition in the response. We will down-convert all messages of the first partition in the I/O thread (like we do today), and only

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

2018-04-16 Thread Apache Jenkins Server
See

[jira] [Resolved] (KAFKA-6650) The controller should be able to handle a partially deleted topic

2018-04-16 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6650?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao resolved KAFKA-6650. Resolution: Fixed Fix Version/s: 2.0.0 merged the PR to trunk. > The controller should be able to

Build failed in Jenkins: kafka-trunk-jdk10 #23

2018-04-16 Thread Apache Jenkins Server
See Changes: [jason] KAFKA-6727; Fix broken Config hashCode() and equals() (#4796) [jason] MINOR: Log the exception thrown by Selector.poll (#4873) -- [...truncated 1.48

[jira] [Created] (KAFKA-6796) Surprising UNKNOWN_TOPIC error for produce/fetch requests to non-replicas

2018-04-16 Thread Jason Gustafson (JIRA)
Jason Gustafson created KAFKA-6796: -- Summary: Surprising UNKNOWN_TOPIC error for produce/fetch requests to non-replicas Key: KAFKA-6796 URL: https://issues.apache.org/jira/browse/KAFKA-6796 Project:

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

2018-04-16 Thread Apache Jenkins Server
See Changes: [jason] KAFKA-6727; Fix broken Config hashCode() and equals() (#4796) -- [...truncated 414.49 KB...] kafka.controller.PartitionLeaderElectionAlgorithmsTest >

Re: [VOTE] KIP-281: ConsumerPerformance: Increase Polling Loop Timeout and Make It Reachable by the End User

2018-04-16 Thread Ted Yu
+1 On Mon, Apr 16, 2018 at 2:25 PM, Alex Dunayevsky wrote: > Hello friends, > > Let's start the vote for KIP-281: ConsumerPerformance: Increase Polling > Loop Timeout and Make It Reachable by the End User: > https://cwiki.apache.org/confluence/display/KAFKA/KIP- >

[VOTE] KIP-281: ConsumerPerformance: Increase Polling Loop Timeout and Make It Reachable by the End User

2018-04-16 Thread Alex Dunayevsky
Hello friends, Let's start the vote for KIP-281: ConsumerPerformance: Increase Polling Loop Timeout and Make It Reachable by the End User: https://cwiki.apache.org/confluence/display/KAFKA/KIP-281%3A+ConsumerPerformance%3A+Increase+Polling+Loop+Timeout+and+Make+It+Reachable+by+the+End+User Thank

Re: [DISCUSS] KIP-281 ConsumerPerformance: Increase Polling Loop Timeout and Make It Reachable by the End User

2018-04-16 Thread Alex Dunayevsky
Hi Jason, Thanks for your feedback and attention to detail (some credit to Kafka 0.8). I've decreased the default polling loop timeout value to 10 seconds and renamed parameter to --timeout. Updates reflected in PR and KIP. Let's start the vote Cheers, Alex Dunayevsky On Thu, Apr 5, 2018, 21:37

Jenkins build is back to normal : kafka-trunk-jdk10 #22

2018-04-16 Thread Apache Jenkins Server
See

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

2018-04-16 Thread Apache Jenkins Server
See Changes: [jason] MINOR: Mention that -1 disables retention by time (#4881) [jason] KAFKA-6514; Add API version as a tag for the RequestsPerSec metric [wangguoz] Kafka-6792: Fix wrong pointer in the link

[jira] [Created] (KAFKA-6795) Add unit test for ReplicaAlterLogDirsThread

2018-04-16 Thread Anna Povzner (JIRA)
Anna Povzner created KAFKA-6795: --- Summary: Add unit test for ReplicaAlterLogDirsThread Key: KAFKA-6795 URL: https://issues.apache.org/jira/browse/KAFKA-6795 Project: Kafka Issue Type:

Re: [VOTE] KIP-270 A Scala wrapper library for Kafka Streams

2018-04-16 Thread Guozhang Wang
Thanks Debasish for the KIP! Will make another pass on the PR itself, but the KIP itself looks good. I'm +1 (binding). Guozhang On Mon, Apr 16, 2018 at 10:51 AM, John Roesler wrote: > Thanks again for this effort. I'm +1 (non-binding). > -John > > On Mon, Apr 16, 2018 at

[jira] [Resolved] (KAFKA-4327) Move Reset Tool from core to streams

2018-04-16 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4327?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-4327. -- Resolution: Won't Fix I'm resolving this ticket as won't fix for now. > Move Reset Tool from

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

2018-04-16 Thread Apache Jenkins Server
See Changes: [jason] MINOR: Mention that -1 disables retention by time (#4881) [jason] KAFKA-6514; Add API version as a tag for the RequestsPerSec metric [wangguoz] Kafka-6792: Fix wrong pointer in the link

[jira] [Resolved] (KAFKA-6514) Add API version as a tag for the RequestsPerSec metric

2018-04-16 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6514?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-6514. Resolution: Fixed Fix Version/s: 1.2.0 > Add API version as a tag for the

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

2018-04-16 Thread Apache Jenkins Server
See -- Started by an SCM change [EnvInject] - Loading node environment variables. Building remotely on H23 (ubuntu xenial) in workspace

Build failed in Jenkins: kafka-trunk-jdk10 #21

2018-04-16 Thread Apache Jenkins Server
See -- Started by an SCM change [EnvInject] - Loading node environment variables. Building remotely on H23 (ubuntu xenial) in workspace

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

2018-04-16 Thread Apache Jenkins Server
See -- Started by an SCM change [EnvInject] - Loading node environment variables. Building remotely on H23 (ubuntu xenial) in workspace

[jira] [Resolved] (KAFKA-6792) Wrong pointer in the link for stream dsl

2018-04-16 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6792?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-6792. -- Resolution: Fixed Fix Version/s: 1.0.0 > Wrong pointer in the link for stream dsl >

Re: [VOTE] KIP-270 A Scala wrapper library for Kafka Streams

2018-04-16 Thread John Roesler
Thanks again for this effort. I'm +1 (non-binding). -John On Mon, Apr 16, 2018 at 9:39 AM, Ismael Juma wrote: > Thanks for the contribution. I haven't reviewed all the new APIs in detail, > but the general approach sounds good to me. +1 (binding). > > Ismael > > On Wed, Apr

[jira] [Created] (KAFKA-6794) Support for incremental replica reassignment

2018-04-16 Thread Jason Gustafson (JIRA)
Jason Gustafson created KAFKA-6794: -- Summary: Support for incremental replica reassignment Key: KAFKA-6794 URL: https://issues.apache.org/jira/browse/KAFKA-6794 Project: Kafka Issue Type:

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

2018-04-16 Thread Apache Jenkins Server
See -- Started by an SCM change [EnvInject] - Loading node environment variables. Building remotely on H23 (ubuntu xenial) in workspace

Build failed in Jenkins: kafka-trunk-jdk10 #20

2018-04-16 Thread Apache Jenkins Server
See -- Started by an SCM change [EnvInject] - Loading node environment variables. Building remotely on H23 (ubuntu xenial) in workspace

Re: [VOTE] KIP-279: Fix log divergence between leader and follower after fast leader fail over

2018-04-16 Thread Ismael Juma
Thanks for the detailed KIP. +1 (binding) Ismael On Sat, Apr 14, 2018 at 3:54 PM, Anna Povzner wrote: > Hi All, > > > I would like to start the vote on KIP-279: Fix log divergence between > leader and follower after fast leader fail over. > > > For reference, here's the KIP

Re: [VOTE] KIP-279: Fix log divergence between leader and follower after fast leader fail over

2018-04-16 Thread Ben Stopford
+1 thanks Anna On Mon, Apr 16, 2018 at 5:37 PM Jason Gustafson wrote: > +1 Thanks for the KIP! > > On Sun, Apr 15, 2018 at 4:04 PM, Damian Guy wrote: > > > Thanks Anna +1 > > On Sun, 15 Apr 2018 at 15:40, Guozhang Wang wrote: > > >

Re: [VOTE] KIP-279: Fix log divergence between leader and follower after fast leader fail over

2018-04-16 Thread Jason Gustafson
+1 Thanks for the KIP! On Sun, Apr 15, 2018 at 4:04 PM, Damian Guy wrote: > Thanks Anna +1 > On Sun, 15 Apr 2018 at 15:40, Guozhang Wang wrote: > > > Anna, thanks for the KIP! +1 from me. > > > > Just one minor comment: `Broker A will respond with

Re: [DISCUSS] KIP-269: Substitution Within Configuration Values

2018-04-16 Thread Ron Dagostino
Hi Rajini. I think a good, illustrative OAuth example is the situation where a Kafka client (whether non-broker or broker, where the latter occurs when OAUTHBEARER is the inter-broker SASL mechanism) needs to authenticate to the token endpoint to retrieve an access token. There are actually 2

Re: [VOTE] KIP-235 Add DNS alias support for secured connection

2018-04-16 Thread Ted Yu
Looks good to me. BTW KAFKA-6195 contains more technical details than the KIP. See if you can enrich the Motivation section with some of the details. Thanks On Fri, Mar 23, 2018 at 12:05 PM, Skrzypek, Jonathan < jonathan.skrzy...@gs.com> wrote: > Hi, > > I would like to start a vote for

RE: [VOTE] KIP-235 Add DNS alias support for secured connection

2018-04-16 Thread Skrzypek, Jonathan
Hi, Could anyone take a look ? Does the proposal sound reasonable ? Jonathan Skrzypek From: Skrzypek, Jonathan [Tech] Sent: 23 March 2018 19:05 To: dev@kafka.apache.org Subject: [VOTE] KIP-235 Add DNS alias support for secured connection Hi, I would like to start a vote for KIP-235

Re: [VOTE] KIP-270 A Scala wrapper library for Kafka Streams

2018-04-16 Thread Ismael Juma
Thanks for the contribution. I haven't reviewed all the new APIs in detail, but the general approach sounds good to me. +1 (binding). Ismael On Wed, Apr 11, 2018 at 3:09 AM, Debasish Ghosh < debasish.gh...@lightbend.com> wrote: > Hello everyone - > > This is in continuation to the discussion

Re: [VOTE] KIP-270 A Scala wrapper library for Kafka Streams

2018-04-16 Thread Bill Bejeck
+1 -Bill On Mon, Apr 16, 2018 at 5:14 AM, Matthias J. Sax wrote: > +1 (binding) > > On 4/16/18 9:33 AM, Michael Noll wrote: > > +1 (non-binding) > > > > Thanks for contributing and shepherding this, Debasish and team! > > > > And thanks also to Alexis Seigneurin for the

[jira] [Created] (KAFKA-6793) Unnecessary warning log message

2018-04-16 Thread Anna O (JIRA)
Anna O created KAFKA-6793: - Summary: Unnecessary warning log message Key: KAFKA-6793 URL: https://issues.apache.org/jira/browse/KAFKA-6793 Project: Kafka Issue Type: Bug Components:

Re: [VOTE] #2 KIP-248: Create New ConfigCommand That Uses The New AdminClient

2018-04-16 Thread Viktor Somogyi
Hi Rajini, The current ConfigCommand would still be possible to use, therefore those who wish to set up SCRAM or initial quotas would be able to continue doing it through kafka-run-class.sh. In an ideal world I'd keep it in the current ConfigCommand command so we wouldn't mix the zookeeper and

Re: [DISCUSS]KIP-216: IQ should throw different exceptions for different errors

2018-04-16 Thread Matthias J. Sax
Thanks for the update Vito! It's up to you to keep the details part in the KIP or not. The (incomplete) question was, if we need `StateStoreFailException` or if existing `InvalidStateStoreException` could be used? Do you suggest that `InvalidStateStoreException` is not thrown at all anymore,

Re: [DISCUSS] KIP-269: Substitution Within Configuration Values

2018-04-16 Thread Rajini Sivaram
Hi Ron, Thanks for the analysis, this is very useful. Reducing the feature to the minimum required for the scenarios helps (though I was hoping that the redact flag would one day help with improving SASL diagnostics, that can be for another day). In the past when users requested different

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

2018-04-16 Thread Apache Jenkins Server
See Changes: [rajinisivaram] KAFKA-6696 Trogdor should support destroying tasks (#4759) [rajinisivaram] KAFKA-6771. Make specifying partitions more flexible (#4850) --

Re: [VOTE] KIP-270 A Scala wrapper library for Kafka Streams

2018-04-16 Thread Matthias J. Sax
+1 (binding) On 4/16/18 9:33 AM, Michael Noll wrote: > +1 (non-binding) > > Thanks for contributing and shepherding this, Debasish and team! > > And thanks also to Alexis Seigneurin for the original start of the Scala > wrapper API. > > Best, > Michael > > > > On Thu, Apr 12, 2018 at 10:24

Build failed in Jenkins: kafka-trunk-jdk10 #19

2018-04-16 Thread Apache Jenkins Server
See Changes: [rajinisivaram] KAFKA-6696 Trogdor should support destroying tasks (#4759) [rajinisivaram] KAFKA-6771. Make specifying partitions more flexible (#4850) --

Re: [DISCUSS] KIP-282: Add the listener name to the authentication context

2018-04-16 Thread Mickael Maison
I have not seen any replies yet =( Considering how minor it is, if there are no objections I'll start a vote in a few days On Thu, Apr 5, 2018 at 5:21 PM, Mickael Maison wrote: > Hi all, > > I have submitted KIP-282 to add the listener name to the authentication >

Re: [VOTE] #2 KIP-248: Create New ConfigCommand That Uses The New AdminClient

2018-04-16 Thread Rajini Sivaram
Hi Viktor, The KIP proposes to remove the ability to configure using ZooKeeper. This means we will no longer have the ability to start up a cluster with SCRAM credentials since we first need to create SCRAM credentials before brokers can start if the broker uses SCRAM for inter-broker

Re: [VOTE] KIP-270 A Scala wrapper library for Kafka Streams

2018-04-16 Thread Michael Noll
+1 (non-binding) Thanks for contributing and shepherding this, Debasish and team! And thanks also to Alexis Seigneurin for the original start of the Scala wrapper API. Best, Michael On Thu, Apr 12, 2018 at 10:24 AM, Damian Guy wrote: > Thanks for the KIP Debasish - +1