Re: [VOTE] KIP-354 Time-based log compaction policy

2018-09-04 Thread Brett Rann
> That's a fair point. We should make 0 = disable, to be consistent with the other settings. -1 is used elsewhere for disable and when seeing it in a config it's clear that it's a special meaning. 0 doesn't have to mean instant, it just means as quickly as possible. I don't think 0 is intuitive fo

Re: [VOTE] KIP-369 Alternative Partitioner to Support "Always Round-Robin" Selection

2018-09-04 Thread Abhimanyu Nagrath
+1 On Wed, Sep 5, 2018 at 2:39 AM Magesh Nandakumar wrote: > +1 ( non-binding) > > On Tue, Sep 4, 2018 at 3:39 AM M. Manna wrote: > > > Hello, > > > > I have made necessary changes as per the original discussion thread, and > > would like to put it for votes. > > > > Thank you very much for you

Re: [VOTE] KIP-354 Time-based log compaction policy

2018-09-04 Thread xiongqi wu
Thanks for comments. Today, when creating topic, client only does simple local validation and doesn't check against broker's configurations. We cannot just let users to create a configuration in zookeeper and dishonor the user's choice in broker side. I agree we need a better way to enforce the

Re: [DISCUSS] KIP 368: Allow SASL Connections to Periodically Re-Authenticate

2018-09-04 Thread Colin McCabe
On Tue, Sep 4, 2018, at 17:43, Ron Dagostino wrote: > Hi Colin. Different organizations will rely on different token lifetimes, > but anything shorter than an hour feels like it would be pretty > aggressive. An hour or more will probably be most common. Thanks. That's helpful to give me a sense

Re: [VOTE] KIP-354 Time-based log compaction policy

2018-09-04 Thread Colin McCabe
On Tue, Sep 4, 2018, at 17:47, xiongqi wu wrote: > Colin, > Thank you for comments. > see my inline reply below. > > Xiongqi (Wesley) Wu > > > On Tue, Sep 4, 2018 at 5:24 PM Colin McCabe wrote: > > > Hi Xiongqi, > > > > Thanks for this KIP. > > > > The name seems a bit ambiguous. Our compacti

Re: [VOTE] KIP-354 Time-based log compaction policy

2018-09-04 Thread Colin McCabe
On Tue, Sep 4, 2018, at 18:09, xiongqi wu wrote: > Colin, > > I will keep the title for now, since all the previous discussions and links > are tied to this title. > > I can change the title at the end or add a clarification note in the doc. Hi Xiongqi, It's fine to keep the email thread titles

Re: [VOTE] KIP-354 Time-based log compaction policy

2018-09-04 Thread xiongqi wu
Colin, I will keep the title for now, since all the previous discussions and links are tied to this title. I can change the title at the end or add a clarification note in the doc. Xiongqi (Wesley) Wu On Tue, Sep 4, 2018 at 5:47 PM xiongqi wu wrote: > Colin, > Thank you for comments. > see m

Re: [DISCUSS] KIP 368: Allow SASL Connections to Periodically Re-Authenticate

2018-09-04 Thread Ron Dagostino
<<< It works for authentication because at that point we know that nobody << wrote: > Hi Ron, > > Thanks for the response. > > Retries: If we reuse authentication code path for re-authentication, we > might want to treat failures in the same way without retries, but for now, > lets leave it in. >

Re: [VOTE] KIP-354 Time-based log compaction policy

2018-09-04 Thread xiongqi wu
Colin, Thank you for comments. see my inline reply below. Xiongqi (Wesley) Wu On Tue, Sep 4, 2018 at 5:24 PM Colin McCabe wrote: > Hi Xiongqi, > > Thanks for this KIP. > > The name seems a bit ambiguous. Our compaction policies are already > time-based, after all. It seems like this change i

Re: [DISCUSS] KIP-347: Enable batching in FindCoordinatorRequest

2018-09-04 Thread Colin McCabe
Hi Yishun, I agree with Guozhang. NetworkClient is the wrong place to put things which are specific to a particular message type. NetworkClient should not have to care what the type of the message is that it is sending. Adding type-specific handling is much more "ugly and dirty" than adding s

