Re: [DISCUSSION] KIP-418: A method-chaining way to branch KStream

2019-03-27 Thread Paul Whalen
Ivan, I tried to make a very rough proof of concept of a fluent API based off of KStream here (https://github.com/apache/kafka/pull/6512), and I think I succeeded at removing both cons. - Compatibility: I was incorrect earlier about compatibility issues, there aren't any direct ones. I

Re: MirrorMaker 2.0 and Streams interplay (topic naming control)

2019-03-27 Thread Paul Whalen
John, You make a good case for it already being a public API, so my nerves are definitely eased on that front. I do think we have a path to move forward with the user space solution, and if I get a chance, I'm going to try proving it out with a trivial demo using an early MM2 build - but it's

Re: [ANNOUNCE] Apache Kafka 2.2.0

2019-03-27 Thread Guozhang Wang
Hello Mickael, Thanks for reporting this, I double checked the release process and it is indeed mentioned: https://cwiki.apache.org/confluence/display/KAFKA/Release+Process @Matthias J Sax : I've copied the javadocs from release repo to kafka-site github and it should have fixed it now.

[jira] [Resolved] (KAFKA-8166) Kafka 2.2 Javadoc broken

2019-03-27 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-8166?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-8166. -- Resolution: Fixed Assignee: Guozhang Wang Fix Version/s: 2.2.0 > Kafka 2.2

Re: [DISCUSS] KIP-433: Provide client API version to authorizer

2019-03-27 Thread Colin McCabe
Thanks, Ying Zheng. Looks good overall. One question is, should the version be specified as a Kafka version rather than as a RPC API version? I don't think most users are aware of RPC versions, but something like "min kafka version" would be easier to understand. That is how we handle the

Re: [VOTE] KIP-422: Use the default value of max.poll.interval in Streams

2019-03-27 Thread John Roesler
Ah, good point, Guozhang. I'll remove that mention from the KIP. On Wed, Mar 27, 2019 at 3:30 PM Bill Bejeck wrote: > +1 for me, > > Thanks, > Bill > > On Wed, Mar 27, 2019 at 4:13 PM Guozhang Wang wrote: > > > +1 from me. > > > > Though note that we cannot make such changes in older versions

[jira] [Created] (KAFKA-8168) Add a generated ApiMessageType class

2019-03-27 Thread Colin P. McCabe (JIRA)
Colin P. McCabe created KAFKA-8168: -- Summary: Add a generated ApiMessageType class Key: KAFKA-8168 URL: https://issues.apache.org/jira/browse/KAFKA-8168 Project: Kafka Issue Type:

Re: [VOTE] KIP-422: Use the default value of max.poll.interval in Streams

2019-03-27 Thread Bill Bejeck
+1 for me, Thanks, Bill On Wed, Mar 27, 2019 at 4:13 PM Guozhang Wang wrote: > +1 from me. > > Though note that we cannot make such changes in older versions since even > if we release new versions out of those branches they are considered > bug-fix only and hence should not have any interface

[jira] [Resolved] (KAFKA-1149) Please delete old releases from mirroring system

2019-03-27 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1149?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-1149. -- Resolution: Fixed > Please delete old releases from mirroring system >

Re: [VOTE] KIP-422: Use the default value of max.poll.interval in Streams

2019-03-27 Thread Guozhang Wang
+1 from me. Though note that we cannot make such changes in older versions since even if we release new versions out of those branches they are considered bug-fix only and hence should not have any interface impacting changes. Guozhang On Wed, Mar 27, 2019 at 12:55 PM John Roesler wrote: >

[jira] [Created] (KAFKA-8167) Document named stateful operators

2019-03-27 Thread John Roesler (JIRA)
John Roesler created KAFKA-8167: --- Summary: Document named stateful operators Key: KAFKA-8167 URL: https://issues.apache.org/jira/browse/KAFKA-8167 Project: Kafka Issue Type: Task

[VOTE] KIP-422: Use the default value of max.poll.interval in Streams

2019-03-27 Thread John Roesler
Hi all, Since the KIP is so small, I'm going to optimistically start the vote for KIP-422 to remove our "max int" default max.poll.interval.ms in Streams and fall back to the Consumer default of five minutes.

Re: [DISCUSS] KIP-422: Return to default max poll interval in Streams

2019-03-27 Thread John Roesler
Since this KIP is so small in scope, I'll optimistically start the vote (in a separate thread). If there are objections, we can return to the discussion. Thanks, -John On Wed, Mar 27, 2019 at 2:31 PM John Roesler wrote: > Thanks for the response, Guozhang. > > Oh, I should have mentioned that.

Re: [DISCUSS] KIP-422: Return to default max poll interval in Streams

2019-03-27 Thread John Roesler
Thanks for the response, Guozhang. Oh, I should have mentioned that. I was thinking that we might as well make this change in 1.0+, not just in trunk. I'll update the KIP. Thanks, -John On Wed, Mar 27, 2019 at 12:30 PM Guozhang Wang wrote: > This is a reasonable one and we should probably do

Re: MirrorMaker 2.0 and Streams interplay (topic naming control)

2019-03-27 Thread John Roesler
Hi Paul, Sorry for overlooking the "offset translation" MM2 feature. I'm glad Ryanne was able to confirm this would work. I'm just one voice, but FWIW, I think that the internal topic naming scheme is a public API. We document the structure of the naming scheme in several places. We also

[jira] [Created] (KAFKA-8166) Kafka 2.2 Javadoc broken

2019-03-27 Thread Sachin NS (JIRA)
Sachin NS created KAFKA-8166: Summary: Kafka 2.2 Javadoc broken Key: KAFKA-8166 URL: https://issues.apache.org/jira/browse/KAFKA-8166 Project: Kafka Issue Type: Bug Components:

[jira] [Created] (KAFKA-8165) Streams task causes Out Of Memory after connection and store restoration

2019-03-27 Thread Di Campo (JIRA)
Di Campo created KAFKA-8165: --- Summary: Streams task causes Out Of Memory after connection and store restoration Key: KAFKA-8165 URL: https://issues.apache.org/jira/browse/KAFKA-8165 Project: Kafka

Re: [DISCUSS] KIP-401 TransformerSupplier/ProcessorSupplier enhancements

2019-03-27 Thread Guozhang Wang
Hello Paul, Thanks for the uploaded PR and the detailed description! I've made a pass on it and left some comments. Overall I think I agree with you that passing in the storebuilder directly that store name is more convienent as it does not require another `addStore` call, but we just need to

Re: [DISCUSS] KIP-422: Return to default max poll interval in Streams

2019-03-27 Thread Guozhang Wang
This is a reasonable one and we should probably do it post 1.0 already. I'm +1 (binding) on this. Guozhang On Wed, Mar 27, 2019 at 10:23 AM John Roesler wrote: > Hi all, > > I'd like to propose KIP-422 to remove our "max int" default > max.poll.interval.ms in Streams and fall back to the

[DISCUSS] KIP-422: Return to default max poll interval in Streams

2019-03-27 Thread John Roesler
Hi all, I'd like to propose KIP-422 to remove our "max int" default max.poll.interval.ms in Streams and fall back to the Consumer default of five minutes. https://cwiki.apache.org/confluence/display/KAFKA/KIP-442%3A+Return+to+default+max+poll+interval+in+Streams Permalink:

Re: [DISCUSS] KIP-439: Deprecate Interface WindowStoreIterator

2019-03-27 Thread Guozhang Wang
Hello Matthias, Just to clarify the naming conflicts is between the newly added function and the old functions that we want to deprecate / remove right? The existing ones have different signatures with parameters so that they should not have conflicts. I was thinking about just make the change

Re: [DISCUSSION] KIP-418: A method-chaining way to branch KStream

2019-03-27 Thread Ivan Ponomarev
OK, let me summarize what we have discussed up to this point. First, it seems that it's commonly agreed that branch API needs improvement. Motivation is given in the KIP. There are two potential ways to do it: 1. (as origianlly proposed) new KafkaStreamsBrancher<..>() .branch(predicate1,

Re: [VOTE] KIP-392: Allow consumers to fetch from the closest replica

2019-03-27 Thread Jason Gustafson
@Jun Re; 200: It's a fair point that it is useful to minimize the client changes that are needed to get a benefit from affinity. I think the high level argument that this is mostly the concern of operators and should be under their control. Since there is a protocol bump here, users will have to

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2019-03-27 Thread Harsha
Hi All, Thanks for your initial feedback. We updated the KIP. Please take a look and let us know if you have any questions. https://cwiki.apache.org/confluence/display/KAFKA/KIP-405%3A+Kafka+Tiered+Storage Thanks, Harsha On Wed, Feb 6, 2019, at 10:30 AM, Harsha wrote: > Thanks Eno,

Re: [VOTE] KIP-392: Allow consumers to fetch from the closest replica

2019-03-27 Thread Jun Rao
Hi, Guozhang, In general, users may want to optimize affinity in different ways, e.g. latency, cost, etc. I am not sure if all those cases can by captured by client IP addresses. So, it seems that having a rack.id in the consumer is still potentially useful. Thanks, Jun On Wed, Mar 27, 2019 at

Re: [VOTE] KIP-392: Allow consumers to fetch from the closest replica

2019-03-27 Thread Guozhang Wang
Hello Jun, Regarding 200: if we assume that most client would not bother setting rack.id at all and affinity can be determined w/o rack.id via TCP header, plus rack.id may not be "future-proof" additional information is needed as well, then do we still need to change the protocol of metadata

Re: Speeding up integration tests

2019-03-27 Thread Viktor Somogyi-Vass
Hi All, I've created a PR for what we have internally for retrying flaky tests. Any reviews and ideas are welcome: https://github.com/apache/kafka/pull/6506 It's basically collects the failed classes and reruns them at the end. If they successful it overwrites the test report. Thanks, Viktor On

Re: [ANNOUNCE] Apache Kafka 2.2.0

2019-03-27 Thread Mickael Maison
Thanks Matthias for running this release! The links to the 2.2 javadocs are broken: https://kafka.apache.org/22/javadoc/index.html The same happened for 2.1.0 (https://lists.apache.org/thread.html/57f2940225bcce36c3a01ec524dc967e81cb20e159d9b1c851b712f4@%3Cdev.kafka.apache.org%3E) so we probably

[jira] [Created] (KAFKA-8164) Improve test passing rate by rerunning flaky tests

2019-03-27 Thread Viktor Somogyi-Vass (JIRA)
Viktor Somogyi-Vass created KAFKA-8164: -- Summary: Improve test passing rate by rerunning flaky tests Key: KAFKA-8164 URL: https://issues.apache.org/jira/browse/KAFKA-8164 Project: Kafka

Re: Failing test on PR

2019-03-27 Thread Viktor Somogyi-Vass
Hi Mateusz, Just write a comment that only says "retest this please" (without the quotation marks). Best, Viktor On Wed, Mar 27, 2019 at 1:02 PM Mateusz Zakarczemny wrote: > Hi, > I'm working on https://github.com/apache/kafka/pull/4807 PR. Last PR build > failed in some random place. My

[jira] [Created] (KAFKA-8163) SetSchemaMetadata SMT does not apply to nested types

2019-03-27 Thread pierre bouvret (JIRA)
pierre bouvret created KAFKA-8163: - Summary: SetSchemaMetadata SMT does not apply to nested types Key: KAFKA-8163 URL: https://issues.apache.org/jira/browse/KAFKA-8163 Project: Kafka Issue

[jira] [Created] (KAFKA-8162) IBM JDK Class not found error when handling SASL authentication exception

2019-03-27 Thread Arkadiusz Firus (JIRA)
Arkadiusz Firus created KAFKA-8162: -- Summary: IBM JDK Class not found error when handling SASL authentication exception Key: KAFKA-8162 URL: https://issues.apache.org/jira/browse/KAFKA-8162 Project:

Failing test on PR

2019-03-27 Thread Mateusz Zakarczemny
Hi, I'm working on https://github.com/apache/kafka/pull/4807 PR. Last PR build failed in some random place. My changes are related to console consumer and the job failed in *kafka.api.ConsumerBounceTest.testRollingBrokerRestartsWithSmallerMaxGroupSizeConfigDisruptsBigGroup* I'm assuming it's some

[jira] [Resolved] (KAFKA-8160) To add ACL with SSL authentication

2019-03-27 Thread JIRA
[ https://issues.apache.org/jira/browse/KAFKA-8160?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sönke Liebau resolved KAFKA-8160. - Resolution: Information Provided Hi [~suseem...@gmail.com]   you can absolutely use SSL based

[jira] [Created] (KAFKA-8161) config conflict when run script bin/kafka-configs.sh --entity-type topics --entity-name flink-mirror-test1 --zookeeper 10.39.0.54:2181/kafka102_test --add-config 'foll

2019-03-27 Thread Haiping (JIRA)
Haiping created KAFKA-8161: -- Summary: config conflict when run script bin/kafka-configs.sh --entity-type topics --entity-name flink-mirror-test1 --zookeeper 10.39.0.54:2181/kafka102_test --add-config

Re: [ANNOUNCE] Apache Kafka 2.2.0

2019-03-27 Thread Sanjeev Kumar
Congratulations for the 2.2.0 release! Regards, Sanjeev On Wed, Mar 27, 2019 at 6:43 AM Guozhang Wang wrote: > Thanks Matthias for the release! > > On Tue, Mar 26, 2019 at 9:31 PM Dongjin Lee wrote: > > > Congratulations and thanks for your great work, Matthias!! > > > > Best, > > Dongjin > >