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

2018-04-11 Thread Debasish Ghosh
Hello everyone - This is in continuation to the discussion regarding https://cwiki.apache.org/confluence/display/KAFKA/KIP-270+-+A+Scala+Wrapper+Library+for+Kafka+Streams, which is a KIP for implementing a Scala wrapper library for Kafka Streams. We have had a PR (https://github.com/apache/kafka/

[jira] [Created] (KAFKA-6778) DescribeConfigs does not return error for non-existent topic

2018-04-11 Thread Magnus Edenhill (JIRA)
Magnus Edenhill created KAFKA-6778: -- Summary: DescribeConfigs does not return error for non-existent topic Key: KAFKA-6778 URL: https://issues.apache.org/jira/browse/KAFKA-6778 Project: Kafka

Re: RE: [DISCUSS] KIP-280: Enhanced log compaction

2018-04-11 Thread Luís Cabral
Hi all, On my own previous statement: bq. Not that I mind doing it directly (I intend to use a Java client), but please be aware that a String binary representation is based on the charset encoding, while the Long binary representation varies according to the language. I went back to double

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

2018-04-11 Thread Ted Yu
+1 Original message From: Debasish Ghosh Date: 4/11/18 3:09 AM (GMT-08:00) To: dev@kafka.apache.org Subject: [VOTE] KIP-270 A Scala wrapper library for Kafka Streams Hello everyone - This is in continuation to the discussion regarding https://cwiki.apache.org/confluence/dis

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

2018-04-11 Thread Rajini Sivaram
Hi Dhruvil, Thanks for the KIP. This is a great improvement to reduce OOMs in brokers during down-conversion. Just a couple of minor questions: The goals state: "*Provide appropriate configuration parameters to manage maximum memory usage during down-conversion on the broker.*" Which config para

Re: [DISCUSS] KIP-277 - Fine Grained ACL for CreateTopics API

2018-04-11 Thread Colin McCabe
Hi Edoardo, Permissions on the Cluster singleton are very powerful. For example, ALTER CLUSTER gives you the ability to add or remove any other ACLs you like (essentially unlimited permissions). ALTERCONFIGS CLUSTER give syou the ability to reconfigure the brokers, and so forth. The general

[DISCUSS] KIP-285: Connect Rest Extension Plugin

2018-04-11 Thread Magesh Nandakumar
Hi, We have posted KIP-285: Connect Rest Extension Plugin to add the ability to provide Rest Extensions to Connect Rest API. https://cwiki.apache.org/confluence/display/KAFKA/KIP- 285%3A+Connect+Rest+Extension+Plugin Please take a look. Your feedback is appreciated. Thanks, Magesh

Re: RE: [DISCUSS] KIP-280: Enhanced log compaction

2018-04-11 Thread Guozhang Wang
Hello Luís, Regarding the timestamp: it is designed to be mainly used for indicating the time when this record is generated (i.e. CREATE_TIME at the producer side, it will set the timestamp), or when the record has been appended to Kafka brokers (i.e. LOG_APPEND_TIME at the broker side, where prod

[DISCUSS] KIP-285: Connect Rest Extension Plugin

2018-04-11 Thread Magesh Nandakumar
Hi, We have posted KIP-285: Connect Rest Extension Plugin to add the ability to provide Rest Extensions to Connect Rest API. https://cwiki.apache.org/confluence/display/KAFKA/KIP-285%3A+Connect+Rest+Extension+Plugin Please take a look. Your feedback is appreciated. Thanks, Magesh

Re: RE: [DISCUSS] KIP-280: Enhanced log compaction

2018-04-11 Thread Luís Cabral
Hi Guozhang, bq. I'm not sure I understand you statement that it is used to determine the "version" of the record I do not mean that it is "used", but if what you meant is that you would prefer to use that field instead of a header? This is in relation to a previous point of yours: >>> 1) I'm

Re: [DISCUSS] KIP-285: Connect Rest Extension Plugin

2018-04-11 Thread Gwen Shapira
Thanks Magesh! The lack of authentication and authorization in the REST API is definitely a painpoint and something that can prevent a company from adopting connect (or surviving an audit after they did!). One thing that isn't clear to me from the KIP: In the past when we contributed pluggable int

Re: [DISCUSS] KIP-285: Connect Rest Extension Plugin

2018-04-11 Thread Magesh Nandakumar
Hi Gwen, Thanks for your feedback. As part of this KIP, I was planning on adding a skeleton implementation in the examples but that can be used only as a reference. At this moment, there is no plan to add a concrete implementation of the plugin. Let me know your thoughts :). Thanks, Magesh On We

Re: [DISCUSS] KIP-285: Connect Rest Extension Plugin

2018-04-11 Thread Gwen Shapira
Not a blocker, but it will continue a good tradition to add something simple that works. I'm thinking plain-text auth and maybe ZK storage (like the Kafka authorizer). It doesn't have to provide real security in any sense, but it will allow people to test and experiment. Otherwise, if I write an im

Re: [DISCUSS] KIP-285: Connect Rest Extension Plugin

