Re: [DISCUSS] KIP-213 Support non-key joining in KTable

2017-10-25 Thread Onur Karaman
Done. I changed my KIP to KIP-214. So this KIP doesn't need to change. On Wed, Oct 25, 2017 at 10:33 PM, Onur Karaman wrote: > Looks like Jan technically made his KIP wiki page first so I'll just > change my KIP number. > > On Wed, Oct 25, 2017 at 4:59 PM,

Re: [DISCUSS] KIP-213: Add zookeeper.max.in.flight.requests config to the broker

2017-10-25 Thread Onur Karaman
It looks like I hit a KIP number collision on the wiki page. Let's move the discussion over to the thread with subject "[DISCUSS] KIP-214: Add zookeeper.max.in.flight.requests config to the broker". https://www.mail-archive.com/dev@kafka.apache.org/msg81900.html On Wed, Oct 25, 2017 at 4:17 PM,

[DISCUSS] KIP-214: Add zookeeper.max.in.flight.requests config to the broker

2017-10-25 Thread Onur Karaman
Hey everyone. Giving this another shot since it looks like there was a KIP number collision on the wiki page. I made a config kip, KIP-214: Add zookeeper.max.in.flight.requests config to the broker:

Re: [DISCUSS] KIP-213 Support non-key joining in KTable

2017-10-25 Thread Onur Karaman
Looks like Jan technically made his KIP wiki page first so I'll just change my KIP number. On Wed, Oct 25, 2017 at 4:59 PM, Matthias J. Sax wrote: > Thanks a lot for the KIP. Can we please move the discussion to the dev > list? > > Thus, after fixing the KIP collision,

[GitHub] kafka pull request #4112: MINOR: Rename and change package of async ZooKeepe...

2017-10-25 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/4112 ---

[GitHub] kafka pull request #4137: KAFKA-6119: Bump epoch when expiring transactions ...

2017-10-25 Thread apurvam
GitHub user apurvam opened a pull request: https://github.com/apache/kafka/pull/4137 KAFKA-6119: Bump epoch when expiring transactions in the TransactionCoordinator A description of the problem is in the JIRA. I have added an integration test which reproduces the original

Re: [VOTE] KIP-207:The Offsets which ListOffsetsResponse returns should monotonically increase even during a partition leader change

2017-10-25 Thread Jason Gustafson
+1. Thanks for the KIP. On Mon, Oct 23, 2017 at 11:30 AM, Colin McCabe wrote: > On Mon, Oct 23, 2017, at 10:29, Jason Gustafson wrote: > > Thanks for the KIP. I'm assuming the new behavior only affects > > ListOffsets requests from the consumer. > > That's a very good point.

[GitHub] kafka pull request #4136: KAFKA-6100: Down-grade RocksDB to 5.7.3

2017-10-25 Thread guozhangwang
GitHub user guozhangwang opened a pull request: https://github.com/apache/kafka/pull/4136 KAFKA-6100: Down-grade RocksDB to 5.7.3 You can merge this pull request into a Git repository by running: $ git pull https://github.com/guozhangwang/kafka K6100-rocksdb-580-regression

Re: [DISCUSS] KIP-213 Support non-key joining in KTable

2017-10-25 Thread Matthias J. Sax
Thanks a lot for the KIP. Can we please move the discussion to the dev list? Thus, after fixing the KIP collision, just start a new DISCUSS thread. Thx. -Matthias On 10/25/17 4:20 PM, Ted Yu wrote: > Have you seen the email a moment ago from Onur which uses the same KIP > number ? > > Looks

[jira] [Created] (KAFKA-6127) Streams should never block infinitely

2017-10-25 Thread Matthias J. Sax (JIRA)
Matthias J. Sax created KAFKA-6127: -- Summary: Streams should never block infinitely Key: KAFKA-6127 URL: https://issues.apache.org/jira/browse/KAFKA-6127 Project: Kafka Issue Type: Bug

[jira] [Created] (KAFKA-6126) Reduce rebalance time by not checking if created topics are available

2017-10-25 Thread Matthias J. Sax (JIRA)
Matthias J. Sax created KAFKA-6126: -- Summary: Reduce rebalance time by not checking if created topics are available Key: KAFKA-6126 URL: https://issues.apache.org/jira/browse/KAFKA-6126 Project:

[jira] [Created] (KAFKA-6125) Avoid third party exception to flow through streams code base

2017-10-25 Thread Matthias J. Sax (JIRA)
Matthias J. Sax created KAFKA-6125: -- Summary: Avoid third party exception to flow through streams code base Key: KAFKA-6125 URL: https://issues.apache.org/jira/browse/KAFKA-6125 Project: Kafka