Re: [DISCUSS] KIP 368: Allow SASL Connections to Periodically Re-Authenticate

2018-09-04 Thread Ron Dagostino
Hi Colin. Different organizations will rely on different token lifetimes, but anything shorter than an hour feels like it would be pretty aggressive. An hour or more will probably be most common. << wrote: > Hi Ron, > > Thanks for the KIP. > > What is the frequency at which you envision bearer

Re: [DISCUSS] KIP-349 Priorities for Source Topics

2018-09-04 Thread nick
> On Sep 4, 2018, at 4:20 PM, Jan Filipiak wrote: > > what I meant is litterally this interface: > > https://samza.apache.org/learn/documentation/0.7.0/api/javadocs/org/apache/samza/system/chooser/MessageChooser.html > >

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

2018-09-04 Thread Matthias J. Sax
Hi, I am just catching up on this thread. I did not read everything so far, but want to share couple of initial thoughts: Headers: I think there is a fundamental difference between header usage in this KIP and KP-258. For 258, we add headers to changelog topic that are owned by Kafka Streams an

Re: [DISCUSS] KIP-297: Externalizing Secrets for Connect Configurations

2018-09-04 Thread Colin McCabe
Hi Robert, This seems like a reasonable behavior to me. However, adding this behavior to FileConfigProvider seems like it might give someone who accidentally configures a directory rather than a file a nasty surprise. How about adding a DirectoryConfigProvider which adds this behavior? best,

Re: [DISCUSS] KIP 368: Allow SASL Connections to Periodically Re-Authenticate

2018-09-04 Thread Colin McCabe
Hi Ron, Thanks for the KIP. What is the frequency at which you envision bearer tokens changing at? Did you consider the alternate solution of terminating connections when the bearer token changed? best, Colin On Tue, Aug 28, 2018, at 07:28, Ron Dagostino wrote: > Hi everyone. I created KIP 3

Re: [VOTE] KIP-354 Time-based log compaction policy

2018-09-04 Thread Colin McCabe
Hi Xiongqi, Thanks for this KIP. The name seems a bit ambiguous. Our compaction policies are already time-based, after all. It seems like this change is focused around adding a “max.compaction.lag.ms." Perhaps the KIP title should be something like "add maximum compaction lag time"? > The

Provide clients Header implementation without accessing internal package

2018-09-04 Thread Jungtaek Lim
Hi Kafka devs, I just noticed that the only implementation for org.apache.kafka.common.header.Header is RecordHeader which is exposed to the 'internal' package, and some of end users just picked the class because they don't want to create their own. IMHO it can give false signal for end users on t

Re: [DISCUSS] KIP-158: Kafka Connect should allow source connectors to set topic-specific settings for new topics

2018-09-04 Thread Randall Hauch
Okay, I think I cleaned up the formatting issues in the KIP wiki page. And while implementing I realized that it'd be helpful to be able to limit via the connector configuration and the rules which topics are created. I added the `topic.creation.${ruleName}.policy` behavior, with possible values of

[VOTE] KIP-354 Time-based log compaction policy

2018-09-04 Thread xiongqi wu
Let's VOTE for this KIP. KIP: https://cwiki.apache.org/confluence/display/KAFKA/KIP-354 %3A+Time-based+log+compaction+policy Implementation: https://github.com/apache/kafka/pull/5611 Xiongqi (Wesley) Wu

[jira] [Created] (KAFKA-7377) update metrics module from yammer to dropwizrd

2018-09-04 Thread RAJKUMAR NATARAJAN (JIRA)
RAJKUMAR NATARAJAN created KAFKA-7377: - Summary: update metrics module from yammer to dropwizrd Key: KAFKA-7377 URL: https://issues.apache.org/jira/browse/KAFKA-7377 Project: Kafka Issue

Re: [DISCUSS] KIP 368: Allow SASL Connections to Periodically Re-Authenticate

2018-09-04 Thread Rajini Sivaram
Hi Ron, Thanks for the response. Retries: If we reuse authentication code path for re-authentication, we might want to treat failures in the same way without retries, but for now, lets leave it in. Implementation: I think we are on the same page. *I believe Streams and Connect are taken care of

Re: [VOTE] KIP-369 Alternative Partitioner to Support "Always Round-Robin" Selection

