Re: [DISCUSS] KIP-712: Shallow Mirroring

2021-04-01 Thread Henry Cai
Jun, On the EOS support, I looked at KIP-98 and it seems to me all the control is specified in the RecordBatch header fields: 1. Whether the batch is a control batch which contains commit or abort marker 2. Whether the batch is transactional which contains PID In read_committed isolation level

Build failed in Jenkins: Kafka » kafka-trunk-jdk8 #633

2021-04-01 Thread Apache Jenkins Server
See Changes: [github] KAFKA-12587 Remove KafkaPrincipal#fromString for 3.0 (#10447) -- [...truncated 7.35 MB...] TransactionsTest >

Build failed in Jenkins: Kafka » kafka-trunk-jdk15 #693

2021-04-01 Thread Apache Jenkins Server
See Changes: [github] KAFKA-12587 Remove KafkaPrincipal#fromString for 3.0 (#10447) [github] MINOR: support ImplicitLinkedHashCollection#sort (#10456) [github] MONOR: Remove redudant

Build failed in Jenkins: Kafka » kafka-trunk-jdk11 #662

2021-04-01 Thread Apache Jenkins Server
See Changes: [github] KAFKA-12587 Remove KafkaPrincipal#fromString for 3.0 (#10447) [github] MINOR: support ImplicitLinkedHashCollection#sort (#10456) --

Re: Re: [DISCUSS] KIP-706: Add method "Producer#produce" to return CompletionStage instead of Future

