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

2019-07-02 Thread Apache Jenkins Server
See -- [...truncated 2.85 MB...] org.apache.kafka.connect.transforms.CastTest > castWholeRecordValueSchemalessString PASSED org.apache.kafka.connect.transforms.CastTest >

Re: [DISCUSS] KIP-221: Repartition Topic Hints in Streams

2019-07-02 Thread Matthias J. Sax
Levani, Thanks for picking up this KIP! And thanks for summarizing everything. Even if some points may have been discussed already (can't really remember), it's helpful to get a good summary to refresh the discussion. I think your reasoning makes sense. With regard to the distinction between

[jira] [Created] (KAFKA-8624) 这边是服务端的kafka,版本是 kafka_2.11-1.0.0.2,当客户端向主题发送消息的时候抛出如下异常,客户端跟服务端的版本不一致吗?

2019-07-02 Thread CHARELS (JIRA)
CHARELS created KAFKA-8624: -- Summary: 这边是服务端的kafka,版本是 kafka_2.11-1.0.0.2,当客户端向主题发送消息的时候抛出如下异常,客户端跟服务端的版本不一致吗? Key: KAFKA-8624 URL: https://issues.apache.org/jira/browse/KAFKA-8624 Project: Kafka

Re: Synchronized consumption + processing based on timestamps?

2019-07-02 Thread Matthias J. Sax
I think you can only achieve this, if 1) you don't use two clients, but only one client that reads both partitions or 2) let both clients exchange data about their time progress -Matthias On 7/2/19 6:01 PM, Adam Bellemare wrote: > Hi All > > The use-case is pretty simple. Lets say we have

Re: [DISCUSS] KIP-213: Second follow-up on Foreign Key Joins

2019-07-02 Thread Matthias J. Sax
Thanks for the example. I was thinking about the problem a little bit, and I believe we should look at it in some more details. Basically, there are 3 cases: a) insert new record LHS b) delete record LHS c) update exiting record LHS For those cases we want different things to happen: a-1) sent

Synchronized consumption + processing based on timestamps?

2019-07-02 Thread Adam Bellemare
Hi All The use-case is pretty simple. Lets say we have a history of events with the following: key=userId, value = (timestamp, productId) and we want to remap it to (just as we would with an internal topic): key=productId, value=(original_timestamp, userId) Now, say I have 30 days of backlog,

Re: [DISCUSS] KIP-480 : Sticky Partitioner

2019-07-02 Thread Justine Olshan
Hello again, Another update to the interface has been made to the KIP. Please let me know if you have any feedback! Thank you, Justine On Fri, Jun 28, 2019 at 2:52 PM Justine Olshan wrote: > Hi all, > I made some changes to the KIP. > The idea is to clean up the code, make behavior more

Re: [DISCUSS] KIP-447: Producer scalability for exactly once semantics

2019-07-02 Thread Boyang Chen
Thank you Guozhang for the questions, inline answers are below. On Tue, Jul 2, 2019 at 3:14 PM Boyang Chen wrote: > Hey all, > > I have done a fundamental polish of KIP-447 > > and >

Re: [DISCUSS] KIP-447: Producer scalability for exactly once semantics

2019-07-02 Thread Boyang Chen
Hey all, I have done a fundamental polish of KIP-447 and written a design doc depicting internal

Re: [DISCUSS] KIP-484: Expose metrics for group and transaction metadata loading duration

2019-07-02 Thread Anastasia Vela
After further discussion with Anna, we decided the following: - add the average metric, but noted in the KIP that the average value may look low at times when there are many empty partitions that have a 0ms load time - set the window to 30sec, because there is no significance difference if we set

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

2019-07-02 Thread Apache Jenkins Server
See Changes: [cshapi] KAFKA-8560; The Kafka protocol generator should support common [rajinisivaram] MINOR: Fix version for ConnectDistributed system test, remove 0.9.0.1

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