2018-09-04 Thread Magesh Nandakumar
+1 ( non-binding) On Tue, Sep 4, 2018 at 3:39 AM M. Manna wrote: > Hello, > > I have made necessary changes as per the original discussion thread, and > would like to put it for votes. > > Thank you very much for your suggestion and guidance so far. > > Regards, >

Re: [DISCUSS] KIP-297: Externalizing Secrets for Connect Configurations

2018-09-04 Thread Robert Yokota
Hi everyone, Currently the FileConfigProvider, when passed a path that represents a Properties file, will read the file as a set of key-value pairs. I've filed https://issues.apache.org/jira/browse/KAFKA-7370, which proposes to augment FileConfigProvider so that when a path representing a directo

[jira] [Resolved] (KAFKA-7211) MM should handle timeouts in commitSync

2018-09-04 Thread Dong Lin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7211?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Dong Lin resolved KAFKA-7211. - Resolution: Fixed > MM should handle timeouts in commitSync > --- > >

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

2018-09-04 Thread Jan Filipiak
Just on remark here. The high-watermark could be disregarded. The decision about the forward depends on the size of the aggregated map. Only 1 element long maps would be unpacked and forwarded. 0 element maps would be published as delete. Any other count of map entries is in "waiting for correc

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

2018-09-04 Thread Jan Filipiak
I was completely brain-dead in my mail before. Completly missed that you already repartition back for the user and only apply the high water mark filtering after the second repartition source. I missed the sink / source bounce while poking, sorry for the confusion. Yes a group by can give the s

Re: [DISCUSS] KIP-347: Enable batching in FindCoordinatorRequest

2018-09-04 Thread Guozhang Wang
Hello Yishun, I reviewed the latest wiki page, and noticed that the special handling logic needs to be in the NetworkClient. Comparing it with another alternative way, i.e. we add the fall-back logic in the AdminClient, as well as in the ConsumerClient to capture the UnsupportedException and fall

Re: [DISCUSS] KIP-349 Priorities for Source Topics

2018-09-04 Thread Jan Filipiak
Hi Nick, sorry for not beeing so helpfull. I don't quite understand what _this_ would be in your email. Is this the part in question? /interface TopicPrioritizer { List prioritize(List topicPriorities); } // //public void registerTopicPrioritizer(TopicPrioritizer topicPrioritizer);// / this

Re: [EXTERNAL] [VOTE] KIP-310: Add a Kafka Source Connector to Kafka Connect

2018-09-04 Thread McCaig, Rhys
Bumping this thread. > On Aug 14, 2018, at 11:00 AM, McCaig, Rhys wrote: > > Bumping this thread. Looking for some binding votes or further request for > discussion. > >> On Aug 10, 2018, at 12:38 PM, McCaig, Rhys wrote: >> >> Thanks Stephane! >> >> If there is a desire for further discuss

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

2018-09-04 Thread Adam Bellemare
Yep, I definitely misunderstood some of the groupBy and groupByKey functionality. I would say disregard what I said in my previous email w.r.t. my assumptions about record size. I was looking into the code more today and I did not understand it correctly the first time I read it. It does look like

Re: Add to contributor list

2018-09-04 Thread Guozhang Wang
Hi Murali, Note you do not need PMC member to send you the invitation for becoming a contributor; you only need it to be a committer. All you need to do is go to https://issues.apache.org/ and create an account. Guozhang On Mon, Sep 3, 2018 at 4:27 AM, Murali Mani wrote: > Guozhang, > > Cou

Re: [VOTE] KIP-357: Add support to list ACLs per principal

2018-09-04 Thread Adam Bellemare
+1 (non binding) - would really like to see this one. On Mon, Sep 3, 2018 at 12:18 PM, Mickael Maison wrote: > +1 (non binding) > On Mon, Sep 3, 2018 at 3:14 PM Manikumar > wrote: > > > > bump up! waiting for 2 more binding votes! > > > > On Tue, Aug 28, 2018 at 7:36 AM Satish Duggana > > >

[jira] [Resolved] (KAFKA-4949) Calling kaka-consumer-group.sh to get the consumer offset throws OOM with heap space error