2018-04-11 Thread Magesh Nandakumar
Sounds Good. I will update the KIP to include an extension that authenticates Basic Auth Credentials against a properties file. Let me know if that works. Thanks Magesh On Wed, Apr 11, 2018 at 9:51 AM, Gwen Shapira wrote: > Not a blocker, but it will continue a good tradition to add something >

Re: [DISCUSS] KIP-285: Connect Rest Extension Plugin

2018-04-11 Thread Randall Hauch
Very nice proposal, Magesh. I like the approach and the new concepts and interfaces, but I do have a few comments/suggestions about some specific details: 1. In the "Motivation" section, perhaps it makes sense to briefly describe one or two somewhat concrete examples of how this is useful.

Re: RE: [DISCUSS] KIP-280: Enhanced log compaction

2018-04-11 Thread Guozhang Wang
> I do not mean that it is "used", but if what you meant is that you would prefer to use that field instead of a header? > This is in relation to a previous point of yours: I think maybe we have a mis-communication here: I'm not against the idea of using headers, but just trying to argue that we c

Re: RE: [DISCUSS] KIP-280: Enhanced log compaction

2018-04-11 Thread Ted Yu
bq. 2. if the config value is "timestamp", look into the offset field; I think you meant looking into timestamp field. Cheers On Wed, Apr 11, 2018 at 10:18 AM, Guozhang Wang wrote: > > I do not mean that it is "used", but if what you meant is that you would > prefer to use that field instead o

Re: RE: [DISCUSS] KIP-280: Enhanced log compaction

2018-04-11 Thread Guozhang Wang
If you are referring to, for example: -4611686018427387904 > 0 -4611686018427387904 > 4611686018427387903 It is because when we compare the bytes we do not treat them as longs at all, so we just compare them based on bytes; I admit that if users's header types have some semantic meanings (e.g. i

Re: RE: [DISCUSS] KIP-280: Enhanced log compaction

2018-04-11 Thread Guozhang Wang
Yup, lazy copy-paste punishment :P Guozhang On Wed, Apr 11, 2018 at 10:19 AM, Ted Yu wrote: > bq. 2. if the config value is "timestamp", look into the offset field; > > I think you meant looking into timestamp field. > > Cheers > > On Wed, Apr 11, 2018 at 10:18 AM, Guozhang Wang > wrote: > >

[jira] [Resolved] (KAFKA-6752) Unclean leader election metric no longer working

2018-04-11 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6752?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao resolved KAFKA-6752. Resolution: Fixed Fix Version/s: 1.1.1 2.0.0 Merged to trunk and 1.1. > Unclean l

[jira] [Resolved] (KAFKA-6447) Add Delegation Token Operations to KafkaAdminClient

2018-04-11 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6447?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao resolved KAFKA-6447. Resolution: Fixed Fix Version/s: 2.0.0 Merged the PR to trunk. > Add Delegation Token Operations to

[jira] [Created] (KAFKA-6779) Kafka connect JMX metrics returns inconsistent metrics while running multiple connectors

2018-04-11 Thread Sairam Polavarapu (JIRA)
Sairam Polavarapu created KAFKA-6779: Summary: Kafka connect JMX metrics returns inconsistent metrics while running multiple connectors Key: KAFKA-6779 URL: https://issues.apache.org/jira/browse/KAFKA-6779

Re: [DISCUSS] KIP-278: Add version option to Kafka's commands

2018-04-11 Thread Jason Gustafson
Hey Sasaki, Yeah, I don't feel too strongly about only supporting --version. I agree it may help discoverability given the current approach. On the other hand, if we refactored all of the tools so that we could use a common set of base options, it might be a little annoying to have to continue sup

[jira] [Created] (KAFKA-6780) log cleaner shouldn't clean messages beyond high watermark

2018-04-11 Thread Jun Rao (JIRA)
Jun Rao created KAFKA-6780: -- Summary: log cleaner shouldn't clean messages beyond high watermark Key: KAFKA-6780 URL: https://issues.apache.org/jira/browse/KAFKA-6780 Project: Kafka Issue Type: Bug

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

2018-04-11 Thread Jun Rao
Hi, Anna, Thanks for the KIP. Looks good to me. Great point on bounding the cleaning point in a compacted topic by high watermark. Filed https://issues.apache.org/jira/browse/KAFKA-6780 to track it. Jun On Thu, Apr 5, 2018 at 12:17 PM, Anna Povzner wrote: > Hi, > > > I just created KIP-279 t

Re: [DISCUSS] KIP-276: Add StreamsConfig prefix for different consumers

2018-04-11 Thread Ewen Cheslack-Postava
On Thu, Apr 5, 2018 at 3:28 PM, Matthias J. Sax wrote: > Ewen, > > I cannot completely follow your argument. Can you elaborate a little > bit? After reading you mail, I am not sure if you prefer config > inheritance or not? And if, to what extend? > So we've had issues with config complexity in

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

2018-04-11 Thread Jason Gustafson
Hey Dhruvil, Thanks for the KIP. Looks good overall. I have a few questions about the new configs: 1. I'm mainly wondering how necessary the configs are given the improvements in this KIP to reduce memory pressure from down-conversion. The reason I ask is that we'll be stuck with this config for

