Re: [DISCUSS] KIP-585: Conditional SMT

2020-04-06 Thread Tom Bentley
To come back about a point Chris made: 1. Instead of the new "ConfigDef config(Map props)" method, > what would you think about adopting a similar approach as the framework > uses with connectors, by adding a "Config validate(Map > props)" method that can perform custom validation outside of what

Re: [DISCUSS] KIP-587 Suppress detailed responses for handled exceptions in security-sensitive environments

2020-04-06 Thread Connor Penhale
Hi Colin, We did not find a specific security vulnerability. Our customer had auditors in their environment, and they identified Kafka Connect as out of compliance with their particular standards, something that happens all the time for REST-based applications. What these security auditors

Re: [DISCUSS] KIP-588: Allow producers to recover gracefully from transaction timeouts

2020-04-06 Thread Boyang Chen
Thanks Guozhang for the review! On Sun, Apr 5, 2020 at 5:47 PM Guozhang Wang wrote: > Hello Boyang, > > Thank you for the proposed KIP. Just some minor comments below: > > 1. Could you also describe which producer APIs could potentially throw the > new TransactionTimedOutException, and also how

Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-04-06 Thread Tom Bentley
Hi Boyang, Thanks for the KIP! When a broker proxies a request to the controller how does the authenticated principal get propagated? I think a couple of things might complicate this: 1. A PrincipalBuilder might be in use, 2. A Principal does not have to be serializable. Kind regards, Tom

Topics naming convention

2020-04-06 Thread Fares Oueslati
Hello, I'm here to get some advice regarding Kafka topics naming. If I name my topic "users" and I publish Avro events with a schema registry using a given compatibility mode. One day, due to business reasons, I have to break my schema, what would you do? add a new topic (let's call it

Re: [External] Topics naming convention

2020-04-06 Thread Brian Sang
If the schema is no longer compatible I think it makes sense to create a new topic. Offsets wise you can just have the consumer finish consuming from the old topic/schema (i.e. migrate all producers to use the new schema format first) and then migrate consumers to use the new schema format (and

Re: [DISCUSS] KIP-584: Versioning scheme for features

2020-04-06 Thread Jun Rao
Hi, Kowshik, Thanks for the reply. A few more replies below. 100.6 You can look for the sentence "This operation requires ALTER on CLUSTER." in KIP-455. Also, you can check its usage in KafkaApis.authorize(). 110. From the external client/tooling perspective, it's more natural to use the

Re: 2.5.0 Producer.sendOffsetsToTransaction() and Backwards Compatibility

2020-04-06 Thread Gary Russell
Thanks, all, >Just to clarify, even for Streams client it cannot detect automatically the broker's version and hence as KIP-447 proposed, the customer needs to set a config value indicating that she is assured the broker version is newer and hence the new API can be used. Yes, I noticed that;

Re: [DISCUSS] KIP-588: Allow producers to recover gracefully from transaction timeouts

2020-04-06 Thread Guozhang Wang
Regarding 2), sounds good, I saw UNKNOWN_PRODUCER_ID is properly handled today in produce / add-partitions-to-txn / add-offsets-to-txn / end-txn responses, so that should be well covered. Could you reflect this in the wiki page that the broker should be responsible for using different error codes

Re: [DISCUSS] KIP-584: Versioning scheme for features

2020-04-06 Thread Guozhang Wang
Hello Kowshik, For 2) above, my motivations is more from the flexibility on client side instead of version deprecation: let's say a client talks to the cluster learned that the cluster-wide version for feature is X, while the client itself only knows how to execute the feature up to version Y ( <

[jira] [Created] (KAFKA-9825) Kafka protocol BNF format should have some way to display tagged fields

