[jira] [Resolved] (KAFKA-7137) ability to trigger compaction for tombstoning and GDPR

2018-09-05 Thread Brett Rann (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7137?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Brett Rann resolved KAFKA-7137. --- Resolution: Duplicate > ability to trigger compaction for tombstoning and GDPR >

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

2018-09-05 Thread Brett Rann
OK thanks for that clarification. I see why you're uncomfortable with 0 now. I'm not really fussed. I just prefer consistency in configuration options. Personally I lean towards treating 0 and 1 similarly in that scenario, because it favours the person thinking about setting the configurations,

[jira] [Resolved] (KAFKA-7365) max.poll.records setting in Kafka Consumer is not working

2018-09-05 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7365?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax resolved KAFKA-7365. Resolution: Invalid > max.poll.records setting in Kafka Consumer is not working >

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

2018-09-05 Thread xiongqi wu
I want to honor the minimum value of segment.ms (which is 1ms) to force roll an active segment. So if we set "max.compaction.lag.ms" any value > 0, the minimum of max.compaction.lag.ms and segment.ms will be used to seal an active segment. If we set max.compaction.lag.ms to 0, the current

[jira] [Resolved] (KAFKA-7353) Connect logs 'this' for anonymous inner classes

2018-09-05 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7353?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava resolved KAFKA-7353. -- Resolution: Fixed Fix Version/s: 2.1.0 2.0.1

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

2018-09-05 Thread Brett Rann
You're rolling a new segment if the condition is met right? So I'm struggling to understand the relevance of segment.ms here. Maybe an example would help my understanding: segment.ms=999 *min.cleanable.dirty.ratio=1* max.compaction.lag.ms=1 When a duplicate message comes in, after 1ms the

Re: Question about connector rebalancing

2018-09-05 Thread Gwen Shapira
Can you use stand-alone mode in that case? On Wed, Sep 5, 2018 at 7:12 PM, Chen He wrote: > Hi Kafka experts > > I have a question about connector rebalancing issue. Why don't we make it > option, I mean have a parameter that turn on/off it instead of having it as > a must? > > We can have a

[jira] [Created] (KAFKA-7381) Parameterize connector rebalancing behavior

2018-09-05 Thread Chen He (JIRA)
Chen He created KAFKA-7381: -- Summary: Parameterize connector rebalancing behavior Key: KAFKA-7381 URL: https://issues.apache.org/jira/browse/KAFKA-7381 Project: Kafka Issue Type: Improvement

Question about connector rebalancing

2018-09-05 Thread Chen He
Hi Kafka experts I have a question about connector rebalancing issue. Why don't we make it option, I mean have a parameter that turn on/off it instead of having it as a must? We can have a parameter like: "connector.rebalancing.enable" parameter and make it as "true" by default. It allows users

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

2018-09-05 Thread xiongqi wu
To make it clear, I don't against using -1 as disabled, but we need to come up with the meaning of "0". If "0" means immediate compaction, but the actual compaction lag will be segment.ms. It has longer lag than setting the value to be half of segment.ms. We cannot provide "0" as max compaction

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

2018-09-05 Thread Brett Rann
-1 is consistent as "special" with these settings for example: log.retention.bytes socket.received.buffer.bytes socket.send.buffer.bytes queued.max.request.bytes retention.bytes retention.ms and acks. Where it may mean no limit, use OS defaults, max (acks), etc. I don't see much convention of 0

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

2018-09-05 Thread Ron Dagostino
I just thought of another alternative if the imports are the concern. KafkaChannel could expose the fact that it can create an additional Authenticator instance on the side (what I referred to as notYetAuthenticatedAuthenticator in the PR) and it could let kafka.server.KafkaApis drive the whole

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

2018-09-05 Thread Brett Rann
If segment.ms can't be set to 0, then we're not being consistent by using 0 for this new setting? I throw out -1 for consideration again :) On Thu, Sep 6, 2018 at 10:03 AM xiongqi wu wrote: > Thanks. I will document after PR is merged. > > BTW, Kafka enforce the minimum of "segment.ms" to 1, we

