RE: Use self contained tokens instead of ACL

2017-11-02 Thread Postmann, P. (Peter)
Manikumar, Sönke! Thanks, the Patch looks very promising. As far as I understood the token is stored in Zookeeper and when the clients reconnects or connects to another broker it uses the tokened and hmac for authentication. I think that’s an optimization. It could be not used and instead we

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

2017-11-02 Thread Paolo Patierno
Thanks Jason ! I have just updated the KIP with DeleteRecordsOptions definition. Paolo Patierno Senior Software Engineer (IoT) @ Red Hat Microsoft MVP on Azure & IoT Microsoft Azure Advisor Twitter : @ppatierno Linkedin : paolopatierno

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

2017-11-02 Thread Paolo Patierno
Hi Colin, I have just updated the KIP mentioning that this new method should replace the "legacy" Scala API used for deleting records today. Thanks, Paolo. Paolo Patierno Senior Software Engineer (IoT) @ Red Hat Microsoft MVP on Azure & IoT Microsoft Azure Advisor Twitter : @ppatierno

Re: Use self contained tokens instead of ACL

2017-11-02 Thread Manikumar
Hi, Yes, the token details and scram credentials are stored in Zookeeper and clients connects using tokenId and hmac for scram authentication. We plan to implement token storage as pluggable interface. Thanks, Manikumar On Thu, Nov 2, 2017 at 2:09 PM, Postmann, P. (Peter) < peter.postm...@ing.c

Re: [DISCUSS] KIP-218: Make KafkaFuture.Function java 8 lambda compatible

2017-11-02 Thread Tom Bentley
Hi Steven, I notice you've renamed the template's "Rejected Alternatives" section to "Other Alternatives", suggesting they're not rejected yet (or, if you have rejected them, I think you should give your reasons). Personally, I'd like to understand the arguments against simply replacing KafkaFutu

Re: [ANNOUNCE] Apache Kafka 1.0.0 Released

2017-11-02 Thread Paolo Patierno
Congratulations for this milestone ! Thanks to Gouzhang for running the release ! Paolo Patierno Senior Software Engineer (IoT) @ Red Hat Microsoft MVP on Azure & IoT Microsoft Azure Advisor Twitter : @ppatierno Linkedin : paolopatierno

Re: [ANNOUNCE] Apache Kafka 1.0.0 Released

2017-11-02 Thread Xin Wang
Great Job! - Xin 2017-11-02 18:30 GMT+08:00 Paolo Patierno : > Congratulations for this milestone ! > > > Thanks to Gouzhang for running the release ! > > > Paolo Patierno > Senior Software Engineer (IoT) @ Red Hat > Microsoft MVP on Azure & IoT > Microsoft Azure Advisor > > Twitter : @ppatierno

Re: [ANNOUNCE] Apache Kafka 1.0.0 Released

2017-11-02 Thread Eno Thereska
Congrats! Eno On Thu, Nov 2, 2017 at 10:55 AM, Xin Wang wrote: > Great Job! > > - Xin > > 2017-11-02 18:30 GMT+08:00 Paolo Patierno : > > > Congratulations for this milestone ! > > > > > > Thanks to Gouzhang for running the release ! > > > > > > Paolo Patierno > > Senior Software Engineer (IoT)

Re: [ANNOUNCE] Apache Kafka 1.0.0 Released

2017-11-02 Thread Mickael Maison
Great milestone ! Thanks for running this release. On Thu, Nov 2, 2017 at 11:10 AM, Eno Thereska wrote: > Congrats! > > Eno > > On Thu, Nov 2, 2017 at 10:55 AM, Xin Wang wrote: > >> Great Job! >> >> - Xin >> >> 2017-11-02 18:30 GMT+08:00 Paolo Patierno : >> >> > Congratulations for this mileston

Re: [ANNOUNCE] Apache Kafka 1.0.0 Released