[jira] [Created] (KAFKA-6124) Revisit default config for internal client with regard to resilience

2017-10-25 Thread Matthias J. Sax (JIRA)
Matthias J. Sax created KAFKA-6124: -- Summary: Revisit default config for internal client with regard to resilience Key: KAFKA-6124 URL: https://issues.apache.org/jira/browse/KAFKA-6124 Project:

[jira] [Created] (KAFKA-6123) MetricsReporter does not get auto-generated client.id

2017-10-25 Thread Kevin Lu (JIRA)
Kevin Lu created KAFKA-6123: --- Summary: MetricsReporter does not get auto-generated client.id Key: KAFKA-6123 URL: https://issues.apache.org/jira/browse/KAFKA-6123 Project: Kafka Issue Type:

Re: [DISCUSS] KIP-213: Add zookeeper.max.in.flight.requests config to the broker

2017-10-25 Thread Ted Yu
This is for KAFKA-5894, right ? Please fill out the JIRA link. +1 on this proposal. On Wed, Oct 25, 2017 at 4:11 PM, Onur Karaman wrote: > Hey everyone. > > I made a config kip, KIP-213: Add zookeeper.max.in.flight.requests config > to the broker: >

[jira] [Resolved] (KAFKA-6047) Allow retries configuration for InternalTopicManager

2017-10-25 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6047?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax resolved KAFKA-6047. Resolution: Duplicate The new {{KafkaAdminClient}} allows to configure number of retries.

[jira] [Resolved] (KAFKA-5313) Improve exception handling on coordinator interactions

2017-10-25 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5313?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax resolved KAFKA-5313. Resolution: Duplicate This is contained by all other exception handling tasks of this

[jira] [Resolved] (KAFKA-5302) Improve exception handling on streams client (communication with brokers)

2017-10-25 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5302?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax resolved KAFKA-5302. Resolution: Duplicate After we replaced our internal client with new {{KafkaAdminClient}}

[DISCUSS] KIP-213: Add zookeeper.max.in.flight.requests config to the broker

2017-10-25 Thread Onur Karaman
Hey everyone. I made a config kip, KIP-213: Add zookeeper.max.in.flight.requests config to the broker: https://cwiki.apache.org/confluence/display/KAFKA/KIP-213%3A+Add+zookeeper.max.in.flight.requests+config+to+the+broker Comments are welcome. - Onur

[jira] [Resolved] (KAFKA-5217) Improve Streams internal exception handling

2017-10-25 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5217?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax resolved KAFKA-5217. Resolution: Duplicate This JIRA should be covered by multiple different once. Thus closing

[jira] [Resolved] (KAFKA-5301) Improve exception handling on consumer path

2017-10-25 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5301?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax resolved KAFKA-5301. Resolution: Duplicate This issue is contained by KAFKA-6121 and KAFKA-6122, thus I am

[jira] [Created] (KAFKA-6122) Global Consumer should handle TimeoutException

2017-10-25 Thread Matthias J. Sax (JIRA)
Matthias J. Sax created KAFKA-6122: -- Summary: Global Consumer should handle TimeoutException Key: KAFKA-6122 URL: https://issues.apache.org/jira/browse/KAFKA-6122 Project: Kafka Issue Type:

Re: [DISCUSS] KIP-205: Add getAllKeys() API to ReadOnlyWindowStore