2020-04-06 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-9825: --- Summary: Kafka protocol BNF format should have some way to display tagged fields Key: KAFKA-9825 URL: https://issues.apache.org/jira/browse/KAFKA-9825 Project: Kafka

Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-04-06 Thread Sönke Liebau
Hi Boyang, thanks for the KIP. Sounds good overall. @Tom: I thought about your remark a little and think that in principle we can get away without forwarding the principal at all. Brokers currently authenticate and authorize requests before performing writes to Zookeeper - as long as we don't

Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-04-06 Thread Boyang Chen
Thanks Tom for the question! I'm not super familiar with the Principal stuff, could you elaborate more on the two points you proposed here? I looked up Admin client and just take `createDelegationToken` API for an example, the request data encodes the principal information already, so broker

Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-04-06 Thread Colin McCabe
Hi Sönke, Yeah, that was my thought too. The request has already been validated on the forwarding broker, so we don't need to validate it again. However, you make a good point that it's unfortunate that the audit log would lose the principal information if we didn't forward it as well.

Re: [DISCUSS] (KAFKA-9806) authorize cluster operation when creating internal topics

2020-04-06 Thread Colin McCabe
Hi Paolo, Thanks for finding this issue. Unfortunately, you certainly can't add a new permission requirement to an existing RPC without breaking compatibility. So the current solution in the PR will not work. However, you should be able to have the broker create the topic using its own

Re: [VOTE] KIP-444: Refactor and Augment Metrics for Kafka Streams

2020-04-06 Thread Matthias J. Sax
Thanks Guozhang. SGTM. On 4/2/20 5:08 PM, Guozhang Wang wrote: > A correction for the previous email: > > >- number of alive global threads, INFO >- number of alive restore threads, INFO > > > These two metrics are not going to be added in this KIP, since we do not > have restore

Re: 回复:回复:回复:回复:[Vote] KIP-571: Add option to force remove members in StreamsResetter

2020-04-06 Thread Matthias J. Sax
Overall LGTM. +1 (binding) So you do not plan so support removing a _single static_ member via `StreamsResetter`? We can of course still add this as a follow up but it might be nice to just add it to this KIP right away. Up to you if you want to include it or not. -Matthias On 3/30/20 8:16

Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-04-06 Thread Guozhang Wang
Thanks for the KIP Boyang, this looks good to me. Some minor comments: 1) I think in order to implement the forwarding mechanism the brokers needs some purgatory to keep the forwarded requests; if that's true, should we add some broker-side metrics for those purgatories for debugging purposes?

Re: 2.5.0 Producer.sendOffsetsToTransaction() and Backwards Compatibility

2020-04-06 Thread Matthias J. Sax
I guess one important point to mention is why Kafka Streams needs the internal config though: it's about a save upgrade path. Even if the user tells us that they are on old brokers, we call the new `sendOffsetsToTransaction()` API blindly and let the producer downgrade the request. If the user

Re: [DISCUSS] KIP-584: Versioning scheme for features

2020-04-06 Thread Colin McCabe
Hi Jun, I agree that asking the user to manually upgrade all features to the latest version is a burden. Then the user has to know what the latest version of every feature is when upgrading. What about a simple solution to problem this where we add a flag to the command-line tool like

Re: [DISCUSS] KIP-588: Allow producers to recover gracefully from transaction timeouts

2020-04-06 Thread Boyang Chen
Yep, updated the KIP, thanks! On Mon, Apr 6, 2020 at 3:11 PM Guozhang Wang wrote: > Regarding 2), sounds good, I saw UNKNOWN_PRODUCER_ID is properly handled > today in produce / add-partitions-to-txn / add-offsets-to-txn / end-txn > responses, so that should be well covered. > > Could you

[jira] [Created] (KAFKA-9826) Log cleaning goes in infinite loop when first dirty offset is past start of active segment

2020-04-06 Thread Steve Rodrigues (Jira)
Steve Rodrigues created KAFKA-9826: -- Summary: Log cleaning goes in infinite loop when first dirty offset is past start of active segment Key: KAFKA-9826 URL: https://issues.apache.org/jira/browse/KAFKA-9826

[jira] [Resolved] (KAFKA-9815) Consumer may never re-join if inconsistent metadata is received once

