Re: [DISCUSS] KIP-607: Add Metrics to Record the Memory Used by RocksDB to Kafka Streams

2020-07-22 Thread Bruno Cadonna
Hi Guozhang, Thank you for your feedback! I answered inline. Best, Bruno On 21.07.20 00:39, Guozhang Wang wrote: Hello Bruno, Thanks for the updated KIP. I made a pass and here are some comments: 1) What's the motivation of keeping it as INFO while KIP-471 metrics are defined in DEBUG?

Fwd: Re: [DISCUSS] KIP-607: Add Metrics to Record the Memory Used by RocksDB to Kafka Streams

2020-07-22 Thread Bruno Cadonna
Hi Guozhang, Thank you for your feedback! I answered inline. Best, Bruno On 21.07.20 00:39, Guozhang Wang wrote: Hello Bruno, Thanks for the updated KIP. I made a pass and here are some comments: 1) What's the motivation of keeping it as INFO while KIP-471 metrics are defined in DEBUG?

Re: [DISCUSS] KIP-607: Add Metrics to Record the Memory Used by RocksDB to Kafka Streams

2020-07-22 Thread Bruno Cadonna
Thank you John for the proposal. Indeed I did also not like to extend the RocksDBConfigSetter interface, but couldn't find a way around it. I will remove the interface extension from the KIP and try out your proposal. I need to look into the details but after a first glance, I think we need

[jira] [Created] (KAFKA-10299) Add a Hash SMT transformer

2020-07-22 Thread Brandon Brown (Jira)
Brandon Brown created KAFKA-10299: - Summary: Add a Hash SMT transformer Key: KAFKA-10299 URL: https://issues.apache.org/jira/browse/KAFKA-10299 Project: Kafka Issue Type: New Feature

[jira] [Reopened] (KAFKA-8098) Flaky Test AdminClientIntegrationTest#testConsumerGroups

2020-07-22 Thread Luke Chen (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-8098?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Luke Chen reopened KAFKA-8098: -- > Flaky Test AdminClientIntegrationTest#testConsumerGroups >

Jenkins build is back to normal : kafka-trunk-jdk8 #4734

2020-07-22 Thread Apache Jenkins Server
See

Re: [DISCUSS] KIP-450: Sliding Windows

2020-07-22 Thread Sophie Blee-Goldman
Hey John, Just a few follow-up questions/comments about the whole Windows thing: That's a good way of looking at things; in particular the point about SessionWindows for example requiring a Merger while other "statically enumerable" windows require only an adder seems to touch on the heart of

Re: [DISCUSS] KIP-450: Sliding Windows

2020-07-22 Thread John Roesler
Thanks Leah! 5) Regarding the empty windows, I'm wondering if we should simply propose that the windows should not be emitted downstream of the operator or visible in IQ. Then, it'll be up to the implementation to make it happen. I'm personally not concerned about it, since it seems like there

Jenkins build is back to normal : kafka-trunk-jdk14 #309

2020-07-22 Thread Apache Jenkins Server
See

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