2019-07-02 Thread Apache Jenkins Server
See Changes: [github] MINOR: Make the build compile with Scala 2.13 (#6989) -- [...truncated 6.17 MB...] org.apache.kafka.streams.integration.SuppressionIntegrationTest >

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

2019-07-02 Thread Apache Jenkins Server
See

Re: [DISCUSS] KIP-455 Create an Admin API for Replica Reassignments

2019-07-02 Thread Colin McCabe
On Tue, Jul 2, 2019, at 10:47, Stanislav Kozlovski wrote: > Hey there, I need to start a new thread on KIP-455. I think there might be > an issue with the mailing server. For some reason, my replies to the > previous discussion thread could not be seen by others. After numerous > attempts, Colin

[DISCUSS] KIP-455 Create an Admin API for Replica Reassignments

2019-07-02 Thread Stanislav Kozlovski
Hey there, I need to start a new thread on KIP-455. I think there might be an issue with the mailing server. For some reason, my replies to the previous discussion thread could not be seen by others. After numerous attempts, Colin suggested I start a new thread. Original Discussion Thread:

[jira] [Resolved] (KAFKA-8560) The Kafka protocol generator should support common structures

2019-07-02 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-8560?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira resolved KAFKA-8560. - Resolution: Fixed Fix Version/s: 2.4.0 > The Kafka protocol generator should support

Re: [DISCUSS] KIP-482: The Kafka Protocol should Support Optional Fields

2019-07-02 Thread Colin McCabe
On Tue, Jul 2, 2019, at 02:55, Tom Bentley wrote: > Hi Colin, > > In the example given, of a FooResponse, both the optional fields have the > tag 0x0001. The text says "This number must be unique within the context it > appears in.". My first thought was the example tags, must be wrong. But I >

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

2019-07-02 Thread Colin McCabe
On Tue, Jul 2, 2019, at 09:14, Colin McCabe wrote: > On Mon, Jul 1, 2019, at 23:30, Matthias J. Sax wrote: > > Not sure, if I understand the argument? > > > > Why would anyone need to support multiple client side versions? > > Clients/brokers are forward/backward compatible anyway. > > When

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

2019-07-02 Thread Colin McCabe
On Mon, Jul 1, 2019, at 23:30, Matthias J. Sax wrote: > Not sure, if I understand the argument? > > Why would anyone need to support multiple client side versions? > Clients/brokers are forward/backward compatible anyway. When you're using many different libraries, it is helpful if they don't

Scala 2.13 support

2019-07-02 Thread Ismael Juma
Hi all, Support for Scala 2.13 has been added via: https://github.com/apache/kafka/pull/5454 https://github.com/apache/kafka/pull/6989 I've updated the Jenkins PR and trunk builds to include Scala 2.13 (a separate job for PRs so that it can run in parallel with the other Scala versions, for

[jira] [Created] (KAFKA-8623) KafkaProducer possible deadlock when sending to different topics

2019-07-02 Thread Alexander Bagiev (JIRA)
Alexander Bagiev created KAFKA-8623: --- Summary: KafkaProducer possible deadlock when sending to different topics Key: KAFKA-8623 URL: https://issues.apache.org/jira/browse/KAFKA-8623 Project: Kafka

Fwd: Does anyone fixed Producer TimeoutException problem ?

2019-07-02 Thread Shyam P
Hi, I am facing the below issue. org.apache.kafka.common.errors.TimeoutException: Expiring 1 record(s) for 229 ms has passed since batch creation plus linger time I tried many producer configuration settings. more details below :

[jira] [Created] (KAFKA-8622) Snappy Compression Not Working

2019-07-02 Thread Kunal (JIRA)
Kunal created KAFKA-8622: Summary: Snappy Compression Not Working Key: KAFKA-8622 URL: https://issues.apache.org/jira/browse/KAFKA-8622 Project: Kafka Issue Type: Bug Components:

Re: [DISCUSS] KIP-482: The Kafka Protocol should Support Optional Fields

2019-07-02 Thread Tom Bentley
Hi Colin, In the example given, of a FooResponse, both the optional fields have the tag 0x0001. The text says "This number must be unique within the context it appears in.". My first thought was the example tags, must be wrong. But I think I misunderstood what you meant by "the context" – I

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

2019-07-02 Thread Jukka Karvanen
Hi Matthias, Generally I think using Instant and Duration make the test more readable and that's why I modified KIP based on your suggestion. Now a lot of code use time with long or Long and that make the change more complicated. What I tried to say about the migration is the lines without

Re: Running Kafka from source in Windows

2019-07-02 Thread Bruno Cadonna
Hi Manish, I have never installed Kafka on Windows, I can just point you to some relevant information. See for example the following e-mail on the user mailing list about running Kafka on Windows:

Re: Running Kafka from source in Windows

2019-07-02 Thread Manish G
Thanks Bruno. A question here: Running Kafka this way OR using WSL for it? Which one is a better choice? On Tue, Jul 2, 2019 at 1:19 PM Bruno Cadonna wrote: > Hi Manish, > > two bug reports have been already filed for this issue. > > https://issues.apache.org/jira/browse/KAFKA-6988 >

Re: Running Kafka from source in Windows

2019-07-02 Thread Bruno Cadonna
Hi Manish, two bug reports have been already filed for this issue. https://issues.apache.org/jira/browse/KAFKA-6988 https://issues.apache.org/jira/browse/KAFKA-7998 You can find a description of the root cause and work-arounds in the bug reports and the following GitHub issue:

[jira] [Created] (KAFKA-8621) KIP-486: Support for pluggable KeyStore and TrustStore

2019-07-02 Thread MAULIN VASAVADA (JIRA)
MAULIN VASAVADA created KAFKA-8621: -- Summary: KIP-486: Support for pluggable KeyStore and TrustStore Key: KAFKA-8621 URL: https://issues.apache.org/jira/browse/KAFKA-8621 Project: Kafka

[DISCUSS] KIP-486 Support for pluggable KeyStore and TrustStore

2019-07-02 Thread maulin . vasavada
Hi all, Starting this thread to discuss about KIP 486 - https://cwiki.apache.org/confluence/display/KAFKA/KIP-486%3A+Support+for+pluggable+KeyStore+and+TrustStore Thanks Maulin

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

2019-07-02 Thread Matthias J. Sax
Not sure, if I understand the argument? Why would anyone need to support multiple client side versions? Clients/brokers are forward/backward compatible anyway. Also, if one really supports multiple client side versions, won't they use multiple shaded dependencies for different versions? Last,