2020-04-06 Thread Jason Gustafson (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9815?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-9815. Fix Version/s: 2.4.2 2.5.0 Resolution: Fixed > Consumer may

Build failed in Jenkins: kafka-trunk-jdk11 #1328

2020-04-06 Thread Apache Jenkins Server
See Changes: [github] KAFKA-9815; Ensure consumer always re-joins if JoinGroup fails (#8420) [github] MINOR: Fix log cleaner offset range log message (#8435) --

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

2020-04-06 Thread Apache Jenkins Server
See Changes: [github] KAFKA-9815; Ensure consumer always re-joins if JoinGroup fails (#8420) [github] MINOR: Fix log cleaner offset range log message (#8435) --

Jenkins build is back to normal : kafka-trunk-jdk11 #1327

2020-04-06 Thread Apache Jenkins Server
See

[jira] [Created] (KAFKA-9827) MM2 doesnt replicate data

2020-04-06 Thread Dmitry (Jira)
Dmitry created KAFKA-9827: - Summary: MM2 doesnt replicate data Key: KAFKA-9827 URL: https://issues.apache.org/jira/browse/KAFKA-9827 Project: Kafka Issue Type: Bug Components: mirrormaker

Build failed in Jenkins: kafka-2.5-jdk8 #89

2020-04-06 Thread Apache Jenkins Server
See Changes: [jason] KAFKA-9815; Ensure consumer always re-joins if JoinGroup fails (#8420) -- [...truncated 2.90 MB...] org.apache.kafka.streams.TopologyTestDriverTest >

Re: [DISCUSS] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-04-06 Thread Boyang Chen
Thanks for the various inputs everyone! I think Sonke and Colin's suggestions make sense. The tagged field also avoids the unnecessary protocol changes for affected requests. Will add it to the header. As for the verification, I'm not sure whether it's necessary to require a higher permission

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

2020-04-06 Thread Apache Jenkins Server
See Changes: [github] KAFKA-9753: A few more metrics to add (#8371) -- [...truncated 5.99 MB...] org.apache.kafka.streams.TopologyTestDriverTest >

[jira] [Created] (KAFKA-9824) Consumer loses partition offset and resets post 2.4.1 version upgrade

2020-04-06 Thread Nitay Kufert (Jira)
Nitay Kufert created KAFKA-9824: --- Summary: Consumer loses partition offset and resets post 2.4.1 version upgrade Key: KAFKA-9824 URL: https://issues.apache.org/jira/browse/KAFKA-9824 Project: Kafka

[DISCUSS] (KAFKA-9806) authorize cluster operation when creating internal topics

2020-04-06 Thread Paolo Moriello
Hello everybody, I've opened a Jira to fix a bug on creation of internal topics. This happens when the topics are created under insufficient ACLs: eg. __consumer_offset is created but subsequent updateMetadata and leaderIsr requests fail; the topic is than in an inconsistent state and it is

Re: [DISCUSS] KIP-584: Versioning scheme for features

2020-04-06 Thread Kowshik Prakasam
Hi Colin, Thanks a lot for the explanation! I've updated the KIP based on your suggestions. Please find my response to your comments below. > If you can just treat "not present" as version level 0, you can have just checks like the second one. This should lead to simpler code. (Kowshik): Good

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

2020-04-06 Thread Apache Jenkins Server
See Changes: [github] KAFKA-9784: Add OffsetFetch to group concurrency test (#8383) [github] KAFKA-9793: Expand the try-catch for task commit in HandleAssignment --

Re: [DISCUSS] KIP-584: Versioning scheme for features

2020-04-06 Thread Kowshik Prakasam
Hi Guozhang, Thanks for the insightful feedback and questions! I have updated the KIP in response to some of the suggestions. Please find my response below. > 1. Could you explain a bit what would the "the set of features supported by > a broker" information, beyond the cluster-level finalized

Build failed in Jenkins: kafka-trunk-jdk11 #1326

2020-04-06 Thread Apache Jenkins Server
See Changes: [github] KAFKA-9784: Add OffsetFetch to group concurrency test (#8383) [github] KAFKA-9793: Expand the try-catch for task commit in HandleAssignment --

Re: [DISCUSS] KIP-585: Conditional SMT

2020-04-06 Thread Tom Bentley
Hi, Hi all, Thanks for the discussion so far. It seems a bit weird to me that when configuring the Conditional SMT with a DSL you would use a concise, intuitive DSL for expressing the condition, but not for the transforms that it's guarding. It also seems natural, if you support this for