2020-07-22 Thread Apache Jenkins Server
See Changes: [github] MINOR: Fix deprecation version for NotLeaderForPartitionException [github] MINOR: Fix SslEngineFactory javadoc (#9055) -- [...truncated 6.36

[jira] [Created] (KAFKA-10300) fix flaky core/group_mode_transactions_test.py

2020-07-22 Thread Chia-Ping Tsai (Jira)
Chia-Ping Tsai created KAFKA-10300: -- Summary: fix flaky core/group_mode_transactions_test.py Key: KAFKA-10300 URL: https://issues.apache.org/jira/browse/KAFKA-10300 Project: Kafka Issue

Re: [DISCUSS] KIP-450: Sliding Windows

2020-07-22 Thread John Roesler
Thanks, all, I can see how my conclusion was kind of a leap. What Matthias said is indeed what I was thinking. When you provide a window definition to the windowBy() method, you are selecting an algorithm that will be used to compute the windows from the input data. I didn’t mean the code

Re: [DISCUSS] KIP-617: Allow Kafka Streams State Stores to be iterated backwards

2020-07-22 Thread Matthias J. Sax
Finally cycling back to this. Overall I like the KIP. Two comments: - I tried to figure out why the two InMemoerySessionStore methods are deprecated and it seems those annotations are there since the class was added; as this seems to be a bug, and there are no backward compatibility concerns,

Re: [DISCUSS] KIP-450: Sliding Windows

2020-07-22 Thread Matthias J. Sax
I think what John tries to say is the following: We have `windowedBy(Windows)` that accept hopping/tumbling windows but also custom window and we use a specific algorithm. Note, that custom windows must "work" based on the used algorithm. For session windows we have `windowedBy(SessionWindows)`

Re: [DISCUSS] KIP-450: Sliding Windows

2020-07-22 Thread Sophie Blee-Goldman
> > Users could pass in a custom `SessionWindows` as > long as the session algorithm works correctly for it. Well not really, SessionWindows is a final class. TimeWindows is also a final class, so neither of these can be extended/customized. For a given Windows then there would only be three

Re: [DISCUSS] KIP-646 Serializer API should support ByteBuffer

2020-07-22 Thread Chia-Ping Tsai
Thanks for quick feedback! Ismael > Are there options with lower impact that still help us achieve the goal for > those who need it? > For example, it could be an opt-in thing instead of forcing the world to > change. It seems to me there are two alternatives. 1. Introduce an new extended

Re: [DISCUSS] KIP-450: Sliding Windows

2020-07-22 Thread John Roesler
Thanks for the reply, Sophie. That all sounds about right to me. The Windows “interface”/algorithm is quite flexible, so it makes sense for it to be extensible. Different implementations can (and do) enumerate different windows to suit different use cases. On the other hand, I can’t think

Kafka - Controller Broker

2020-07-22 Thread Nag Y
come across this phrase from https://niqdev.github.io/devops/kafka/ and https://livebook.manning.com/book/kafka-streams-in-action/chapter-2/109 (Kafka Streams in Action ) The controller broker is responsible for setting up leader/follower relationships for all partitions of a topic. If a Kafka

Re: [DISCUSS] KIP-450: Sliding Windows

2020-07-22 Thread Leah Thomas
Hi Matthias, Thanks for the suggestions, I've updated the KIP and child page accordingly and addressed some below. 1) For the mandatory grace period, we should use a static builder method > that take two parameters. > That makes sense, I've changed that in the public API. Btw: this

[DISCUSS] KIP-646 Serializer API should support ByteBuffer

2020-07-22 Thread Chia-Ping Tsai
hi folks, I would like to discuss KIP-646. The KIP plans to use ByteBuffer to be the return type of Serializer#serialize. It opens the door to manage the memory more effectively and flexible. https://cwiki.apache.org/confluence/x/RiR4CQ The change involved by this KIP is huge so it would be

Re: [VOTE] 2.6.0 RC1

2020-07-22 Thread Randall Hauch
Any thoughts, Rajini? On Mon, Jul 20, 2020 at 9:55 PM Randall Hauch wrote: > > When I was checking the documentation for RC1 after the tag was pushed, I > noticed that the fix Rajini mentioned in the RC0 vote thread ( > https://github.com/apache/kafka/pull/8979 >

Confluent Kafka - Schema Registry on windows

2020-07-22 Thread Nag Y
I happened to see an example how to run schema registry using "schema-registry-start.bat" from windows on 5.0.1 I didnt see the file in 5.5.0 . Is the schema registry not supported in windows now ? IT seems only the way to go about running schema registry in windows through dockers . Please

Confluent Platform- KTable clarification

2020-07-22 Thread Nag Y
I understood A KStream is an abstraction of a record stream and A KTable is an abstraction of a changelog stream ( updates or inserts) and the semantics around it. However, this is where some confusion arises .. From confluent documentation

Confluent Docker Images

2020-07-22 Thread Nag Y
The Docker images are huge for each confluent component - ZK, registry etc .. Is there any other place I can download one image that contains all the components ? REPOSITORY TAG IMAGE ID CREATED SIZE

Re: [DISCUSS] KIP-646 Serializer API should support ByteBuffer

2020-07-22 Thread Ismael Juma
Hi Chia-Ping, Thanks for the KIP. It seems like the path chosen here would cause a massive impact to user code. Are there options with lower impact that still help us achieve the goal for those who need it? For example, it could be an opt-in thing instead of forcing the world to change. Ismael

[jira] [Resolved] (KAFKA-10163) Implement Broker side changes

2020-07-22 Thread Rajini Sivaram (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10163?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram resolved KAFKA-10163. Fix Version/s: 2.7.0 Reviewer: Rajini Sivaram Assignee: David Jacot

Build failed in Jenkins: kafka-trunk-jdk14 #308

2020-07-22 Thread Apache Jenkins Server
See Changes: [github] KAFKA-10163; Throttle Create Topic, Create Partition and Delete Topic -- [...truncated 3.20 MB...] org.apache.kafka.streams.TopologyTestDriverTest

[jira] [Resolved] (KAFKA-10164) Implement Admin side changes

2020-07-22 Thread Rajini Sivaram (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10164?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram resolved KAFKA-10164. Fix Version/s: 2.7.0 Reviewer: Rajini Sivaram Assignee: David Jacot

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

2020-07-22 Thread Apache Jenkins Server
See Changes: [github] KAFKA-10163; Throttle Create Topic, Create Partition and Delete Topic -- [...truncated 3.18 MB...]

[jira] [Resolved] (KAFKA-10274) Transaction system test uses inconsistent timeouts

2020-07-22 Thread Jun Rao (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10274?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao resolved KAFKA-10274. - Fix Version/s: 2.6.0 Resolution: Fixed Merged this to trunk and 2.6 branch. > Transaction

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

2020-07-22 Thread Apache Jenkins Server
See

Jenkins build is back to normal : kafka-2.6-jdk8 #93

2020-07-22 Thread Apache Jenkins Server
See