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

2018-01-19 Thread Randall Hauch
Sonke, Have you had a chance to update the KIP and kick off a VOTE thread? We need to do this ASAP if we want this to make the KIP deadline for 1.1, which is Jan 23! On Tue, Jan 16, 2018 at 10:33 PM, Ewen Cheslack-Postava wrote: > Sonke, > > I'm fine filtering some control characters. The trimm

[VOTE] KIP-145: Expose Record Headers in Kafka Connect

2018-01-19 Thread Randall Hauch
Hi everyone, I'd like to start the voting on this KIP to add support for headers in Connect.: *https://cwiki.apache.org/confluence/display/KAFKA/KIP-145+-+Expose+Record+Headers+in+Kafka+Connect *

Re: Kafka compacted topic question.

2018-01-19 Thread Matthias J. Sax
Yes and no. There is a background compaction thread that runs periodically (you can configure the scheduling for this thread). Thus, compaction happens async. It's correct, that the current head segments is not considered for compaction. There is also no de-duplication on write, but message will

Re: How to always consume from latest offset in kafka-streams

2018-01-19 Thread Matthias J. Sax
That is not supported out-of-box. Configuration "auto.offset.reset" only triggers, if there are not committed offsets and there is KS config to change this behavior. A possible workaround might be (but not sure if I want to recommend this), to increase KafkaStreams commit interval via StreamsConf

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

2018-01-19 Thread Dong Lin
Hey Jun, I think we can probably have a static method in Util class to decode the byte[]. Both KafkaConsumer implementation and the user application will be able to decode the byte array and log its content for debug purpose. So it seems that we can still print the information we want. It is just

Re: [VOTE] KIP-208: Add SSL support to Kafka Connect REST interface

2018-01-19 Thread Jakub Scholz
@Jason: Actually ... wouldn't it be better to name it only "rest.advertised.listener"? The "security" doesn't make much sense without the protocol. What do you think? On Fri, Jan 19, 2018 at 10:51 PM, Jakub Scholz wrote: > Hi Jason, > > Thanks for the vote. Yeah, I think that renaming it to " >

Re: How many kafka streams app is recommended to run on single machine in production ?

2018-01-19 Thread Matthias J. Sax
Multiple answers: - a KafkaStreams instance start one *processing* thread by default (you can configure more processing threads, too) - internally, KafkaStreams uses two KafkaConsumers and one KafkaProducer (if you turn on EOS, it uses even more KafkaProducers): a KafkaConsumer starts a backgroun

Re: [VOTE] KIP-227: Introduce Fetch Requests that are Incremental to Increase Partition Scalability

2018-01-19 Thread Jun Rao
Hi, Colin, Thanks for the KIP. Looks good to me overall. Just a couple of more comments. 1. As I mentioned earlier, it might be useful to add some metrics for monitoring the usage of the session cache. For example, it would be useful to know how many slots are being used (or unused), # of total p

Re: [DISCUSS] KIP-208: Add SSL support to Kafka Connect REST interface

2018-01-19 Thread Jakub Scholz
FYI: For those not following the VOTE thread I updated the KIP and changed the field "rest.advertised.security.protocol" to "rest.advertised.security.listener" as suggested by Jason. On Fri, Jan 19, 2018 at 11:29 AM, Jakub Scholz wrote: > Hi all, > > I did one more update to the KIP-208. I

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

2018-01-19 Thread Jun Rao
Hi, Dong, The issue with using just byte[] for OffsetEpoch is that it won't be printable, which makes debugging harder. Also, KIP-222 proposes a listGroupOffset() method in AdminClient. If that gets adopted before this KIP, we probably want to include OffsetEpoch in the AdminClient too. Thanks,

Re: [VOTE] KIP-222 - Add "describe consumer group" to KafkaAdminClient

2018-01-19 Thread Jun Rao
Hi, Jorge, Thanks for the KIP. Looks good to me overall. A few comments below. 1. It seems that ConsumerDescription should be MemberDescription? 2. Each offset can have an optional metadata. So, in ListGroupOffsetsResult, perhaps it's better to have KafkaFuture>, where OffsetAndMetadata contains

Re: [VOTE] KIP-208: Add SSL support to Kafka Connect REST interface

2018-01-19 Thread Jakub Scholz
Hi Jason, Thanks for the vote. Yeah, I think that renaming it to " rest.advertised.security.listener" is good idea. Unless someone else objects, I will update the KIP. Thanks & Regards Jakub On Fri, Jan 19, 2018 at 6:09 PM, Jason Gustafson wrote: > +1 from me. I just had one suggestion. I saw

Re: Kafka compacted topic question.

2018-01-19 Thread Matt Farmer
Yeah, and I thought I answered your question? I think the compaction happens when new segments are created. Sorry if I’m still misunderstanding. > On Jan 19, 2018, at 3:55 PM, Rahul Bhattacharjee > wrote: > > Thanks Matt for the response .I was asking about the log compaction >

Re: [VOTE] KIP-208: Add SSL support to Kafka Connect REST interface

2018-01-19 Thread Gwen Shapira
+1 (binding), Thank you!!! On Fri, Jan 19, 2018 at 9:09 AM Jason Gustafson wrote: > +1 from me. I just had one suggestion. I saw in the discussion thread that > you considered allowing multiple listeners for each protocol type, but > ultimately rejected it. Maybe to leave the door a little more