[jira] [Created] (KAFKA-7380) Global thread restore blocks KafkaStreams#start()

2018-09-05 Thread Matthias J. Sax (JIRA)
Matthias J. Sax created KAFKA-7380: -- Summary: Global thread restore blocks KafkaStreams#start() Key: KAFKA-7380 URL: https://issues.apache.org/jira/browse/KAFKA-7380 Project: Kafka Issue

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

2018-09-05 Thread xiongqi wu
Thanks. I will document after PR is merged. BTW, Kafka enforce the minimum of "segment.ms" to 1, we cannot set " segment.ms" to 0. I also updated the title of this KIP. Xiongqi (Wesley) Wu On Wed, Sep 5, 2018 at 4:34 PM Brett Rann wrote: > I withdraw my comments on -1 since i'm in the

Re: [VOTE] KIP-291: Have separate queues for control requests and data requests

2018-09-05 Thread Jun Rao
Hi, Lucas, Thanks for the updated KIP. For monitoring the network thread utilization for the control plane, we already have the metric io-ratio at the selector level (idlePct is 1 - io-ratio). So, we just need to give that selector a meaningful name. For monitoring the io thread utilization for

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

2018-09-05 Thread Brett Rann
I withdraw my comments on -1 since i'm in the minority. :) Can we make sure 0 gets documented as meaning disabled here: https://kafka.apache.org/documentation/#brokerconfigs ? And while there it would be good if segment.ms is documented that 0 is disabled too. (there's some hierarchy of

[jira] [Created] (KAFKA-7379) send.buffer.bytes should be allowed to set -1 in KafkaStreams

2018-09-05 Thread Badai Aqrandista (JIRA)
Badai Aqrandista created KAFKA-7379: --- Summary: send.buffer.bytes should be allowed to set -1 in KafkaStreams Key: KAFKA-7379 URL: https://issues.apache.org/jira/browse/KAFKA-7379 Project: Kafka

Re: [VOTE] KIP-291: Have separate queues for control requests and data requests

2018-09-05 Thread Lucas Wang
Thanks Jun for your quick response. It looks like I forgot to click the "Update" button, :) It's updated now. Regarding the idle ratio metrics for the additional threads, I discussed with Joel, and think they are not as useful, and I added our reasoning in the last paragraph of the "How are

Re: [VOTE] KIP-291: Have separate queues for control requests and data requests

2018-09-05 Thread Jun Rao
Hi, Lucas, Thanks for the reply. Have you actually updated the KIP? The wiki says that it's last updated on Aug. 22. and some of the changes that you mentioned (#1 and #3) are not there. Also, regarding Joel's comment on network/request idle ratio metrics, could you comment on whether they

Re: [VOTE] KIP-363: Make FunctionConversions private

2018-09-05 Thread John Roesler
Hi Joan, At the end of the day, I have to agree with Matthias. It seems a little silly to keep it around, but we did already release it as public. Maintenance is the penalty we have to pay for insufficiently scrutinizing the code we have released. Sorry you've gotten dragged into it. My

Re: [VOTE] KIP-366 - Make FunctionConversations private

2018-09-05 Thread John Roesler
I'm a +1 (non-binding) because we doubt the class is in use. If you decide to copy it to a private version and deprecate the original instead, as Matthias suggested, I would still be a +1. Thanks, -John On Sat, Sep 1, 2018 at 6:47 AM Joan Goyeau wrote: > Hi, > > As pointed out in this comment

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

2018-09-05 Thread Ron Dagostino
Hi again, Rajini, I realized a couple of potential concerns with using the TransportLayer directly during re-authentication. First, in the blocking I/O use case, the owner of the NetworkClient instance calls NetworkClientUtils.sendAndReceive() to send requests. This method assumes the caller