2018-09-04 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4949?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-4949. -- Resolution: Duplicate Resolving as duplicate of KAFKA-4090 > Calling kaka-consumer-group.sh to get the

Re: [DISCUSS] KIP-360: Improve handling of unknown producer

2018-09-04 Thread Jason Gustafson
Bump. Thanks to Magnus for noticing that I forgot to link to the KIP: https://cwiki.apache.org/confluence/display/KAFKA/KIP-360%3A+Improve+handling+of+unknown+producer . -Jason On Tue, Aug 21, 2018 at 4:37 PM, Jason Gustafson wrote: > Hi All, > > I have a proposal to improve the transactional/i

[jira] [Resolved] (KAFKA-3971) Consumers drop from coordinator and cannot reconnect

2018-09-04 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3971?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-3971. -- Resolution: Auto Closed Closing inactive issue. Please reopen if the issue still exists in newer versi

Re: [DISCUSS] KIP 368: Allow SASL Connections to Periodically Re-Authenticate

2018-09-04 Thread Ron Dagostino
Hi Rajini. I'm glad you agree this is a good security addition. Regarding your questions/comments: 1. Retries. << requestBuilder, long createdTimeMs, boolean expectResponse, Requ

[jira] [Resolved] (KAFKA-1950) Expose methods from SerializationTestUtils to all tests

2018-09-04 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1950?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-1950. -- Resolution: Auto Closed Closing as SerializationTestUtils class is removed from code. > Expose methods

[jira] [Resolved] (KAFKA-949) Integrate kafka into YARN

2018-09-04 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-949?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-949. - Resolution: Resolved Resolving as duplicate of KAFKA-1754 > Integrate kafka into YARN > -

[jira] [Created] (KAFKA-7376) After Kafka upgrade to v2.0.0 , Controller unable to communicate with brokers on SASL_SSL

2018-09-04 Thread Sridhar (JIRA)
Sridhar created KAFKA-7376: -- Summary: After Kafka upgrade to v2.0.0 , Controller unable to communicate with brokers on SASL_SSL Key: KAFKA-7376 URL: https://issues.apache.org/jira/browse/KAFKA-7376 Project:

[jira] [Created] (KAFKA-7375) Improve error messages verbosity

2018-09-04 Thread Seweryn Habdank-Wojewodzki (JIRA)
Seweryn Habdank-Wojewodzki created KAFKA-7375: - Summary: Improve error messages verbosity Key: KAFKA-7375 URL: https://issues.apache.org/jira/browse/KAFKA-7375 Project: Kafka Issu

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

2018-09-04 Thread Adam Bellemare
As I was looking more into RocksDB TTL, I see that we currently do not support it in Kafka Streams due to a number of technical reasons. As I don't think that I will be tackling that JIRA at the moment, the current implementation is indeed unbounded in the highwater table growth. An alternate opti

Re: [DISCUSS] KIP-349 Priorities for Source Topics

2018-09-04 Thread nick
@Jan - can you comment on whether or not this is what you had in mind ? -- Nick > On Aug 30, 2018, at 10:18 AM, n...@afshartous.com wrote: > > > Just clarifying that the API below would be in addition to the API specified > in KIP-349 > > > https://cwiki.apache.org/confluence/displ

Re: [DISCUSS] KIP 368: Allow SASL Connections to Periodically Re-Authenticate

2018-09-04 Thread Rajini Sivaram
Hi Ron, Thanks for the KIP. This is going to be a really useful feature to tighten security. I have a few comments/questions: 1. Retries: What is the state of a client connection in between re-authentication retries? Can it continue to process requests? Also, can you describe scenarios

[VOTE] KIP-369 Alternative Partitioner to Support "Always Round-Robin" Selection

2018-09-04 Thread M. Manna
Hello, I have made necessary changes as per the original discussion thread, and would like to put it for votes. Thank you very much for your suggestion and guidance so far. Regards,

[jira] [Created] (KAFKA-7374) Tiered Storage

2018-09-04 Thread JIRA
Maciej Bryński created KAFKA-7374: - Summary: Tiered Storage Key: KAFKA-7374 URL: https://issues.apache.org/jira/browse/KAFKA-7374 Project: Kafka Issue Type: Improvement Components: