Re: [DISCUSS] KIP-478 Strongly Typed Processor API

2019-06-21 Thread Matthias J. Sax
I think `RecordProcessor` is a good name. -Matthias On 6/21/19 5:09 PM, John Roesler wrote: > After kicking the naming around a bit more, it seems like any package > name change is a bit "weird" because it fragments the package and > directory structure. If we can come up with a reasonable name

Re: [DISCUSS] KIP-466: Add support for List serialization and deserialization

2019-06-21 Thread Matthias J. Sax
Yes, something like this. I did not think about good configuration parameter names yet. I am also not sure if I understand all proposed configs atm. But all configs should be listed and explained in the KIP anyway, and we can discuss further after you have updated the KIP (I can ask more detailed

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

2019-06-21 Thread Apache Jenkins Server
See Changes: [github] MINOR: Reflection free implementation of `defaultKerberosRealm` (#6978) -- [...truncated 2.52 MB...] org.apache.kafka.clients.InFlightRequestsTest

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

2019-06-21 Thread Apache Jenkins Server
See Changes: [wangguoz] MINOR: Fix DistributedHerderTest after adding reason to maybeLeaveGroup [github] KAFKA-8106: Skipping ByteBuffer allocation of key / value / headers in

Re: Preliminary blog post for the Apache Kafka 2.3.0 release

2019-06-21 Thread Ismael Juma
Thanks Colin! Maybe we should mention that restarts are much faster when you have a lot of partitions: https://issues.apache.org/jira/browse/KAFKA-7283 "We did experiments on a cluster with 15 brokers, each of which has ~3k segments (and there are 31.8k partitions with RF=3 which are evenly

Re: Preliminary blog post for the Apache Kafka 2.3.0 release

2019-06-21 Thread Konstantine Karantasis
Looks great to me Colin. Thanks for the small adjustments! Cheers, Konstantine On Thu, Jun 20, 2019 at 9:50 AM Colin McCabe wrote: > On Thu, Jun 20, 2019, at 00:23, Matthias J. Sax wrote: > > Great blog post, Colin! > > > > Two comments: > > > > > > (1) KIP-258: "future features" -> "the

[jira] [Created] (KAFKA-8586) Source task producers silently fail to send records

2019-06-21 Thread Chris Egerton (JIRA)
Chris Egerton created KAFKA-8586: Summary: Source task producers silently fail to send records Key: KAFKA-8586 URL: https://issues.apache.org/jira/browse/KAFKA-8586 Project: Kafka Issue

[jira] [Created] (KAFKA-8585) Controller should make LeaderAndIsr updates optimistically

2019-06-21 Thread Jason Gustafson (JIRA)
Jason Gustafson created KAFKA-8585: -- Summary: Controller should make LeaderAndIsr updates optimistically Key: KAFKA-8585 URL: https://issues.apache.org/jira/browse/KAFKA-8585 Project: Kafka

Re: [DISCUSS] KIP-478 Strongly Typed Processor API

2019-06-21 Thread John Roesler
After kicking the naming around a bit more, it seems like any package name change is a bit "weird" because it fragments the package and directory structure. If we can come up with a reasonable name for the interface after all, it seems like the better choice. The real challenge is that the

Re: [VOTE] 2.3.0 RC3

2019-06-21 Thread Guozhang Wang
Hi Colin, Thanks for the new RC, +1 (binding). Verified the javadoc, maven repo, and ran unit tests on 2.12 binary. Guozhang On Fri, Jun 21, 2019 at 1:23 PM Colin McCabe wrote: > Hi Ismael, > > Good catch. This should be fixed now. > > It seems that if the previously staged Sonatype

Re: [DISCUSS] KIP-478 Strongly Typed Processor API

2019-06-21 Thread John Roesler
Hi all, I've updated the KIP with the feedback so far. The naming question is still the biggest (only?) outstanding issue. It would be good to hear some more thoughts on it. As we stand now, there's one vote for changing the package name to something like 'typedprocessor', one for changing the

[jira] [Created] (KAFKA-8584) Allow "bytes" type to generated a ByteBuffer rather than byte arrays

2019-06-21 Thread Guozhang Wang (JIRA)
Guozhang Wang created KAFKA-8584: Summary: Allow "bytes" type to generated a ByteBuffer rather than byte arrays Key: KAFKA-8584 URL: https://issues.apache.org/jira/browse/KAFKA-8584 Project: Kafka

Re: [VOTE] KIP-476: Add Java AdminClient interface

2019-06-21 Thread Matthias J. Sax
I still think, that an interface does not need to know anything about its implementation. But I am also fine if we add a factory method to the new interface if that is preferred by most people. -Matthias On 6/21/19 7:10 AM, Ismael Juma wrote: > This is even more reason not to deprecate

Re: [DISCUSS] KIP-470: TopologyTestDriver test input and output usability improvements

2019-06-21 Thread Jukka Karvanen
Hi Bill, These topic objects are only interfacing TopologyTestDriver, not affecting the internal functionality of it. In my plan the internal data structures are using those Producer/ConsumerRecords as earlier. That way I don't see how those could be affected. Jukka On Fri, 21 Jun 2019, 20:57

Re: [DISCUSS] KIP-470: TopologyTestDriver test input and output usability improvements

2019-06-21 Thread Jukka Karvanen
Hi, TestRecord is already included in wiki page. It has mainly constructors and methods to access data fields. Jukka On Fri, 21 Jun 2019, 19:05 Guozhang Wang, wrote: > 1) Got it, could you list this class along with all its functions in the > proposed public APIs as well? > > 2) Ack, thanks!

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

