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

2020-04-03 Thread Apache Jenkins Server
See Changes: [github] KAFKA-6145: Pt. 5 Implement high availability assignment (#8337) [github] KAFKA-9778: Add methods to validate and assert connector configurations [github] KAFKA-9809: Shrink transaction ti

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

2020-04-03 Thread Kowshik Prakasam
Hey Boyang, Great point! You are right, thanks for the suggestion! Yes, we can just use ZK watches to propagate finalized features information. I have updated the KIP write up with this change. As a result, I feel the design is simpler as we have also eliminated the changes to UpdateMetadataReques

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

2020-04-03 Thread Kowshik Prakasam
Hey Jun, Thanks a lot for the great feedback! Please note that the design has changed a little bit on the KIP, and we now propagate the finalized features metadata only via ZK watches (instead of UpdateMetadataRequest from the controller). Please find below my response to your questions/feedback,

[jira] [Created] (KAFKA-9813) __consumer_offsets loaded with a very long time

2020-04-03 Thread leibo (Jira)
leibo created KAFKA-9813: Summary: __consumer_offsets loaded with a very long time Key: KAFKA-9813 URL: https://issues.apache.org/jira/browse/KAFKA-9813 Project: Kafka Issue Type: Bug Compo

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

2020-04-03 Thread Apache Jenkins Server
See Changes: [github] KAFKA-9778: Add methods to validate and assert connector configurations -- [...truncated 2.37 MB...] org.apache.kafka.streams.state.internals.Cach

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

2020-04-03 Thread Kowshik Prakasam
Hi all, Any other feedback on this KIP before we start the vote? Cheers, Kowshik On Fri, Apr 3, 2020 at 1:27 AM Kowshik Prakasam wrote: > Hey Jun, > > Thanks a lot for the great feedback! Please note that the design > has changed a little bit on the KIP, and we now propagate the finalized > f

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

2020-04-03 Thread Apache Jenkins Server
See Changes: [github] KAFKA-9809: Shrink transaction timeout for streams (#8407) [github] KAFKA-9776: Downgrade TxnCommit API v3 when broker doesn't support -- [...trun

[jira] [Created] (KAFKA-9814) Allow AdminClient.listTopics to list based on expression

2020-04-03 Thread Viktor Somogyi-Vass (Jira)
Viktor Somogyi-Vass created KAFKA-9814: -- Summary: Allow AdminClient.listTopics to list based on expression Key: KAFKA-9814 URL: https://issues.apache.org/jira/browse/KAFKA-9814 Project: Kafka

Re: [DISCUSS] KIP-586: Deprecate commit records without record metadata

2020-04-03 Thread Randall Hauch
Hi, Mario. Thanks for creating this small but useful KIP! The only suggestion I have is to include the JavaDoc for the method we want to deprecate so that we can include the line: * @deprecated Use {@link #commitRecord(SourceRecord, RecordMetadata)} This will make it more clear that there are t

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

2020-04-03 Thread Rajini Sivaram (Jira)
Rajini Sivaram created KAFKA-9815: - Summary: Consumer may never re-join if inconsistent metadata is received once Key: KAFKA-9815 URL: https://issues.apache.org/jira/browse/KAFKA-9815 Project: Kafka

[jira] [Resolved] (KAFKA-9812) Integration tests hang and timeout the entire PR build on jenkins

2020-04-03 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9812?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax resolved KAFKA-9812. Fix Version/s: 2.6.0 Resolution: Fixed > Integration tests hang and timeout the enti

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

2020-04-03 Thread Boyang Chen
Hey Kowshik, thanks for getting the KIP updated. The Zookeeper routing approach makes sense and simplifies the changes. Some follow-ups: 1. Do you mind updating the non-goal section as we are introducing a --feature-force-downgrade to address downgrade concern? 2. For the flags `--feature` seems

Re: [DISCUSS] KIP-585: Conditional SMT

2020-04-03 Thread Gunnar Morling
Hi all, Thanks a lot for this initiative, Tom! To shed some light, the use case where this first came up, were issues we saw with SMTs being applied to the different topics produced by the Debezium change data capture connectors. There are different kinds of topics (for change data, schema histor

[jira] [Resolved] (KAFKA-9809) Shrink transaction timeout for Streams

2020-04-03 Thread Boyang Chen (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9809?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Boyang Chen resolved KAFKA-9809. Resolution: Fixed > Shrink transaction timeout for Streams > --

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

2020-04-03 Thread Apache Jenkins Server
See Changes: [github] KAFKA-9739: Fixes null key changing child node (#8400) -- [...truncated 2.99 MB...] org.apache.kafka.streams.MockTimeTest > shouldSetStartTime STAR

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

2020-04-03 Thread Jun Rao
Hi, Kowshik, Thanks for the reply. A few more comments below. 100.6 For every new request, the admin needs to control who is allowed to issue that request if security is enabled. So, we need to assign the new request a ResourceType and possible AclOperations. See https://cwiki.apache.org/confluen

[jira] [Resolved] (KAFKA-9750) Flaky test kafka.server.ReplicaManagerTest.testFencedErrorCausedByBecomeLeader

2020-04-03 Thread Jason Gustafson (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9750?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-9750. Resolution: Fixed > Flaky test kafka.server.ReplicaManagerTest.testFencedErrorCausedByBecom

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

2020-04-03 Thread Connor Penhale
Hi Chris! Thanks for your feedback! I'll number my responses to your questions / thoughts. 1. Apologies on that lack of clarity! I settled on "Detailed exception information has been suppressed. Please see logs." (https://github.com/apache/kafka/pull/8355/files#diff-64c265986e7bbe40cdd79f831e96

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

2020-04-03 Thread Kowshik Prakasam
Hey Boyang, Thanks for the feedback! I've updated the KIP. Please find below my response. > 1. Do you mind updating the non-goal section as we are introducing a > --feature-force-downgrade to address downgrade concern? (Kowshik): This was already mentioned. Look for non-goal: 1-b. > 2. For the

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

2020-04-03 Thread Apache Jenkins Server
See Changes: [github] KAFKA-9739: Fixes null key changing child node (#8400) -- [...truncated 6.02 MB...] org.apache.kafka.streams.TopologyTestDriverTest > shouldThrow

[jira] [Created] (KAFKA-9816) Group Coordinator should be stricter about leader returned assignment

2020-04-03 Thread Guozhang Wang (Jira)
Guozhang Wang created KAFKA-9816: Summary: Group Coordinator should be stricter about leader returned assignment Key: KAFKA-9816 URL: https://issues.apache.org/jira/browse/KAFKA-9816 Project: Kafka

[jira] [Resolved] (KAFKA-9139) Dynamic broker config types aren't being discovered

2020-04-03 Thread Brian Byrne (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9139?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Brian Byrne resolved KAFKA-9139. Resolution: Duplicate > Dynamic broker config types aren't being discovered > -

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

2020-04-03 Thread Colin McCabe
Hi Kowshik, The discussion on ZooKeeper reads versus writes makes sense to me. The important thing to keep in mind here is that in the bridge release, all brokers can read from ZooKeeper, but only the controller writes. Why do we need both UpdateFeaturesRequest and DeleteFeaturesRequest? It s

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

2020-04-03 Thread Colin McCabe
Hi Connor, If we are putting security-sensitive information into REST responses, that is a bug that needs to be fixed, not worked around with a configuration option. Do you have an example of security-sensitive information appearing in the exception text? Why do you feel that PCI-DSS requires

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

2020-04-03 Thread Colin McCabe
Also, if you do find a security issue, the process to follow is here: https://kafka.apache.org/project-security.html . best, Colin On Fri, Apr 3, 2020, at 14:20, Colin McCabe wrote: > Hi Connor, > > If we are putting security-sensitive information into REST responses, > that is a bug that nee

Re: [DISCUSS] KIP-586: Deprecate commit records without record metadata

2020-04-03 Thread Mario Molina
Hi, I've updated the comment in the KIP (even though it was already included in the PR). Thanks! Mario On Fri, 3 Apr 2020 at 09:01, Randall Hauch wrote: > Hi, Mario. > > Thanks for creating this small but useful KIP! The only suggestion I have > is to include the JavaDoc for the method we want

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

2020-04-03 Thread Apache Jenkins Server
See Changes: [github] KAFKA-9812: fix infinite loop in test code (#8411) [manikumar] KAFKA-9775: Fix IllegalFormatConversionException in ToolsUtils -- [...truncated 5.99

[jira] [Created] (KAFKA-9817) Poor performance for electLeaders controller RPC

2020-04-03 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-9817: --- Summary: Poor performance for electLeaders controller RPC Key: KAFKA-9817 URL: https://issues.apache.org/jira/browse/KAFKA-9817 Project: Kafka Issue Type: Bug

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

2020-04-03 Thread Apache Jenkins Server
See Changes: [github] KAFKA-9812: fix infinite loop in test code (#8411) [manikumar] KAFKA-9775: Fix IllegalFormatConversionException in ToolsUtils [github] KAFKA-9750; Fix race condition with log dir reassign

[jira] [Created] (KAFKA-9818) Flaky Test RecordCollectorTest.shouldNotThrowStreamsExceptionOnSubsequentCallIfASendFailsWithContinueExceptionHandler

2020-04-03 Thread Sophie Blee-Goldman (Jira)
Sophie Blee-Goldman created KAFKA-9818: -- Summary: Flaky Test RecordCollectorTest.shouldNotThrowStreamsExceptionOnSubsequentCallIfASendFailsWithContinueExceptionHandler Key: KAFKA-9818 URL: https://issues.apac

[jira] [Created] (KAFKA-9819) Flaky Test StoreChangelogReaderTest#shouldNotThrowOnUnknownRevokedPartition[0]

2020-04-03 Thread Sophie Blee-Goldman (Jira)
Sophie Blee-Goldman created KAFKA-9819: -- Summary: Flaky Test StoreChangelogReaderTest#shouldNotThrowOnUnknownRevokedPartition[0] Key: KAFKA-9819 URL: https://issues.apache.org/jira/browse/KAFKA-9819

[jira] [Resolved] (KAFKA-9807) Race condition updating high watermark allows reads above LSO

2020-04-03 Thread Jason Gustafson (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9807?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-9807. Fix Version/s: 2.4.2 2.5.0 Resolution: Fixed Resolving this. I wi

Build failed in Jenkins: kafka-2.1-jdk8 #263

2020-04-03 Thread Apache Jenkins Server
See Changes: [konstantine] KAFKA-9810: Document Connect Root REST API on / (#8408) -- [...truncated 686.57 KB...] } else if (!path.equals(ZkUtils.ConsumersPath)) {

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

2020-04-03 Thread Apache Jenkins Server
See Changes: [github] KAFKA-9750; Fix race condition with log dir reassign completion (#8412) [github] MINOR: clean up Streams assignment classes and tests (#8406) [github] KAFKA-9810: Document Connect Root RES

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

2020-04-03 Thread Boyang Chen
Hey all, I would like to start off the discussion for KIP-590, a follow-up initiative after KIP-500: https://cwiki.apache.org/confluence/display/KAFKA/KIP-590%3A+Redirect+Zookeeper+Mutation+Protocols+to+The+Controller This KIP proposes to migrate existing Zookeeper mutation paths, including confi

[jira] [Created] (KAFKA-9820) validateMessagesAndAssignOffsetsCompressed allocates batch iterator which is not used

2020-04-03 Thread Lucas Bradstreet (Jira)
Lucas Bradstreet created KAFKA-9820: --- Summary: validateMessagesAndAssignOffsetsCompressed allocates batch iterator which is not used Key: KAFKA-9820 URL: https://issues.apache.org/jira/browse/KAFKA-9820

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

2020-04-03 Thread Apache Jenkins Server
See Changes: [github] KAFKA-9807; Protect LSO reads from concurrent high-watermark updates -- [...truncated 3.00 MB...] org.apache.kafka.streams.TopologyTestDriverTest

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

2020-04-03 Thread Kowshik Prakasam
Hi Jun, Thanks for the feedback and suggestions. Please find my response below. > 100.6 For every new request, the admin needs to control who is allowed to > issue that request if security is enabled. So, we need to assign the new > request a ResourceType and possible AclOperations. See > https:/

[jira] [Created] (KAFKA-9821) Partition may skip assignment with static members and incremental rebalances

2020-04-03 Thread Guozhang Wang (Jira)
Guozhang Wang created KAFKA-9821: Summary: Partition may skip assignment with static members and incremental rebalances Key: KAFKA-9821 URL: https://issues.apache.org/jira/browse/KAFKA-9821 Project: K

Jenkins build is back to normal : kafka-2.5-jdk8 #87

2020-04-03 Thread Apache Jenkins Server
See

Jenkins build is back to normal : kafka-2.3-jdk8 #195

2020-04-03 Thread Apache Jenkins Server
See

Re: 2.5.0 Producer.sendOffsetsToTransaction() and Backwards Compatibility

2020-04-03 Thread Matthias J. Sax
I guess you would need to catch the exception and retry? It's a little unfortunate. Not sure if we could back-port the internal producer config that we add in 2.6 for auto-downgrade to a 2.5 bug fix release? -Matthias On 4/2/20 7:25 PM, Gary Russell wrote: > Thanks Mattias > >> Hence, why do

Build failed in Jenkins: kafka-2.4-jdk8 #182

2020-04-03 Thread Apache Jenkins Server
See Changes: [github] KAFKA-9739: Fixes null key changing child node (#8416) [jason] KAFKA-9750; Fix race condition with log dir reassign completion (#8412) -- [...truncat

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

2020-04-03 Thread Kowshik Prakasam
Hi Colin, Thanks for the feedback! I have updated the KIP based on your feedback. Please find my response below. > The discussion on ZooKeeper reads versus writes makes sense to me. The important thing to keep in mind here is that in the bridge release, > all brokers can read from ZooKeeper, but

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

2020-04-03 Thread Apache Jenkins Server
See Changes: [github] MINOR: Refactor StreamsProducer (#8380) -- [...truncated 2.08 MB...] org.apache.kafka.connect.transforms.CastTest > castWholeRecordValueWithSchem

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

2020-04-03 Thread Apache Jenkins Server
See Changes: [konstantine] KAFKA-9810: Document Connect Root REST API on / (#8408) [jason] KAFKA-9807; Protect LSO reads from concurrent high-watermark updates -- [...trunca

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

2020-04-03 Thread Apache Jenkins Server
See Changes: [github] MINOR: Refactor StreamsProducer (#8380) -- [...truncated 2.06 MB...] org.apache.kafka.connect.transforms.CastTest > testConfigEmpty STARTED org.apa

Re: [DISCUSS] KIP-579: new exception on min.insync.replicas > replication.factor

2020-04-03 Thread James Cheng
> On Apr 2, 2020, at 4:27 AM, Paolo Moriello wrote: > > Hi, > > Thanks a lot for your feedback, I really appreciate your help on this. > > Given what you suggested, I will take some time to update the kip with a > proposal to make invalid configuration requests FAIL. This involves > checking