2017-10-25 Thread Richard Yu
Xavier: There has been two pluses on the voting thread. Are you fine with the current formation? On Tue, Oct 24, 2017 at 4:26 PM, Richard Yu wrote: > I think we can come up with this compromise: range(long timeFrom, long > timeTo) will be changed to getKeys(long

[jira] [Created] (KAFKA-6121) Restore and global consumer should not use auto.offset.reset

2017-10-25 Thread Matthias J. Sax (JIRA)
Matthias J. Sax created KAFKA-6121: -- Summary: Restore and global consumer should not use auto.offset.reset Key: KAFKA-6121 URL: https://issues.apache.org/jira/browse/KAFKA-6121 Project: Kafka

[jira] [Created] (KAFKA-6120) RecordCollectorImpl should not retry sending

2017-10-25 Thread Matthias J. Sax (JIRA)
Matthias J. Sax created KAFKA-6120: -- Summary: RecordCollectorImpl should not retry sending Key: KAFKA-6120 URL: https://issues.apache.org/jira/browse/KAFKA-6120 Project: Kafka Issue Type:

[jira] [Created] (KAFKA-6119) Silent Data Loss in Kafka011 Transactional Producer

2017-10-25 Thread Gary Y. (JIRA)
Gary Y. created KAFKA-6119: -- Summary: Silent Data Loss in Kafka011 Transactional Producer Key: KAFKA-6119 URL: https://issues.apache.org/jira/browse/KAFKA-6119 Project: Kafka Issue Type: Bug

[GitHub] kafka pull request #4123: MINOR: reset state in cleanup, fixes jmx mixin fla...

2017-10-25 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/4123 ---

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

2017-10-25 Thread Apache Jenkins Server
See

Re: [VOTE] 1.0.0 RC3

2017-10-25 Thread Guozhang Wang
The deadline would be: "Please download, test and vote by Friday, October 27, 8pm PT" Guozhang On Wed, Oct 25, 2017 at 12:16 PM, Dana Powers wrote: > Does the voting deadline also need an update? > > > *** Please download, test and vote by Friday, October 20, 8pm PT >

Re: [VOTE] 1.0.0 RC3

2017-10-25 Thread Dana Powers
Does the voting deadline also need an update? > *** Please download, test and vote by Friday, October 20, 8pm PT On Wed, Oct 25, 2017 at 10:37 AM, Guozhang Wang wrote: > Ted: > > Thanks for the reminder. Yes it is a typo. In fact this is the "forth" > candidate of the

[GitHub] kafka pull request #4135: KAFKA-5848: Perform a complete topic name validati...

2017-10-25 Thread vahidhashemian
GitHub user vahidhashemian opened a pull request: https://github.com/apache/kafka/pull/4135 KAFKA-5848: Perform a complete topic name validation in KafkaConsumer's assign/subscribe You can merge this pull request into a Git repository by running: $ git pull

Re: [VOTE] 1.0.0 RC3

2017-10-25 Thread Guozhang Wang
Ted: Thanks for the reminder. Yes it is a typo. In fact this is the "forth" candidate of the release, not the "third" one :) Jaikiran: That's a fair point. Though I do not know how to achieve that with the maven central staging repository mechanism today [1]. If anyone has ideas how to do that

[jira] [Created] (KAFKA-6118) Transient failure in kafka.api.SaslScramSslEndToEndAuthorizationTest.testTwoConsumersWithDifferentSaslCredentials

2017-10-25 Thread Guozhang Wang (JIRA)
Guozhang Wang created KAFKA-6118: Summary: Transient failure in kafka.api.SaslScramSslEndToEndAuthorizationTest.testTwoConsumersWithDifferentSaslCredentials Key: KAFKA-6118 URL:

[GitHub] kafka pull request #4133: Fix typo dev guide title

2017-10-25 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/4133 ---

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

2017-10-25 Thread Apache Jenkins Server
See Changes: [wangguoz] HOTFIX: Remove sysout logging -- [...truncated 977.65 KB...] kafka.utils.UtilsTest > testReadInt PASSED kafka.utils.UtilsTest >

[GitHub] kafka pull request #4134: KAFKA-6075 Kafka cannot recover after an unclean s...

2017-10-25 Thread tedyu
GitHub user tedyu opened a pull request: https://github.com/apache/kafka/pull/4134 KAFKA-6075 Kafka cannot recover after an unclean shutdown on Windows As Vahid commented, Files.deleteIfExists(file.toPath) seems to destabilize Windows environment. This PR reverts to

[GitHub] kafka-site issue #103: MINOR: Fix typo in title

2017-10-25 Thread joel-hamill
Github user joel-hamill commented on the issue: https://github.com/apache/kafka-site/pull/103 @guozhangwang done https://github.com/apache/kafka/pull/4133 ---

[GitHub] kafka pull request #4133: Fix typo dev guide title

2017-10-25 Thread joel-hamill
GitHub user joel-hamill opened a pull request: https://github.com/apache/kafka/pull/4133 Fix typo dev guide title You can merge this pull request into a Git repository by running: $ git pull https://github.com/joel-hamill/kafka dev-guide-title Alternatively you can review

Re: [VOTE] KIP-205: Add all() and range() API to ReadOnlyWindowStore

2017-10-25 Thread Damian Guy
+1 On Tue, 24 Oct 2017 at 16:46 Guozhang Wang wrote: > +1. Thanks. > > On Mon, Oct 23, 2017 at 8:11 PM, Richard Yu > wrote: > > > Hi all, > > > > I want to propose KIP-205 for the addition of new API. It is about adding > > methods similar to

Re: [DISCUSS] KIP-209 Connection String Support

2017-10-25 Thread Clebert Suconic
> > This will cause a lot of compatibility problems, right? If I switch > back and forth between two Kafka versions, they will support slightly > different sets of configuration parameters. It seems saner to simply > ignore configuration parameters that we don't understand, like we do > now.

[jira] [Resolved] (KAFKA-6114) kafka Java API Consumer and producer Offset value comparison?