Re: [DISCUSS] KIP-276: Add StreamsConfig prefix for different consumers

2018-04-11 Thread Matthias J. Sax
Thanks for the explanation. I am not sure if I follow your objection about inheritance. Basically, every config that is not overwritten inherits the default setting -- so there is actually already a hierarchy in place from my point of view. With regard to security config, I guess the issue is, th

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

2018-04-11 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-6058: KIP-222; Add Consumer Group operations to Admin API [jason] MINOR: Fix doc - `FileMessageSet` was replaced by `FileRecords` (#4852) [ismael] MINOR: Fix AsyncProducerTest bug

[jira] [Resolved] (KAFKA-6773) Kafka consumer without group.id crashes when requesting offset on a topic-partition

2018-04-11 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6773?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-6773. Resolution: Fixed Fix Version/s: (was: 1.1.1) (was: 1.2.0)

[jira] [Created] (KAFKA-6781) Transient failure in KafkaAdminClientTest.testDescribeConsumerGroupOffsets

2018-04-11 Thread Jason Gustafson (JIRA)
Jason Gustafson created KAFKA-6781: -- Summary: Transient failure in KafkaAdminClientTest.testDescribeConsumerGroupOffsets Key: KAFKA-6781 URL: https://issues.apache.org/jira/browse/KAFKA-6781 Project:

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

2018-04-11 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-6058: KIP-222; Add Consumer Group operations to Admin API [jason] MINOR: Fix doc - `FileMessageSet` was replaced by `FileRecords` (#4852) [ismael] MINOR: Fix AsyncProducerTest bug

Wiki edit permissions?

2018-04-11 Thread Noah Zoschke
Hello. I would like to add the Segment kafka-go client to the clients page. https://github.com/segmentio/kafka-go https://cwiki.apache.org/confluence/display/KAFKA/Clients#Clients-Go(AKAgolang) Can I get edit access to make this change? Thanks!

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

2018-04-11 Thread Dhruvil Shah
Hi Rajini, Thanks for the comments. Which config parameters are these referring to? This refers to a proposal that was later rejected. I have removed this goal from the KIP as it is no longer valid. What exactly is a chunk going to be I have updated the KIP to remove references to the fixed bu

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

2018-04-11 Thread Dhruvil Shah
Hi Jason, 1. The motivation for adding the configuration was that even though we are improving memory usage, there is still overhead involved with down-conversion - the broker still has to do extra work which some users might want to eliminate completely. The proposal also moves the down-conversio

Re: [DISCUSS] KIP-276: Add StreamsConfig prefix for different consumers

2018-04-11 Thread Boyang Chen
Thanks Ewen for sharing your concern in the inheritance, and Matthias for the explanation to each bullet point. I'm not super expert on security, so I would like to explain why I took over this Jira in the first place, i.e. the use case. Our stream application requires millisecond delay guaran

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

2018-04-11 Thread Apache Jenkins Server
See Changes: [github] KAFKA-6773; Allow offset commit/fetch/describe/delete with empty groupId -- [...truncated 1.45 MB...] org.apache.kafka.clients.producer.MockProducer

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

2018-04-11 Thread Apache Jenkins Server
See Changes: [github] KAFKA-6773; Allow offset commit/fetch/describe/delete with empty groupId -- [...truncated 1.48 MB...] kafka.admin.ResetConsumerGroupOffsetTest > test

[DISCUSS] KIP-286: producer.send() should not block on metadata update

2018-04-11 Thread Dong Lin
Hi all, I have created KIP-286: producer.send() should not block on metadata update. See https://cwiki.apache.org/confluence/display/KAFKA/KIP-286%3A+producer.send%28%29+should+not+block+on+metadata+update . The KIP intends to improve user-experience of producer.send() when metadata is not availa

Re: [DISCUSS] KIP-286: producer.send() should not block on metadata update

2018-04-11 Thread Ted Yu
Looks like per-topic queue is introduced. In terms of memory consumption, how does the KIP allocate memory between per-topic queue and per-partition queue ? Thanks On Wed, Apr 11, 2018 at 8:50 PM, Dong Lin wrote: > Hi all, > > I have created KIP-286: producer.send() should not block on metadat

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

2018-04-11 Thread Apache Jenkins Server
See Changes: [wangguoz] MINOR: make Sensor#add idempotent (#4853) -- [...truncated 411.25 KB...] kafka.admin.ResetConsumerGroupOffsetTest > testResetOffsetsToLatest PASSED

Re: Wiki edit permissions?

2018-04-11 Thread Matthias J. Sax
That is your login ID? On 4/11/18 6:04 PM, Noah Zoschke wrote: > Hello. > > I would like to add the Segment kafka-go client to the clients page. > > https://github.com/segmentio/kafka-go > https://cwiki.apache.org/confluence/display/KAFKA/Clients#Clients-Go(AKAgolang) > > Can I get edit access

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

2018-04-11 Thread Apache Jenkins Server
See