2019-06-21 Thread Apache Jenkins Server
See

[jira] [Created] (KAFKA-8583) Optimization for SslTransportLayer#write(ByteBuffer)

2019-06-21 Thread Mao (JIRA)
Mao created KAFKA-8583: -- Summary: Optimization for SslTransportLayer#write(ByteBuffer) Key: KAFKA-8583 URL: https://issues.apache.org/jira/browse/KAFKA-8583 Project: Kafka Issue Type: Improvement

Build failed in Jenkins: kafka-2.3-jdk8 #55

2019-06-21 Thread Apache Jenkins Server
See Changes: [jason] KAFKA-8570; Grow buffer to hold down converted records if it was -- [...truncated 2.94 MB...] kafka.zk.KafkaZkClientTest > testLogDirGetters PASSED

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

2019-06-21 Thread Apache Jenkins Server
See Changes: [jason] KAFKA-8570; Grow buffer to hold down converted records if it was [rajinisivaram] KAFKA-8557: system tests - add support for (optional) interbroker [ismael] KAFKA-8563: Remove redundant

Re: Possible implementation for KAFKA-560

2019-06-21 Thread Colin McCabe
I don't think this requires a change in the protocol. It seems like you should be able to use the high water mark to figure something out here? best, Colin On Fri, Jun 21, 2019, at 04:56, Carlos Manuel Duclos-Vergara wrote: > Hi, > > This is an ancient task, but I feel it is still current

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

2019-06-21 Thread Apache Jenkins Server
See Changes: [jason] KAFKA-8570; Grow buffer to hold down converted records if it was -- [...truncated 927.02 KB...] kafka.server.KafkaConfigTest >

Build failed in Jenkins: kafka-2.2-jdk8 #141

2019-06-21 Thread Apache Jenkins Server
See Changes: [jason] KAFKA-8570; Grow buffer to hold down converted records if it was -- [...truncated 2.75 MB...] kafka.controller.PartitionLeaderElectionAlgorithmsTest >

Re: [VOTE] 2.3.0 RC3

2019-06-21 Thread Colin McCabe
Hi Ismael, Good catch. This should be fixed now. It seems that if the previously staged Sonatype staging repositories (from the previous RCs) are not dropped as part of the release process, the new ones don't get exposed. Maybe we should document this somewhere (it was a bit of a surprise

[jira] [Created] (KAFKA-8582) Consider adding an ExpiredWindowRecordHandler to Suppress

2019-06-21 Thread John Roesler (JIRA)
John Roesler created KAFKA-8582: --- Summary: Consider adding an ExpiredWindowRecordHandler to Suppress Key: KAFKA-8582 URL: https://issues.apache.org/jira/browse/KAFKA-8582 Project: Kafka Issue

[jira] [Created] (KAFKA-8581) Augment ProduceResponse error messaging for specific culprit records

2019-06-21 Thread Guozhang Wang (JIRA)
Guozhang Wang created KAFKA-8581: Summary: Augment ProduceResponse error messaging for specific culprit records Key: KAFKA-8581 URL: https://issues.apache.org/jira/browse/KAFKA-8581 Project: Kafka

[jira] [Resolved] (KAFKA-8106) Reducing the allocation and copying of ByteBuffer when logValidator do validation.

2019-06-21 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-8106?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-8106. -- Resolution: Fixed Fix Version/s: 2.4.0 > Reducing the allocation and copying of

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