2021-04-01 Thread Ismael Juma
To avoid a separate KIP, maybe we can agree on the grace period as part of this KIP. Maybe 3 releases (~1 year) is a good target? Ismael On Thu, Apr 1, 2021, 6:39 PM Chia-Ping Tsai wrote: > > Deprecating `send` is going to be extremely disruptive to all existing > > users (if you use -Werror,

[jira] [Created] (KAFKA-12606) Some improvements for produce record validation

2021-04-01 Thread Jason Gustafson (Jira)
Jason Gustafson created KAFKA-12606: --- Summary: Some improvements for produce record validation Key: KAFKA-12606 URL: https://issues.apache.org/jira/browse/KAFKA-12606 Project: Kafka Issue

Re: Re: [DISCUSS] KIP-706: Add method "Producer#produce" to return CompletionStage instead of Future

2021-04-01 Thread Chia-Ping Tsai
> Deprecating `send` is going to be extremely disruptive to all existing > users (if you use -Werror, it will require updating every call site). Have > we considered encouraging the usage of the new method while not deprecating > the old methods? We could consider deprecation down the line. The

Re: Re: [DISCUSS] KIP-706: Add method "Producer#produce" to return CompletionStage instead of Future

2021-04-01 Thread Chia-Ping Tsai
hi Tom, thanks for all your suggestions! > 2. I'm not sure that having separate Builder.topic() and .partition() > methods is better than forcing people to set the target via a single method > call. For example, `Builder.target(String topic)`, `Builder.target(String > topic, int partition)`,

Build failed in Jenkins: Kafka » kafka-2.8-jdk8 #89

2021-04-01 Thread Apache Jenkins Server
See Changes: [Randall Hauch] KAFKA-12474: Handle failure to write new session keys gracefully (#10396) -- [...truncated 3.62 MB...] SocketServerTest >

Build failed in Jenkins: Kafka » kafka-trunk-jdk11 #661

2021-04-01 Thread Apache Jenkins Server
See Changes: [github] KAFKA-12426: Missing logic to create partition.metadata files in RaftReplicaManager (#10282) -- [...truncated 3.69 MB...]

[jira] [Resolved] (KAFKA-12587) Remove KafkaPrincipal#fromString for 3.0

2021-04-01 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12587?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai resolved KAFKA-12587. Resolution: Fixed > Remove KafkaPrincipal#fromString for 3.0 >

Build failed in Jenkins: Kafka » kafka-trunk-jdk8 #632

2021-04-01 Thread Apache Jenkins Server
See Changes: [github] KAFKA-12426: Missing logic to create partition.metadata files in RaftReplicaManager (#10282) -- [...truncated 3.67 MB...]

Build failed in Jenkins: Kafka » kafka-trunk-jdk15 #692

2021-04-01 Thread Apache Jenkins Server
See Changes: [github] KAFKA-12426: Missing logic to create partition.metadata files in RaftReplicaManager (#10282) -- [...truncated 3.69 MB...]

Jenkins build is back to normal : Kafka » kafka-2.6-jdk8 #112

2021-04-01 Thread Apache Jenkins Server
See

Re: [DISCUSS] KIP-718: Make KTable Join on Foreign key unopinionated

2021-04-01 Thread John Roesler
Thanks Marco, Sorry if I caused any trouble! I don’t remember what I was thinking before, but reasoning about it now, you might need the fine-grained choice if: 1. The number or size of records in each partition of both tables is small(ish), but the cardinality of the join is very high. Then

[jira] [Created] (KAFKA-12605) kafka consumer churns through buffer memory iterating over records

2021-04-01 Thread radai rosenblatt (Jira)
radai rosenblatt created KAFKA-12605: Summary: kafka consumer churns through buffer memory iterating over records Key: KAFKA-12605 URL: https://issues.apache.org/jira/browse/KAFKA-12605 Project:

Re: [DISCUSS] KIP-725: Streamlining configurations for TimeWindowedDeserializer.

2021-04-01 Thread John Roesler
Hi Sagar, I think this is a good proposal. The only change I might recommend is to change the config to more closely align with the other one, for example: “window.inner.class.deserializer” But it’s very minor; I leave it to your judgement. Thanks, John On Fri, Mar 26, 2021, at 03:36, Sagar

Re: [VOTE] KIP-720 Deprecate MirrorMaker v1

2021-04-01 Thread Ismael Juma
Thank you Ryanne. It's hopefully obvious, but I meant MirrorMaker 1 in the following: "we propose deprecating MirrorMaker 2 for future removal." On Thu, Apr 1, 2021 at 3:50 PM Ryanne Dolan wrote: > Ismael, that certainly works for me. I'll update the KIP. Thanks for > raising the issue. > >

Re: [VOTE] KIP-720 Deprecate MirrorMaker v1

2021-04-01 Thread Ryanne Dolan
Ismael, that certainly works for me. I'll update the KIP. Thanks for raising the issue. Ryanne On Thu, Apr 1, 2021, 11:45 AM Ismael Juma wrote: > OK. :) Maybe something like: > > "We believe MirrorMaker 2 is an improvement over the original MirrorMaker > when it comes to reliability and

Re: [VOTE] KIP-720 Deprecate MirrorMaker v1

2021-04-01 Thread Ryanne Dolan
Colin, the only feature gap I'm aware of is that users must provide their own ReplicationPolicy in order to replicate topics without renaming them. This is straightforward, and such ReplicationPolicy implementations are easy to find. We could provide one OOTB, and indeed KIP-382 proposes we do so,

[jira] [Resolved] (KAFKA-12426) Missing logic to create partition.metadata files in RaftReplicaManager

2021-04-01 Thread Jason Gustafson (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12426?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-12426. - Fix Version/s: 3.0.0 Resolution: Fixed > Missing logic to create

Re: [DISCUSS] KIP-712: Shallow Mirroring

2021-04-01 Thread Ryanne Dolan
Henry, a suggestion: instead of introducing a new configuration property which enables the proposed behavior in the existing clients, we could expose this behavior only thru package-private classes (RecordBatchConsumer/Producer or something) which wrap or extend the existing clients. In other

[DISCUSS] KIP-729 Custom validation of records on the broker prior to log append

2021-04-01 Thread Soumyajit Sahu
Hello All, I would like to start a discussion on the KIP-729. https://cwiki.apache.org/confluence/display/KAFKA/KIP-729%3A+Custom+validation+of+records+on+the+broker+prior+to+log+append Thanks! Soumyajit

[DISCUSS] Please review 2.8.0 blog post

2021-04-01 Thread John Roesler
Hello all, In the steady march toward the Apache Kafka 2.8.0 release, I have prepared a draft of the release announcement post: https://blogs.apache.org/preview/kafka/?previewEntry=what-s-new-in-apache5 If you have a moment, I would greatly appreciate your reviews. Thank you, -John

Build failed in Jenkins: Kafka » kafka-trunk-jdk11 #660

2021-04-01 Thread Apache Jenkins Server
See Changes: [github] KAFKA-12474: Handle failure to write new session keys gracefully (#10396) [github] MINOR: Improve reproducability of raft simulation tests (#10422)

Build failed in Jenkins: Kafka » kafka-trunk-jdk8 #631

2021-04-01 Thread Apache Jenkins Server
See Changes: [github] KAFKA-12474: Handle failure to write new session keys gracefully (#10396) [github] MINOR: Improve reproducability of raft simulation tests (#10422)

[jira] [Resolved] (KAFKA-12474) Worker can die if unable to write new session key

2021-04-01 Thread Randall Hauch (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12474?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Randall Hauch resolved KAFKA-12474. --- Fix Version/s: 2.6.2 2.7.1 2.5.2

Build failed in Jenkins: Kafka » kafka-trunk-jdk15 #691

2021-04-01 Thread Apache Jenkins Server
See Changes: [github] KAFKA-12474: Handle failure to write new session keys gracefully (#10396) [github] MINOR: Improve reproducability of raft simulation tests (#10422)

Re: [VOTE] KIP-707: The future of KafkaFuture

2021-04-01 Thread Colin McCabe
+1 (binding). Thanks for the KIP. Colin On Tue, Mar 30, 2021, at 20:36, Chia-Ping Tsai wrote: > Thanks for this KIP. +1 (binding) > > On 2021/03/29 15:34:55, Tom Bentley wrote: > > Hi, > > > > I'd like to start a vote on KIP-707, which proposes to add > > KafkaFuture.toCompletionStage(),

Re: [DISCUSS] KIP-712: Shallow Mirroring

2021-04-01 Thread Colin McCabe
On Thu, Apr 1, 2021, at 09:45, Jun Rao wrote: > Hi, Henry, > > Thanks for the response. > > 1. I agree with Tom that it's worth thinking about a separate class for > shallow iteration instead of trying to add more complexity into the > existing producer/consumer API. We could potentially make

Re: [VOTE] KIP-720 Deprecate MirrorMaker v1

2021-04-01 Thread Colin McCabe
Thanks for bringing this up, Ismael. I agree that we need to figure this out before we accept this KIP. If MM1 is deprecated, then that means we are telling users they need to migrate away from it as soon as they can. I think that rules out adding big new features to MM1, unless those

Build failed in Jenkins: Kafka » kafka-trunk-jdk11 #659

2021-04-01 Thread Apache Jenkins Server
See Changes: [github] KAFKA-12593: Fix Apache License headers (#10452) -- [...truncated 3.69 MB...] KafkaZkClientTest > testUpdateBrokerInfo() STARTED

Build failed in Jenkins: Kafka » kafka-2.6-jdk8 #111

2021-04-01 Thread Apache Jenkins Server
See Changes: [John Roesler] KAFKA-12593: Fix Apache License headers (#10452) -- [...truncated 3.18 MB...] org.apache.kafka.streams.TopologyTestDriverTest >

Build failed in Jenkins: Kafka » kafka-trunk-jdk8 #630

2021-04-01 Thread Apache Jenkins Server
See Changes: [github] KAFKA-12593: Fix Apache License headers (#10452) -- [...truncated 3.67 MB...] LogValidatorTest >

[jira] [Created] (KAFKA-12604) Remove envelope handling from broker

2021-04-01 Thread Jason Gustafson (Jira)
Jason Gustafson created KAFKA-12604: --- Summary: Remove envelope handling from broker Key: KAFKA-12604 URL: https://issues.apache.org/jira/browse/KAFKA-12604 Project: Kafka Issue Type:

Build failed in Jenkins: Kafka » kafka-2.8-jdk8 #88

2021-04-01 Thread Apache Jenkins Server
See Changes: [John Roesler] KAFKA-12593: Fix Apache License headers (#10452) -- [...truncated 3.61 MB...] PasswordEncoderTest > testEncodeDecodeAlgorithms()

Build failed in Jenkins: Kafka » kafka-trunk-jdk15 #690

2021-04-01 Thread Apache Jenkins Server
See Changes: [github] KAFKA-12600: Remove deprecated config value `default` for client config `client.dns.lookup` (#10458) [github] MINOR: Fix typo in MirrorMaker v2 documentation (#10433)

Build failed in Jenkins: Kafka » kafka-trunk-jdk11 #658

2021-04-01 Thread Apache Jenkins Server
See Changes: [github] KAFKA-12600: Remove deprecated config value `default` for client config `client.dns.lookup` (#10458) [github] MINOR: Fix typo in MirrorMaker v2 documentation (#10433)

Build failed in Jenkins: Kafka » kafka-2.8-jdk8 #87

2021-04-01 Thread Apache Jenkins Server
See Changes: [Ismael Juma] KAFKA-12583: Upgrade netty to 4.1.62.Final -- [...truncated 7.23 MB...] PasswordEncoderTest > testEncodeDecodeAlgorithms() STARTED

Build failed in Jenkins: Kafka » kafka-trunk-jdk8 #629

2021-04-01 Thread Apache Jenkins Server
See Changes: [github] KAFKA-12600: Remove deprecated config value `default` for client config `client.dns.lookup` (#10458) [github] MINOR: Fix typo in MirrorMaker v2 documentation (#10433)

[jira] [Created] (KAFKA-12603) Add benchmarks for handleFetchRequest and FetchContext

2021-04-01 Thread Justine Olshan (Jira)
Justine Olshan created KAFKA-12603: -- Summary: Add benchmarks for handleFetchRequest and FetchContext Key: KAFKA-12603 URL: https://issues.apache.org/jira/browse/KAFKA-12603 Project: Kafka

Re: [DISCUSS] KIP-712: Shallow Mirroring

2021-04-01 Thread Jun Rao
Hi, Henry, Thanks for the response. 1. I agree with Tom that it's worth thinking about a separate class for shallow iteration instead of trying to add more complexity into the existing producer/consumer API. We could potentially make the new class an internal API if it's only useful for MM. 3.

Re: [VOTE] KIP-720 Deprecate MirrorMaker v1

2021-04-01 Thread Ismael Juma
OK. :) Maybe something like: "We believe MirrorMaker 2 is an improvement over the original MirrorMaker when it comes to reliability and functionality for the majority of use cases. We intend to focus on MirrorMaker 2 for future development and hence we propose deprecating MirrorMaker 2 for future

[GitHub] [kafka-site] agapic closed pull request #339: MINOR: Added Axon to powered-by page

2021-04-01 Thread GitBox
agapic closed pull request #339: URL: https://github.com/apache/kafka-site/pull/339 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service,

Re: [VOTE] KIP-720 Deprecate MirrorMaker v1

2021-04-01 Thread Ryanne Dolan
Ah, do you mind wording it for me, Ismael? Or do you mean I should just remove the "MM1 is still useful" part? Ryanne On Thu, Apr 1, 2021, 11:01 AM Ismael Juma wrote: > Can we please add proper motivation? I'm -1 with the current motivation > even though I'm in favor of the change. > > On Thu,

Re: [VOTE] KIP-720 Deprecate MirrorMaker v1

2021-04-01 Thread Ismael Juma
Can we please add proper motivation? I'm -1 with the current motivation even though I'm in favor of the change. On Thu, Apr 1, 2021, 8:46 AM Ryanne Dolan wrote: > Hey y'all, looks like we've got the requisite votes for this to pass, and > the various concerns wrt KIP-712 are now being discussed

[jira] [Resolved] (KAFKA-12593) Some Scala, Python, and Gradle files contain the wrong license header

2021-04-01 Thread John Roesler (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12593?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] John Roesler resolved KAFKA-12593. -- Resolution: Fixed > Some Scala, Python, and Gradle files contain the wrong license header >

Re: [VOTE] KIP-720 Deprecate MirrorMaker v1

2021-04-01 Thread Ryanne Dolan
Hey y'all, looks like we've got the requisite votes for this to pass, and the various concerns wrt KIP-712 are now being discussed on that thread. So I'm going to go ahead and close the vote here. Thanks for the votes! Ryanne On Fri, Mar 26, 2021, 11:26 PM Ismael Juma wrote: > It does mean

[jira] [Created] (KAFKA-12602) The LICENSE and NOTICE files don't list everything they should

2021-04-01 Thread John Roesler (Jira)
John Roesler created KAFKA-12602: Summary: The LICENSE and NOTICE files don't list everything they should Key: KAFKA-12602 URL: https://issues.apache.org/jira/browse/KAFKA-12602 Project: Kafka

Build failed in Jenkins: Kafka » kafka-trunk-jdk15 #689

2021-04-01 Thread Apache Jenkins Server
See Changes: [github] KAFKA-12952: Remove deprecated LogConfig.Compact (#10451) -- [...truncated 3.69 MB...] AuthorizerIntegrationTest >

Build failed in Jenkins: Kafka » kafka-trunk-jdk11 #657

2021-04-01 Thread Apache Jenkins Server
See Changes: [github] KAFKA-12952: Remove deprecated LogConfig.Compact (#10451) -- [...truncated 3.69 MB...] KafkaZkClientTest > testPropagateLogDir()

Build failed in Jenkins: Kafka » kafka-trunk-jdk8 #628

2021-04-01 Thread Apache Jenkins Server
See Changes: [github] KAFKA-12952: Remove deprecated LogConfig.Compact (#10451) -- [...truncated 3.67 MB...] LogValidatorTest >

[jira] [Created] (KAFKA-12601) Remove deprecated `delegation.token.master.key`

2021-04-01 Thread Ismael Juma (Jira)
Ismael Juma created KAFKA-12601: --- Summary: Remove deprecated `delegation.token.master.key` Key: KAFKA-12601 URL: https://issues.apache.org/jira/browse/KAFKA-12601 Project: Kafka Issue Type:

Re: [DISCUSS] KIP-712: Shallow Mirroring

2021-04-01 Thread Tom Bentley
Hi Henry, Jun and Ismael, A few things make me wonder if building this into the existing Producer and Consumer APIs is really the right thing to do: 1. Type safety. The existing Producer and Consumer are both generic in K and V, but those type parameters are meaningless in the batch case. For

[jira] [Resolved] (KAFKA-12583) Upgrade of netty-codec due to CVE-2021-21295

2021-04-01 Thread Ismael Juma (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12583?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma resolved KAFKA-12583. - Fix Version/s: 2.8.0 Resolution: Fixed > Upgrade of netty-codec due to CVE-2021-21295 >

[jira] [Created] (KAFKA-12600) Remove deprecated config value `default` for client config `client.dns.lookup`

2021-04-01 Thread Ismael Juma (Jira)
Ismael Juma created KAFKA-12600: --- Summary: Remove deprecated config value `default` for client config `client.dns.lookup` Key: KAFKA-12600 URL: https://issues.apache.org/jira/browse/KAFKA-12600

[jira] [Created] (KAFKA-12598) Remove deprecated --zookeeper in ConfigCommand

2021-04-01 Thread Luke Chen (Jira)
Luke Chen created KAFKA-12598: - Summary: Remove deprecated --zookeeper in ConfigCommand Key: KAFKA-12598 URL: https://issues.apache.org/jira/browse/KAFKA-12598 Project: Kafka Issue Type:

[jira] [Created] (KAFKA-12599) Remove deprecated --zookeeper in preferredReplicaLeaderElectionCommand

2021-04-01 Thread Luke Chen (Jira)
Luke Chen created KAFKA-12599: - Summary: Remove deprecated --zookeeper in preferredReplicaLeaderElectionCommand Key: KAFKA-12599 URL: https://issues.apache.org/jira/browse/KAFKA-12599 Project: Kafka

[jira] [Created] (KAFKA-12597) Remove deprecated --zookeeper in ReassignPartitionsCommands

2021-04-01 Thread Luke Chen (Jira)
Luke Chen created KAFKA-12597: - Summary: Remove deprecated --zookeeper in ReassignPartitionsCommands Key: KAFKA-12597 URL: https://issues.apache.org/jira/browse/KAFKA-12597 Project: Kafka Issue

[jira] [Created] (KAFKA-12596) Remove deprecated --zookeeper in topicCommands

2021-04-01 Thread Luke Chen (Jira)
Luke Chen created KAFKA-12596: - Summary: Remove deprecated --zookeeper in topicCommands Key: KAFKA-12596 URL: https://issues.apache.org/jira/browse/KAFKA-12596 Project: Kafka Issue Type:

Re: [DISCUSS] KIP-712: Shallow Mirroring

2021-04-01 Thread Henry Cai
Jun, Thanks for your insight looking into this KIP, we do believe the shallow iteration will give quite a significant performance boost. On your concerns: 1. Cleaner API. One alternative is to create new batch APIs. On consumer, it would become Consumer.pollBatch returns a ConsumerBatch