2017-10-25 Thread JIRA
[ https://issues.apache.org/jira/browse/KAFKA-6114?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sönke Liebau resolved KAFKA-6114. - Resolution: Invalid Assignee: Sönke Liebau > kafka Java API Consumer and producer Offset

Re: [DISCUSS] KIP-209 Connection String Support

2017-10-25 Thread Clebert Suconic
+1... I will update the KIP by the weekend... (I am taking this on my spare time.. although I can rush it if anyone needs it sooner). On Tue, Oct 24, 2017 at 12:27 PM, Colin McCabe wrote: > Hi Clebert, > > As some other people mentioned, a comma is probably not a great

RE: Use self contained tokens instead of ACL

2017-10-25 Thread Postmann, P. (Peter)
Hi Sönke, Thanks for the fast replay. We don’t want to use Kerberos since we want to do the authorization on Application level and without involvement of a 3rd party during runtime. -Original Message- From: Sönke Liebau [mailto:soenke.lie...@opencore.com.INVALID] Sent: Mittwoch, 25.

Re: [DISCUSS] KIP-204 : adding records deletion operation to the new Admin Client API

2017-10-25 Thread Paolo Patierno
Thanks for all your feedback guys. I have updated my current code as well. I know that the vote for this KIP is not started yet (actually I opened it due to no feedback on this KIP after a while but then the discussion started and it was really useful !) but I have already opened a PR for that.

[GitHub] kafka pull request #4132: KAFKA-5925: Adding records deletion operation to t...

2017-10-25 Thread ppatierno
GitHub user ppatierno opened a pull request: https://github.com/apache/kafka/pull/4132 KAFKA-5925: Adding records deletion operation to the new Admin Client API This is the PR related to the

Re: Use self contained tokens instead of ACL

2017-10-25 Thread Sönke Liebau
The concept you describe sounds similar to what Microsoft calls "claims based authorization". At a high level I should think that using Kerberos as a vehicle to transport the information would be the way to go, as it is established and already supported by Kafka. I believe tickets have a field

[jira] [Created] (KAFKA-6117) One Broker down can't rejoin the cluster

2017-10-25 Thread GangGu (JIRA)
GangGu created KAFKA-6117: - Summary: One Broker down can't rejoin the cluster Key: KAFKA-6117 URL: https://issues.apache.org/jira/browse/KAFKA-6117 Project: Kafka Issue Type: Bug Affects

Use self contained tokens instead of ACL

2017-10-25 Thread Postmann, P. (Peter)
Hi everyone, I´m working on a concept to use Kafka with self-contained tokens (instead of ACL). The idea: - A client requests access to a certain topic (in some kind of portal) - The owner of the topic approves the request (in some kind of portal) - The client

Re: [DISCUSS] KIP-179: Change ReassignPartitionsCommand to use AdminClient

2017-10-25 Thread Tom Bentley
If there are no further comments, I will start a vote on this next week. Thanks, Tom On 20 October 2017 at 08:33, Tom Bentley wrote: > Hi, > > I've made a fairly major update to KIP-179 to propose APIs for setting > throttled rates and throttled replicas with the

Re: Authorize to create a KIP page

2017-10-25 Thread Damian Guy
Hi Jan, You should have permissions now. Thanks, Damian On Wed, 25 Oct 2017 at 09:41 Jan Filipiak wrote: > Hello, > > to get KAFKA-3705 moving a little bit more I am thinking of starting a > KIP to discuss the finaly API design. > > My confluence id is: "jfilipiak"

[GitHub] kafka pull request #4131: Remove maven central repository, use only jcenter

2017-10-25 Thread yinonavraham
GitHub user yinonavraham opened a pull request: https://github.com/apache/kafka/pull/4131 Remove maven central repository, use only jcenter jcenter is a super set on top of maven central, so having both of those repositories is redundant, and the preferred one should be jcenter.

Re: [VOTE] KIP-201: Rationalising policy interfaces

2017-10-25 Thread Tom Bentley
It's been two weeks since I started the vote on this KIP and although there are two votes so far there are no binding votes. Any feedback from committers would be appreciated. Thanks, Tom On 12 October 2017 at 10:09, Edoardo Comar wrote: > Thanks Tom with the last additions

Authorize to create a KIP page

2017-10-25 Thread Jan Filipiak
Hello, to get KAFKA-3705 moving a little bit more I am thinking of starting a KIP to discuss the finaly API design. My confluence id is: "jfilipiak" Best Jan

Re: [DISCUSS] KIP-212: Enforce set of legal characters for connector names

2017-10-25 Thread Sönke Liebau
I've spent some time looking at this and testing various characters and it would appear that Randall's suspicion was spot on. I think we can support a fairly large set of characters with very minor changes. I was put of by the exceptions that were thrown when creating connectors with certain