2019-06-21 Thread Apache Jenkins Server
See Changes: [github] KAFKA-8519 Add trogdor action to slow down a network (#6912) -- [...truncated 2.52 MB...]

Re: [DISCUSS] KIP-470: TopologyTestDriver test input and output usability improvements

2019-06-21 Thread Matthias J. Sax
Thanks for the KIP. The idea to add InputTopic and OutputTopic abstractions is really neat! Couple of minor comment: (1) It's a little confusing that you list all method (existing, proposed to deprecate, and new one) of `TopologyTestDriver` in the KIP. Maybe only list the ones you propose to

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

2019-06-21 Thread Apache Jenkins Server
See Changes: [github] KAFKA-8519 Add trogdor action to slow down a network (#6912) -- [...truncated 2.51 MB...] org.apache.kafka.streams.scala.WordCountTest >

Re: [DISCUSS] KIP-466: Add support for List serialization and deserialization

2019-06-21 Thread Matthias J. Sax
Thanks for the update! I think that `ListDeserializer`, `ListSerializer`, and `ListSerde` should have an default constructor and it should be possible to pass in the `Class listClass` information via a configuration. Otherwise, KafkaStreams cannot use it as default serde. For the primitive

Re: [DISCUSS] KIP-466: Add support for List serialization and deserialization

2019-06-21 Thread Development
I made and pushed necessary commits, so we could review the final version under PR https://github.com/apache/kafka/pull/6592 I also need some advice on writing tests for this new serde. So far I only have two test cases (roundtrip and empty payload), I’m not sure if it is enough. Thank y’all

Re: [DISCUSS] KIP-470: TopologyTestDriver test input and output usability improvements

2019-06-21 Thread Bill Bejeck
Jukka, Thanks for the KIP. I like the changes overall. One thing I wanted to confirm, and this may be me being paranoid, but will the changes for input/output topic affect how the TopologyTestDriver works with internal topics when there are sub-topologies created? On Fri, Jun 21, 2019 at 12:05

Re: [DISCUSS] KIP-466: Add support for List serialization and deserialization

2019-06-21 Thread John Roesler
Hey Daniyar, Looks good to me! Thanks for considering it. Thanks, -John On Fri, Jun 21, 2019 at 9:04 AM Development wrote: > Hey John and Matthias, > > Yes, now I see it all. I’m storing lots of redundant information. > Here is my final idea. Yes, now a user should pass a list type. I

[jira] [Resolved] (KAFKA-8563) Minor: Remove method call in networkSend. Rely on java's vargs boxing/autoboxing

2019-06-21 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-8563?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma resolved KAFKA-8563. Resolution: Fixed Fix Version/s: 2.4.0 > Minor: Remove method call in networkSend. Rely on

[jira] [Resolved] (KAFKA-8570) Downconversion could fail when log contains out of order message formats

2019-06-21 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-8570?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-8570. Resolution: Fixed Fix Version/s: 2.2.2 2.1.2

Re: [DISCUSS] KIP-470: TopologyTestDriver test input and output usability improvements

2019-06-21 Thread Guozhang Wang
1) Got it, could you list this class along with all its functions in the proposed public APIs as well? 2) Ack, thanks! On Thu, Jun 20, 2019 at 11:27 PM Jukka Karvanen wrote: > Hi Guozhang, > > 1) This TestRecord is new class in my proposal. So it is a simplified > version of ProducerRecord

[jira] [Resolved] (KAFKA-6408) Kafka MirrorMaker doesn't replicate messages when .* regex is used