[jira] [Created] (KAFKA-7378) Consumer poll hangs if broker shutdown while consumer attempting to connect.

2018-09-05 Thread John Calcote (JIRA)
John Calcote created KAFKA-7378: --- Summary: Consumer poll hangs if broker shutdown while consumer attempting to connect. Key: KAFKA-7378 URL: https://issues.apache.org/jira/browse/KAFKA-7378 Project:

Re: [VOTE] KIP-365: Materialized, Serialized, Joined, Consumed and Produced with implicit Serde

2018-09-05 Thread John Roesler
Hi Joan, Damian makes 3 binding votes, and the vote has been open longer than 72 hours, so your KIP vote has passed! It's customary for you to send a final reply to this thread stating that the vote has passed, and stating the number of binding and non-binding +1s. Also please update the

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

2018-09-05 Thread Adam Bellemare
I'm currently testing using a Windowed Store to store the highwater mark. By all indications this should work fine, with the caveat being that it can only resolve out-of-order arrival for up to the size of the window (ie: 24h, 72h, etc). This would remove the possibility of it being unbounded in

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

2018-09-05 Thread Jan Filipiak
Thanks Adam for bringing Matthias to speed! about the differences. I think re-keying back should be optional at best. I would say we return a KScatteredTable with reshuffle() returning KTable to make the backwards repartitioning optional. I am also in a big favour of doing the out of order

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

2018-09-05 Thread xiongqi wu
If we use 0 to indicate immediate compaction, the compaction lag is determined by segment.ms in worst case. If segment.ms is 24 hours, "immediate compaction" is a weaker guarantee than setting any value less than 24 hours. By the definition of "max compaction lag", we cannot have zero lag. So

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

2018-09-05 Thread Ron Dagostino
Hi Rajini. I'm now skeptical of my "ConnectionState.REAUTHENTICATING" idea. The concept of a connection being "READY" or not can impact ConsumerCoordinator (see, for example,

[DISCUSS] KIP-370: Remove Orphan Partitions

2018-09-05 Thread xiongqi wu
This KIP enables broker to remove orphan partitions automatically. https://cwiki.apache.org/confluence/display/KAFKA/KIP-370%3A+Remove+Orphan+Partitions Xiongqi (Wesley) Wu

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

2018-09-05 Thread xiongqi wu
Colin, When a user creates a topic, the user doesn't know broker's configures. If there is 100 brokers, the user can't contact all the brokers to find out the minimum allowed value for segment.ms. Today, the topic creation is done via zookeeper, at this stage, the user doesn't have right view

Re: [VOTE] KIP-358: Migrate Streams API to Duration instead of long ms times

2018-09-05 Thread John Roesler
I'm a +1 (non-binding) On Mon, Sep 3, 2018 at 8:33 AM Nikolay Izhikov wrote: > Dear commiters. > > Please, vote on a KIP. > > В Пт, 31/08/2018 в 12:05 -0500, John Roesler пишет: > > Hi Nikolay, > > > > You can start a PR any time, but we cannot per it (and probably won't do > > serious reviews)

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

2018-09-05 Thread Ron Dagostino
<< wrote: > On Wed, Sep 5, 2018, at 07:34, Ron Dagostino wrote: > > I added a "How To Support Re-Authentication for Other SASL Mechanisms" > > section to the KIP as Rajini suggested. I also added a "Rejected > > Alternative" for the idea of forcibly closing connections on the client > > side

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

2018-09-05 Thread Colin McCabe
On Tue, Sep 4, 2018, at 22:11, Brett Rann wrote: > > 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

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

2018-09-05 Thread Colin McCabe
On Tue, Sep 4, 2018, at 20:25, xiongqi wu wrote: > 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

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