Re: Kafka compacted topic question.

2018-01-19 Thread Rahul Bhattacharjee
Thanks Matt for the response .I was asking about the log compaction of kafka topics. On Fri, Jan 19, 2018 at 12:36 PM, Matt Farmer wrote: > Someone will need to correct me if I’m wrong, but my understanding is that > a topic log on disk is div

Re: Kafka compacted topic question.

2018-01-19 Thread Matt Farmer
Someone will need to correct me if I’m wrong, but my understanding is that a topic log on disk is divided into segments. Compaction will occur when a segment “rolls off” - so when a new active segment is created and the previous segment becomes inactive. Segments can be bounded by size and time

Re: How to always consume from latest offset in kafka-streams

2018-01-19 Thread Matt Farmer
That config setting will only work if there are no offsets stored in the consumer offsets target. Something I’ve done in the past is to make the application.id config setting have a random string component to it. So have “my-app-name-[randomchars]” or some such. This ensures that there are neve

Kafka compacted topic question.

2018-01-19 Thread Rahul Bhattacharjee
Let's say we have a compacted topic (log.cleanup.policy=compact) where lot of updates happen for relatively small set of keys. My question is when does the compaction happen. In memtable , when a new update comes for an already existing key in memtable , the value is simple replaced. or, all the u

How to always consume from latest offset in kafka-streams

2018-01-19 Thread Saloni Vithalani
Our requirement is such that if a kafka-stream app is consuming a partition, it should start it's consumption from latest offset of that partition. This seems like do-able using streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "latest") Now, let's say using above configuration,

How many kafka streams app is recommended to run on single machine in production ?

2018-01-19 Thread Saloni Vithalani
In our architecture, we are assuming to run three jvm processes on one machine (approx.) and each jvm machine can host upto 15 kafka-stream apps. And if I am not wrong each kafka-stream app spawns one java thread. So, this seems like an awkward architecture to have with around 45 kafka-stream apps

Re: [VOTE] KIP-208: Add SSL support to Kafka Connect REST interface

2018-01-19 Thread Jason Gustafson
+1 from me. I just had one suggestion. I saw in the discussion thread that you considered allowing multiple listeners for each protocol type, but ultimately rejected it. Maybe to leave the door a little more open in the future for this, we could rename the config "rest.advertised.security.protocol"

Build failed in Jenkins: kafka-0.10.2-jdk7 #201

2018-01-19 Thread Apache Jenkins Server
See Changes: [rajinisivaram] MINOR: additional check to follower fetch handling (#4448) -- Started by an SCM change [EnvInject] - Loading node environment variables. Build

Re: [VOTE] KIP-208: Add SSL support to Kafka Connect REST interface

2018-01-19 Thread Damian Guy
Thanks Jakub! +1 (binding) On Thu, 18 Jan 2018 at 23:49 Jakub Scholz wrote: > Hi all, > > We still need at least 2 more binding +1s. I think that the PR ( > https://github.com/apache/kafka/pull/4429) is shaping good. If we get the > votes, we should be able to make the 1.1.0 release. > > Thanks

[VOTE] KIP-248: Create New ConfigCommand That Uses The New AdminClient

2018-01-19 Thread Viktor Somogyi
Hi all, I'd like to start the vote on KIP-248: Create New ConfigCommand That Uses The New AdminClient. The KIP can be read here: https://cwiki.apache.org/confluence/display/KAFKA/KIP-248+-+Create+New+ConfigCommand+That+Uses+The+New+AdminClient The discussion thread is at https://lists.apache.org

Re: [DISCUSS] KIP-248 Create New ConfigCommand That Uses The New AdminClient

2018-01-19 Thread Viktor Somogyi
Hi Rajini, Ok, I think I got you. I wasn't calculating with the fact that the parent might not be set, therefore it could be a default user as well or even the default client if nothing else is set (supposing we're talking about the example). So if I'm correct, the quota will be applied in the or

[jira] [Created] (KAFKA-6463) Review logging level for user errors in AdminManager

2018-01-19 Thread Rajini Sivaram (JIRA)
Rajini Sivaram created KAFKA-6463: - Summary: Review logging level for user errors in AdminManager Key: KAFKA-6463 URL: https://issues.apache.org/jira/browse/KAFKA-6463 Project: Kafka Issue Ty

Re: [DISCUSS] KIP-208: Add SSL support to Kafka Connect REST interface

2018-01-19 Thread Jakub Scholz
Hi all, I did one more update to the KIP-208. I added the "ssl.endpoint.identification.algorithm" to the list of supported options. It can be used to enable / disable the hostname validation when the Kafka Connect nodes are forwarding the requests to the leader. It is minor but useful change, so I

Re: [DISCUSS] KIP-249: Add Delegation Token Operations to Kafka Admin Client

2018-01-19 Thread Manikumar
Hi all, We want to include this KIP in the upcoming 1.1.0 release. Please let me know if there are any other comments. If there are no more comments, I'd like to start vote on this KIP. Thanks, On Wed, Jan 17, 2018 at 8:20 AM, Manikumar wrote: > Hi, Jun, > > Thanks for the review. > > 1. Yes