2017-11-02 Thread Tom Bentley
Thanks Guozhang! On 2 November 2017 at 11:22, Mickael Maison wrote: > Great milestone ! Thanks for running this release. > > On Thu, Nov 2, 2017 at 11:10 AM, Eno Thereska > wrote: > > Congrats! > > > > Eno > > > > On Thu, Nov 2, 2017 at 10:55 AM, Xin Wang > wrote: > > > >> Great Job! > >> > >>

Re: [ANNOUNCE] Apache Kafka 1.0.0 Released

2017-11-02 Thread Ismael Juma
Thanks for running the release, Guozhang! Also thanks to all the contributors who made 1.0 possible. :) Ismael On 1 Nov 2017 2:27 pm, "Guozhang Wang" wrote: The Apache Kafka community is pleased to announce the release for Apache Kafka 1.0.0. This is a major release of the Kafka project, and i

Re: [ANNOUNCE] Apache Kafka 1.0.0 Released

2017-11-02 Thread Damian Guy
Thanks Guozhang! On Thu, 2 Nov 2017 at 11:42 Ismael Juma wrote: > Thanks for running the release, Guozhang! Also thanks to all the > contributors who made 1.0 possible. :) > > Ismael > > On 1 Nov 2017 2:27 pm, "Guozhang Wang" wrote: > > The Apache Kafka community is pleased to announce the rele

Re: [ANNOUNCE] Apache Kafka 1.0.0 Released

2017-11-02 Thread UMESH CHAUDHARY
Great news, Congratulations to the team ! On Thu, 2 Nov 2017 at 17:17 Damian Guy wrote: > Thanks Guozhang! > > On Thu, 2 Nov 2017 at 11:42 Ismael Juma wrote: > > > Thanks for running the release, Guozhang! Also thanks to all the > > contributors who made 1.0 possible. :) > > > > Ismael > > > >

Re: [ANNOUNCE] Apache Kafka 1.0.0 Released

2017-11-02 Thread Rajini Sivaram
Guozhang, Thank you for running the release! Regards, Rajini On Thu, Nov 2, 2017 at 12:07 PM, UMESH CHAUDHARY wrote: > Great news, Congratulations to the team ! > > On Thu, 2 Nov 2017 at 17:17 Damian Guy wrote: > > > Thanks Guozhang! > > > > On Thu, 2 Nov 2017 at 11:42 Ismael Juma wrote: >

[jira] [Created] (KAFKA-6158) CONSUMER-ID and HOST values are concatenated if the CONSUMER-ID is > 50 chars

2017-11-02 Thread Gustav Westling (JIRA)
Gustav Westling created KAFKA-6158: -- Summary: CONSUMER-ID and HOST values are concatenated if the CONSUMER-ID is > 50 chars Key: KAFKA-6158 URL: https://issues.apache.org/jira/browse/KAFKA-6158 Proje

Re: [ANNOUNCE] Apache Kafka 1.0.0 Released

2017-11-02 Thread Vahid S Hashemian
Great news. Thanks Guozhang! --Vahid From: Rajini Sivaram To: dev Date: 11/02/2017 05:37 AM Subject:Re: [ANNOUNCE] Apache Kafka 1.0.0 Released Guozhang, Thank you for running the release! Regards, Rajini On Thu, Nov 2, 2017 at 12:07 PM, UMESH CHAUDHARY wrote: > Great

Re: [VOTE] KIP-210: Provide for custom error handling when Kafka Streams fails to produce

2017-11-02 Thread Garret Thompson
+1 (non-binding) Thanks, Matt!

Re: [VOTE] KIP-210: Provide for custom error handling when Kafka Streams fails to produce

2017-11-02 Thread Ted Yu
+1 On Wed, Nov 1, 2017 at 4:50 PM, Guozhang Wang wrote: > +1 (binding) from me. Thanks! > > On Wed, Nov 1, 2017 at 4:50 PM, Guozhang Wang wrote: > > > The vote should stay open for at least 72 hours. The bylaws can be found > > here https://cwiki.apache.org/confluence/display/KAFKA/Bylaws > > >