2019-06-21 Thread Waleed Fateem (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6408?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Waleed Fateem resolved KAFKA-6408. -- Resolution: Not A Problem > Kafka MirrorMaker doesn't replicate messages when .* regex is used

[jira] [Created] (KAFKA-8580) Compute RocksDB Metrics

2019-06-21 Thread Bruno Cadonna (JIRA)
Bruno Cadonna created KAFKA-8580: Summary: Compute RocksDB Metrics Key: KAFKA-8580 URL: https://issues.apache.org/jira/browse/KAFKA-8580 Project: Kafka Issue Type: Sub-task

[jira] [Created] (KAFKA-8579) Expose RocksDB Metrics to JMX

2019-06-21 Thread Bruno Cadonna (JIRA)
Bruno Cadonna created KAFKA-8579: Summary: Expose RocksDB Metrics to JMX Key: KAFKA-8579 URL: https://issues.apache.org/jira/browse/KAFKA-8579 Project: Kafka Issue Type: Sub-task

Re: [VOTE] 2.3.0 RC3

2019-06-21 Thread Ismael Juma
Hi Colin, One more thing: the Maven repo was not updated, it seems. Can you fix that? Ismael On Fri, Jun 21, 2019 at 8:23 AM Ismael Juma wrote: > Hi Colin, > > Thanks, +1 (binding). > > I verified the quickstart and signatures for the source and kafka 2.12 > artifacts with Java 11. One note

[jira] [Created] (KAFKA-8578) Add Sensors and Metrics to Expose RocksDB Metrics

2019-06-21 Thread Bruno Cadonna (JIRA)
Bruno Cadonna created KAFKA-8578: Summary: Add Sensors and Metrics to Expose RocksDB Metrics Key: KAFKA-8578 URL: https://issues.apache.org/jira/browse/KAFKA-8578 Project: Kafka Issue Type:

Re: [VOTE] 2.3.0 RC3

2019-06-21 Thread Ismael Juma
Hi Colin, Thanks, +1 (binding). I verified the quickstart and signatures for the source and kafka 2.12 artifacts with Java 11. One note is that we need to complete the upgrade section for 2.3.0 before we announce the release. No new RC is needed, we just have to have it on the website. Ismael

Re: [VOTE] KIP-471: Expose RocksDB Metrics in Kafka Streams

2019-06-21 Thread Bruno Cadonna
Hi, Since the voting passed the 72 hours and the number of binding votes are reached I will close this voting. KIP-471 passes with - 3 binding votes (Bill, Guozhang, Matthias) - 2 non-binding votes (John, Patrik) Thank you to all for the discussion and the voting. Best, Bruno On Thu, Jun 20,

Re: [VOTE] KIP-476: Add Java AdminClient interface

2019-06-21 Thread Ismael Juma
This is even more reason not to deprecate immediately, there is very little maintenance cost for us. We should be mindful that many of our users (eg Spark, Flink, etc.) typically allow users to specify the kafka clients version and hence avoid using new classes/interfaces for some time. They would

Re: [VOTE] KIP-476: Add Java AdminClient interface

2019-06-21 Thread Ismael Juma
Hi Andy, I didn't see any reason being mentioned why it's an anti pattern or cleaner so I assume it's subjective. :) For what it's worth, the approach of a collection interface providing a default implementation is very common in Scala and it makes a lot of sense in my mind. For example, why does

Re: [VOTE] 2.3.0 RC3

2019-06-21 Thread Kamal Chandraprakash
+1 (non-binding) * Ran unit and integration test on 2.11 and 2.12 * Verified quick start * Ran internal apps on the 3 node cluster On Thu, Jun 20, 2019 at 3:33 AM Colin McCabe wrote: > Hi all, > > We discovered some problems with the second release candidate (RC2) of > 2.3.0. Specifically,

Possible implementation for KAFKA-560

2019-06-21 Thread Carlos Manuel Duclos-Vergara
Hi, This is an ancient task, but I feel it is still current today (specially since as somebody that deals with a Kafka cluster I know that this happens more often than not). The task is about garbage collection of topics in a sort of automated way. After some consideration I started a prototype

Re: [VOTE] KIP-476: Add Java AdminClient interface

2019-06-21 Thread Andy Coates
Hi Ismael, I’m happy enough to not deprecate the existing `AdminClient` class as part of this change. However, note that, the class will likely be empty, i.e. all methods and implementations will be inherited from the interface: public abstract class AdminClient implements Admin { } Not

Re: [VOTE] KIP-476: Add Java AdminClient interface

2019-06-21 Thread Andy Coates
Hi Ismael, Matthias thought having the interface also provide a factory method that returns a specific implementation was a bit of an anti-pattern, and I would tend to agree, though I’ve used this same pattern myself at times where the set of implementations is known. Matthias may want to

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

2019-06-21 Thread Apache Jenkins Server
See Changes: [github] MINOR: Remove legacy kafka.admin.AdminClient (#6947) -- [...truncated 2.43 MB...] org.apache.kafka.streams.test.OutputVerifierTest >

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

2019-06-21 Thread Apache Jenkins Server
See Changes: [github] MINOR: Remove legacy kafka.admin.AdminClient (#6947) -- [...truncated 2.52 MB...] org.apache.kafka.streams.test.OutputVerifierTest >

[jira] [Created] (KAFKA-8577) Flaky Test `DistributedHerderTest.testJoinLeaderCatchUpFails`

2019-06-21 Thread Jason Gustafson (JIRA)
Jason Gustafson created KAFKA-8577: -- Summary: Flaky Test `DistributedHerderTest.testJoinLeaderCatchUpFails` Key: KAFKA-8577 URL: https://issues.apache.org/jira/browse/KAFKA-8577 Project: Kafka

Re: [DISCUSS] KIP-470: TopologyTestDriver test input and output usability improvements

2019-06-21 Thread Jukka Karvanen
Hi Guozhang, 1) This TestRecord is new class in my proposal. So it is a simplified version of ProducerRecord and ConsumerRecord containing only the fields needed to test record content. 2) public final TestInputTopic createInputTopic(final String topicName, final Serde keySerde, final Serde