2018-09-05 Thread Colin McCabe
On Wed, Sep 5, 2018, at 07:34, Ron Dagostino wrote: > I added a "How To Support Re-Authentication for Other SASL Mechanisms" > section to the KIP as Rajini suggested. I also added a "Rejected > Alternative" for the idea of forcibly closing connections on the client > side upon token refresh or on

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

2018-09-05 Thread Colin McCabe
Hi all, I agree that DISCUSS is more appropriate than VOTE at this point, since I don't remember the last discussion coming to a definite conclusion. I guess my concern is that this will add complexity and memory consumption on the server side. In the case of incremental fetch requests, we

Re: [VOTE] KIP-363: Allow performance tools to print final results to output file

2018-09-05 Thread Viktor Somogyi-Vass
+1 (non-binding), Thanks for the KIP! Viktor On Wed, Sep 5, 2018 at 10:14 AM Attila Sasvári wrote: > Hi All, > > I'd like to start the vote on KIP-363: > * > https://cwiki.apache.org/confluence/display/KAFKA/KIP-363%3A+Allow+performance+tools+to+print+final+results+to+output+file > < >

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

2018-09-05 Thread Ron Dagostino
I added a "How To Support Re-Authentication for Other SASL Mechanisms" section to the KIP as Rajini suggested. I also added a "Rejected Alternative" for the idea of forcibly closing connections on the client side upon token refresh or on the server side upon token expiration. It may be a bit

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

2018-09-05 Thread Colin McCabe
On Wed, Sep 5, 2018, at 01:41, Rajini Sivaram wrote: > *Re-authentication vs disconnection:* > In a vast number of secure Kafka deployments, SASL_SSL is the security > protocol (this is the recommended config for OAUTHBEARER). If we require > disconnections on token expiry, we would need new

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

2018-09-05 Thread Robert Yokota
Hi Colin, Thanks for the suggestion, I like it. Do others like the idea of a DirectoryConfigProvider, and if so is it ok if I amend the KIP? Thanks, Robert On Tue, Sep 4, 2018 at 5:33 PM, Colin McCabe wrote: > Hi Robert, > > This seems like a reasonable behavior to me. However, adding this

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

2018-09-05 Thread Adam Bellemare
Hi Matthias Thank you for your feedback, I do appreciate it! > While name spacing would be possible, it would require to deserialize > user headers what implies a runtime overhead. I would suggest to no > namespace for now to avoid the overhead. If this becomes a problem in > the future, we can

Re: Provide clients Header implementation without accessing internal package

2018-09-05 Thread Dongjin Lee
FYI: This discussion is initiated from this PR: https://github.com/apache/spark/pull/22282#discussion_r214632570 (Supporting Kafka Headers functionality in Spark Structured Streaming) Thanks, Dongjin On Wed, Sep 5, 2018 at 7:41 AM Jungtaek Lim wrote: > Hi Kafka devs, > > I just noticed that

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

2018-09-05 Thread Rajini Sivaram
*Re-authentication vs disconnection:* In a vast number of secure Kafka deployments, SASL_SSL is the security protocol (this is the recommended config for OAUTHBEARER). If we require disconnections on token expiry, we would need new connections to be established with an expensive SSL handshake.

[VOTE] KIP-363: Allow performance tools to print final results to output file

2018-09-05 Thread Attila Sasvári
Hi All, I'd like to start the vote on KIP-363: *https://cwiki.apache.org/confluence/display/KAFKA/KIP-363%3A+Allow+performance+tools+to+print+final+results+to+output+file *

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

2018-09-05 Thread Jan Filipiak
On 05.09.2018 02:38, n...@afshartous.com wrote: 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: [DISCUSS] KIP-358: Migrate Streams API to Duration instead of long ms times

2018-09-05 Thread Nikolay Izhikov
Hello, Guys. I've started a VOTE [1], but seems commiters have no chance to look at KIP for now. Can you tell me, is it OK? Should I wait for feedback? For how long? Or something in KIP should be improved before voting? [1]