Re: [DISCUSS] KIP-217: Expose a timeout to allow an expired ZK session to be re-created

2017-11-02 Thread Ted Yu
ZOOKEEPER-2184 is scheduled for 3.4.12 whose release is unknown. I think adding the session recreation on Kafka side should benefit Kafka users, especially those who don't plan to move to 3.4.12+ in the near future. On Wed, Nov 1, 2017 at 6:34 PM, Jun Rao wrote: > Hi, Stephane, > > 3) The diffe

[jira] [Created] (KAFKA-6159) Link to upgrade docs in 100 release notes is broken

2017-11-02 Thread JIRA
Martin Schröder created KAFKA-6159: -- Summary: Link to upgrade docs in 100 release notes is broken Key: KAFKA-6159 URL: https://issues.apache.org/jira/browse/KAFKA-6159 Project: Kafka Issue T

[jira] [Resolved] (KAFKA-6137) RestoreIntegrationTest sometimes fails with assertion error

2017-11-02 Thread Ted Yu (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6137?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ted Yu resolved KAFKA-6137. --- Resolution: Cannot Reproduce > RestoreIntegrationTest sometimes fails with assertion error > -

Re: [ANNOUNCE] Apache Kafka 1.0.0 Released

2017-11-02 Thread Becket Qin
Great news! Thanks for running the release, Guozhang! On Thu, Nov 2, 2017 at 8:19 AM, Vahid S Hashemian wrote: > Great news. Thanks Guozhang! > > --Vahid > > > > > From: Rajini Sivaram > To: dev > Date: 11/02/2017 05:37 AM > Subject:Re: [ANNOUNCE] Apache Kafka 1.0.0 Released >

[jira] [Created] (KAFKA-6160) expose partitions under min isr in kafka-topics.sh command

2017-11-02 Thread Mitchell (JIRA)
Mitchell created KAFKA-6160: --- Summary: expose partitions under min isr in kafka-topics.sh command Key: KAFKA-6160 URL: https://issues.apache.org/jira/browse/KAFKA-6160 Project: Kafka Issue Type: Im

[jira] [Created] (KAFKA-6161) Make configure() and close() empty by default in serde interfaces

2017-11-02 Thread Evgeny Veretennikov (JIRA)
Evgeny Veretennikov created KAFKA-6161: -- Summary: Make configure() and close() empty by default in serde interfaces Key: KAFKA-6161 URL: https://issues.apache.org/jira/browse/KAFKA-6161 Project:

Re: [DISCUSS] KIP-218: Make KafkaFuture.Function java 8 lambda compatible

2017-11-02 Thread Steven Aerts
Hi Tom, Nice observation. I changed "Rejected Alternatives" section to "Other Alternatives", as I see myself as too much of an outsider to the kafka community to be able to decide without this discussion. I see two major factors to decide: - how soon will KIP-118 (drop support of java 7) be impl

[GitHub] kafka pull request #4171: MINOR: Change version format in release notes pyth...

2017-11-02 Thread guozhangwang
GitHub user guozhangwang opened a pull request: https://github.com/apache/kafka/pull/4171 MINOR: Change version format in release notes python code @ijuma @ewencp You can merge this pull request into a Git repository by running: $ git pull https://github.com/guozhangwang/kafka

[jira] [Created] (KAFKA-6162) Stream Store tries to create directory with invalid name on Windows

2017-11-02 Thread Nitzan Niv (JIRA)
Nitzan Niv created KAFKA-6162: - Summary: Stream Store tries to create directory with invalid name on Windows Key: KAFKA-6162 URL: https://issues.apache.org/jira/browse/KAFKA-6162 Project: Kafka

[GitHub] kafka pull request #4172: MINOR: Remove clients/out directory

2017-11-02 Thread ijuma
GitHub user ijuma opened a pull request: https://github.com/apache/kafka/pull/4172 MINOR: Remove clients/out directory It was committed inadvertently. You can merge this pull request into a Git repository by running: $ git pull https://github.com/ijuma/kafka remove-out-folder

Kafka- Requesting for help

2017-11-02 Thread Suraj Ramesh
Hi, I would request to help me with this issue. I should not commit offset when any exception comes while processing a message. I am using below approach to manually comit offset. Can you please help me in getting uncommited offsets to re-process them in later point of time. import org.apache.k

Re: [DISCUSS] KIP-217: Expose a timeout to allow an expired ZK session to be re-created

2017-11-02 Thread Jun Rao
Stephane, Jeff, Another option is to not expose the reconnect timeout config and just retry the creation of Zookeeper forever. This is an improvement from the current situation and if zookeeper-2184 is fixed in the future, we don't need to deprecate the config. Thanks, Jun On Thu, Nov 2, 2017 a

Re: [DISCUSS] KIP-217: Expose a timeout to allow an expired ZK session to be re-created

2017-11-02 Thread Stephane Maarek
Hi Jun I think this is a better option. Would that change require a kip then as it's not a change in public API ? @ted it was marked as a blocked for 3.4.11 but they pushed it. It seems that the owner of the pr hasn't acted in over a year and I think someone needs to take ownership of that. Addit

[GitHub] kafka pull request #4163: MINOR: build.gradle: sourceCompatibility, targetCo...

2017-11-02 Thread cmccabe
Github user cmccabe closed the pull request at: https://github.com/apache/kafka/pull/4163 ---

Re: [DISCUSS] KIP-217: Expose a timeout to allow an expired ZK session to be re-created

2017-11-02 Thread Ted Yu
Stephane: bq. hasn't acted in over a year The above fact implies some reluctance from the zookeeper community to fully solve the issue (maybe due to technical issues). Anyway, we should plan on not relying on the fix to go through in the near future. As for Jun's latest suggestion, I think we sho

[jira] [Created] (KAFKA-6163) Broker should fail fast on startup if an error occurs while loading logs

2017-11-02 Thread JIRA
Xavier Léauté created KAFKA-6163: Summary: Broker should fail fast on startup if an error occurs while loading logs Key: KAFKA-6163 URL: https://issues.apache.org/jira/browse/KAFKA-6163 Project: Kafka

[GitHub] kafka pull request #4172: MINOR: Remove clients/out directory

2017-11-02 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/4172 ---

Build failed in Jenkins: kafka-trunk-jdk9 #165

2017-11-02 Thread Apache Jenkins Server
See Changes: [rajinisivaram] MINOR: Remove clients/out directory -- Started by an SCM change [EnvInject] - Loading node environment variables. Building remotely on H25 (cou

[jira] [Created] (KAFKA-6164) ClientQuotaManager threads prevent shutdown when encountering an error loading logs

2017-11-02 Thread JIRA
Xavier Léauté created KAFKA-6164: Summary: ClientQuotaManager threads prevent shutdown when encountering an error loading logs Key: KAFKA-6164 URL: https://issues.apache.org/jira/browse/KAFKA-6164 Pro

Re: [DISCUSS] KIP-217: Expose a timeout to allow an expired ZK session to be re-created

2017-11-02 Thread Ted Yu
The following JIRA provides some background on why upgrading immediately following new release may not be prudent (though I expect this to be rare): ZOOKEEPER-2347 On Thu, Nov 2, 2017 at 3:00 PM, Ted Yu wrote: > Stephane: > bq. hasn't acted in over a year > > The above fact implies some relucta

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

2017-11-02 Thread Apache Jenkins Server
See Changes: [rajinisivaram] MINOR: Remove clients/out directory -- [...truncated 381.87 KB...] kafka.api.TransactionsTest > testFencingOnSendOffsets STARTED kafka.api.T

[GitHub] kafka pull request #4171: MINOR: Change version format in release notes pyth...

2017-11-02 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/4171 ---

[GitHub] kafka pull request #4168: MINOR: update producer client request timeout in s...

2017-11-02 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/4168 ---

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

2017-11-02 Thread Apache Jenkins Server
See

Re: [DISCUSS] KIP-217: Expose a timeout to allow an expired ZK session to be re-created

2017-11-02 Thread Jeff Widman
+1 for permanent retry under the covers (without an exposed/later deprecated config). That said, I understand the reality that sometimes we have to workaround an unfixed issue in another project, so if you think best to expose a config, then I have no objections. Mainly I wanted to make sure you'd

kafka-pr-jdk9-scala2.12 keeps failing

2017-11-02 Thread Ted Yu
Hi, I took a look at recent runs under https://builds.apache. org/job/kafka-pr-jdk9-scala2.12 All the recent runs failed with: Could not update commit status of the Pull Request on GitHub. org.kohsuke.github.HttpException: Server returned HTTP response code: 201, message: 'Created' for URL: https

Re: kafka-pr-jdk9-scala2.12 keeps failing

2017-11-02 Thread Guozhang Wang
Noticed that as well, could we track down to which git commit / version upgrade caused the issue? Guozhang On Thu, Nov 2, 2017 at 6:25 PM, Ted Yu wrote: > Hi, > I took a look at recent runs under https://builds.apache. > org/job/kafka-pr-jdk9-scala2.12 > > All the recent runs failed with: > >

Build failed in Jenkins: kafka-trunk-jdk9 #166

2017-11-02 Thread Apache Jenkins Server
See Changes: [wangguoz] MINOR: Change version format in release notes python code [wangguoz] MINOR: update producer client request timeout in system test -- Started by an

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

2017-11-02 Thread Apache Jenkins Server
See

Re: kafka-pr-jdk9-scala2.12 keeps failing

2017-11-02 Thread Ted Yu
Looking at earlier runs, e.g. : https://builds.apache.org/job/kafka-pr-jdk9-scala2.12/2384/console FAILURE: Build failed with an exception. * What went wrong: Could not determine java version from '9.0.1'. This was the first build with 'out of range of int' exception: https://builds.apache.or

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

2017-11-02 Thread Apache Jenkins Server
See Changes: [wangguoz] MINOR: update producer client request timeout in system test -- [...truncated 380.55 KB...] kafka.server.epoch.LeaderEpochFileCacheTest > shouldNo

[GitHub] kafka pull request #4173: KAFKA-6156: Metric tag name should not contain col...

2017-11-02 Thread huxihx
GitHub user huxihx opened a pull request: https://github.com/apache/kafka/pull/4173 KAFKA-6156: Metric tag name should not contain colons. Windows directory paths often contain colons which are now allowed in yammer metrics. Should convert to its corresponding Unix style path before

[jira] [Created] (KAFKA-6165) Kafka Brokers goes down with outOfMemoryError.

2017-11-02 Thread kaushik srinivas (JIRA)
kaushik srinivas created KAFKA-6165: --- Summary: Kafka Brokers goes down with outOfMemoryError. Key: KAFKA-6165 URL: https://issues.apache.org/jira/browse/KAFKA-6165 Project: Kafka Issue Type

Re: kafka-pr-jdk9-scala2.12 keeps failing

2017-11-02 Thread Ismael Juma
This looks to be an issue in Jenkins, not in Kafka. Apache Infra updated Java 9 to 9.0.1 and it seems to have broken some of the Jenkins code. Ismael On 3 Nov 2017 1:53 am, "Ted Yu" wrote: > Looking at earlier runs, e.g. : > https://builds.apache.org/job/kafka-pr-jdk9-scala2.12/2384/console > >