Re: [VOTE] KIP-659: Improve TimeWindowedDeserializer and TimeWindowedSerde to handle window size

2020-09-29 Thread Sophie Blee-Goldman
There are two cases where you need to specify the window size -- directly using a Consumer (eg the console consumer) or reading as an input topic within Streams. We need a config for the first case, since you can't pass a Deserializer object to the console consumer. In the Streams case, the

Re: [VOTE] KIP-478 Strongly Typed Streams Processor API

2020-09-29 Thread Paul Whalen
John, I totally agree that adding a method to Processor is cumbersome and not a good path. I was imagining maybe a separate interface that could be used in the appropriate context, but I don't think that makes too much sense - it's just too far away from what Kafka Streams is. I was originally

Re: [DISCUSS] KIP-598: Augment TopologyDescription with store and source / sink serde information

2020-09-29 Thread Sophie Blee-Goldman
Hey Guozhang, what's the status of this KIP? I was recently digging through a particularly opaque Streams application and it occurred to me that it might also be useful to print the kind of store attached to each node (eg RocksDBWindowStore, InMemoryKeyValueStore, custom, etc). That made me think

Re: [VOTE] KIP-478 Strongly Typed Streams Processor API

2020-09-29 Thread Sophie Blee-Goldman
FWIW, while I'm really not a fan of Optional in general, I agree that its usage here seems appropriate. Even for those rare software developers who carefully read all the docs several times over, I think it wouldn't be too hard to miss a note about the RecordMetadata possibly being null.

Re: [VOTE] KIP-478 Strongly Typed Streams Processor API

2020-09-29 Thread Sophie Blee-Goldman
> > Does my reply address your concerns? Yes; also, I definitely misread part of the proposal earlier and thought you had put the timestamp field in RecordMetadata. Sorry for not giving things a closer look before responding! I'm not sure my original message made much sense given the

[DISCUSS] Release Deadlines

2020-09-29 Thread Matthias J. Sax
Hi, when we introduced time based releases, we added certain deadlines to streamline the release process and to make sure we can ship the release on time. Based on early experience, we adjusted those deadlines and introduced new deadlines which improved the situation. However, we still have the

Re: [VOTE] KIP-478 Strongly Typed Streams Processor API

2020-09-29 Thread John Roesler
Thanks for the reply, Sophie, I think I may have summarized too much in my prior reply. In the currently proposed KIP, any caller of forward() must supply a Record, which consists of: * key * value * timestamp * headers (with a convenience constructor that sets empty headers) These aren't what

Re: [VOTE] KIP-478 Strongly Typed Streams Processor API

2020-09-29 Thread Sophie Blee-Goldman
> > However, the record metadata is only defined when the parent forwards > while processing a real record, not when it calls forward from the punctuator Can we take a step back for a second...why wouldn't you be required to set the RecordContext yourself when calling forward from a Punctuator?

Re: [VOTE] KIP-671: Add method to Shutdown entire Streams Application

2020-09-29 Thread Matthias J. Sax
Thanks Walker. The proposed API changes LGTM. +1 (binding) One minor nit: you should also mention the global-thread that also needs to be shutdown if requested by the user. Minor side question: should we actually terminate a thread and create a new one, or instead revive the existing thread

Re: [VOTE] KIP-478 Strongly Typed Streams Processor API

2020-09-29 Thread John Roesler
Oh, I guess one other thing I should have mentioned is that I’ve recently discovered that in cases where the context is undefined, we currently just fill in dummy values for the context. So there’s a good chance that real applications in use are depending on undefined context without even

Re: [VOTE] KIP-478 Strongly Typed Streams Processor API

2020-09-29 Thread John Roesler
Thanks for the review, Paul! I had read some of that debate before. There seems to be some subtext there, because they advise against using Optional in cases like this, but there doesn’t seem to be a specific reason why it’s inappropriate. I got the impression they were just afraid that people

Re: [DISCUSS] KIP-631: The Quorum-based Kafka Controller

2020-09-29 Thread Jason Gustafson
Hey Colin, Thanks for the hard work on this proposal. I'm gradually coming over to the idea of the controllers having separate IDs. One of the benefits is that it allows us to separate the notion of controller liveness from broker liveness, which has always been a tricky detail. I think it's

Re: [DISCUSS] KIP-631: The Quorum-based Kafka Controller

2020-09-29 Thread Jose Garcia Sancio
Hi Jun and Colin, Some comments below. > 62.3 We added some configs in KIP-595 prefixed with "quorum" and we plan to > add some controller specific configs prefixed with "controller". KIP-630 > plans to add some other controller specific configs with no prefix. Should > we standardize all

Re: [VOTE] KIP-478 Strongly Typed Streams Processor API

2020-09-29 Thread Paul Whalen
Looks pretty good to me, though the Processor#process(Record, Optional) signature caught my eye. There's some debate ( https://stackoverflow.com/questions/31922866/why-should-java-8s-optional-not-be-used-in-arguments) about whether to use Optionals in arguments, and while that's a bit of a

Jenkins build is back to normal : Kafka » kafka-trunk-jdk15 #132

2020-09-29 Thread Apache Jenkins Server
See

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

2020-09-29 Thread Apache Jenkins Server
See Changes: [github] KAFKA-10479; Throw exception if users try to update non-reconfigurable configs of existing listeners (#9284) -- [...truncated 6.65

Re: [DISCUSS] KIP-653: Upgrade log4j to log4j2

2020-09-29 Thread Dongjin Lee
Hi All, As you can see in the PR, I eliminated all compatibility breaks caused by the root logger name change between log4j and log4j2. (i.e., "root" → "") Plus, I rebased the PR onto the latest trunk, with migrating raft module into log4j2. Please have a look. And please note that now we have

Re: [VOTE] KIP-671: Add method to Shutdown entire Streams Application

2020-09-29 Thread Bill Bejeck
Thanks for the KIP Walker. +1 (binding) -Bill On Tue, Sep 29, 2020 at 4:59 PM Guozhang Wang wrote: > +1 again on the KIP. > > On Tue, Sep 29, 2020 at 1:51 PM Leah Thomas wrote: > > > Hey Walker, > > > > Thanks for the KIP! I'm +1, non-binding. > > > > Cheers, > > Leah > > > > On Tue, Sep 29,

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

2020-09-29 Thread Apache Jenkins Server
See Changes: [github] KAFKA-10479; Throw exception if users try to update non-reconfigurable configs of existing listeners (#9284) -- [...truncated 3.35

[jira] [Resolved] (KAFKA-9061) StreamStreamJoinIntegrationTest flaky test failures

2020-09-29 Thread Guozhang Wang (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9061?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-9061. -- Resolution: Cannot Reproduce > StreamStreamJoinIntegrationTest flaky test failures >

Re: [VOTE] KIP-478 Strongly Typed Streams Processor API

2020-09-29 Thread John Roesler
Hello again, all, Thanks for the latest round of discussion. I've taken the recent feedback and come up with an updated KIP that seems actually quite a bit nicer than the prior proposal. The specific diff on the KIP is here:

Re: [VOTE] KIP-671: Add method to Shutdown entire Streams Application

2020-09-29 Thread Guozhang Wang
+1 again on the KIP. On Tue, Sep 29, 2020 at 1:51 PM Leah Thomas wrote: > Hey Walker, > > Thanks for the KIP! I'm +1, non-binding. > > Cheers, > Leah > > On Tue, Sep 29, 2020 at 1:56 PM Walker Carlson > wrote: > > > Hello all, > > > > I made some changes to the KIP the descriptions are on the

Re: [VOTE] KIP-671: Add method to Shutdown entire Streams Application

2020-09-29 Thread Leah Thomas
Hey Walker, Thanks for the KIP! I'm +1, non-binding. Cheers, Leah On Tue, Sep 29, 2020 at 1:56 PM Walker Carlson wrote: > Hello all, > > I made some changes to the KIP the descriptions are on the discussion > thread. If you have already voted I would ask you to confirm your vote. > >

[jira] [Created] (KAFKA-10553) Track handling of topic deletion during reassignment

2020-09-29 Thread Justine Olshan (Jira)
Justine Olshan created KAFKA-10553: -- Summary: Track handling of topic deletion during reassignment Key: KAFKA-10553 URL: https://issues.apache.org/jira/browse/KAFKA-10553 Project: Kafka

Re: [VOTE] KIP-663: API to Start and Shut Down Stream Threads and to Request Closing of Kafka Streams Clients

2020-09-29 Thread Matthias J. Sax
+1 (binding) I am not super happy with the impact on the client state. For example, I don't understand why it's ok to scale out if we lose one thread out of four, but why it's not ok to scale out if we lose one thread out of one (for this case, we would enter ERROR state and cannot add new

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

2020-09-29 Thread Apache Jenkins Server
See Changes: [github] MINOR; Preserve ThrottlingQuotaExceededException when request timeouts after being retried due to a quota violation (KIP-599) (#9344)

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

2020-09-29 Thread Apache Jenkins Server
See Changes: [github] MINOR; Preserve ThrottlingQuotaExceededException when request timeouts after being retried due to a quota violation (KIP-599) (#9344)

[jira] [Created] (KAFKA-10552) Update directory structure to use topic IDs

2020-09-29 Thread Justine Olshan (Jira)
Justine Olshan created KAFKA-10552: -- Summary: Update directory structure to use topic IDs Key: KAFKA-10552 URL: https://issues.apache.org/jira/browse/KAFKA-10552 Project: Kafka Issue Type:

[jira] [Created] (KAFKA-10551) Support topic IDs in Produce request

2020-09-29 Thread Justine Olshan (Jira)
Justine Olshan created KAFKA-10551: -- Summary: Support topic IDs in Produce request Key: KAFKA-10551 URL: https://issues.apache.org/jira/browse/KAFKA-10551 Project: Kafka Issue Type:

[jira] [Resolved] (KAFKA-10479) Throw exception if users try to update configs of existent listeners

2020-09-29 Thread Jason Gustafson (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10479?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-10479. - Fix Version/s: 2.7.0 Resolution: Fixed > Throw exception if users try to update

Re: [VOTE] KIP-671: Add method to Shutdown entire Streams Application

2020-09-29 Thread Walker Carlson
Hello all, I made some changes to the KIP the descriptions are on the discussion thread. If you have already voted I would ask you to confirm your vote. Otherwise please vote so we can get this feature in. Thanks, Walker On Thu, Sep 24, 2020 at 4:36 PM John Roesler wrote: > Thanks for the

[jira] [Created] (KAFKA-10550) Update kafka-topics.sh to support Topic IDs

2020-09-29 Thread Justine Olshan (Jira)
Justine Olshan created KAFKA-10550: -- Summary: Update kafka-topics.sh to support Topic IDs Key: KAFKA-10550 URL: https://issues.apache.org/jira/browse/KAFKA-10550 Project: Kafka Issue Type:

Re: [DISCUSS] KIP-673: Emit JSONs with new auto-generated schema

2020-09-29 Thread Anastasia Vela
Hi Tom, I'm glad it makes more sense. I modified the KIP just to make it a little more clear as well. Is the concern that deserialization would have no guarantee as to the order it appears? Since ObjectNodes hold the tree structure in a LinkedHashMap, the iterating order to deserialize would be

[jira] [Created] (KAFKA-10549) Add topic ID support to DeleteTopics,ListOffsets, OffsetForLeaders, StopReplica

2020-09-29 Thread Justine Olshan (Jira)
Justine Olshan created KAFKA-10549: -- Summary: Add topic ID support to DeleteTopics,ListOffsets, OffsetForLeaders, StopReplica Key: KAFKA-10549 URL: https://issues.apache.org/jira/browse/KAFKA-10549

[jira] [Created] (KAFKA-10548) Implement Type field and logic for LeaderAndIsrRequests

2020-09-29 Thread Justine Olshan (Jira)
Justine Olshan created KAFKA-10548: -- Summary: Implement Type field and logic for LeaderAndIsrRequests Key: KAFKA-10548 URL: https://issues.apache.org/jira/browse/KAFKA-10548 Project: Kafka

[jira] [Created] (KAFKA-10547) Add topic IDs to MetadataResponse, UpdateMetadata, and Fetch

2020-09-29 Thread Justine Olshan (Jira)
Justine Olshan created KAFKA-10547: -- Summary: Add topic IDs to MetadataResponse, UpdateMetadata, and Fetch Key: KAFKA-10547 URL: https://issues.apache.org/jira/browse/KAFKA-10547 Project: Kafka

[jira] [Created] (KAFKA-10545) Create Topic IDs and Propagate to Brokers

2020-09-29 Thread Justine Olshan (Jira)
Justine Olshan created KAFKA-10545: -- Summary: Create Topic IDs and Propagate to Brokers Key: KAFKA-10545 URL: https://issues.apache.org/jira/browse/KAFKA-10545 Project: Kafka Issue Type:

[jira] [Created] (KAFKA-10546) KIP-478: Deprecate old PAPI

2020-09-29 Thread John Roesler (Jira)
John Roesler created KAFKA-10546: Summary: KIP-478: Deprecate old PAPI Key: KAFKA-10546 URL: https://issues.apache.org/jira/browse/KAFKA-10546 Project: Kafka Issue Type: Sub-task

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

2020-09-29 Thread Jun Rao
Hi, Kowshik, Thanks for the update. Regarding enabling a single rolling restart in the future, could we sketch out a bit how this will work by treating IBP as a feature? For example, IBP currently uses the release version and this KIP uses an integer for versions. How do we bridge the gap between

[jira] [Created] (KAFKA-10544) Convert KTable aggregations to new PAPI

2020-09-29 Thread John Roesler (Jira)
John Roesler created KAFKA-10544: Summary: Convert KTable aggregations to new PAPI Key: KAFKA-10544 URL: https://issues.apache.org/jira/browse/KAFKA-10544 Project: Kafka Issue Type: Sub-task

[jira] [Created] (KAFKA-10541) Convert KTable filters to new PAPI

2020-09-29 Thread John Roesler (Jira)
John Roesler created KAFKA-10541: Summary: Convert KTable filters to new PAPI Key: KAFKA-10541 URL: https://issues.apache.org/jira/browse/KAFKA-10541 Project: Kafka Issue Type: Sub-task

[jira] [Created] (KAFKA-10542) Convert KTable maps to new PAPI

2020-09-29 Thread John Roesler (Jira)
John Roesler created KAFKA-10542: Summary: Convert KTable maps to new PAPI Key: KAFKA-10542 URL: https://issues.apache.org/jira/browse/KAFKA-10542 Project: Kafka Issue Type: Sub-task

[jira] [Created] (KAFKA-10543) Convert KTable joins to new PAPI

2020-09-29 Thread John Roesler (Jira)
John Roesler created KAFKA-10543: Summary: Convert KTable joins to new PAPI Key: KAFKA-10543 URL: https://issues.apache.org/jira/browse/KAFKA-10543 Project: Kafka Issue Type: Sub-task

[jira] [Created] (KAFKA-10540) Convert KStream aggregations to new PAPI

2020-09-29 Thread John Roesler (Jira)
John Roesler created KAFKA-10540: Summary: Convert KStream aggregations to new PAPI Key: KAFKA-10540 URL: https://issues.apache.org/jira/browse/KAFKA-10540 Project: Kafka Issue Type:

[jira] [Created] (KAFKA-10539) Convert KStreamImpl joins to new PAPI

2020-09-29 Thread John Roesler (Jira)
John Roesler created KAFKA-10539: Summary: Convert KStreamImpl joins to new PAPI Key: KAFKA-10539 URL: https://issues.apache.org/jira/browse/KAFKA-10539 Project: Kafka Issue Type: Sub-task

[jira] [Created] (KAFKA-10538) Convert KStreamImpl maps to new PAPI

2020-09-29 Thread John Roesler (Jira)
John Roesler created KAFKA-10538: Summary: Convert KStreamImpl maps to new PAPI Key: KAFKA-10538 URL: https://issues.apache.org/jira/browse/KAFKA-10538 Project: Kafka Issue Type: Sub-task

[jira] [Created] (KAFKA-10537) Convert KStreamImpl filters to new PAPI

2020-09-29 Thread John Roesler (Jira)
John Roesler created KAFKA-10537: Summary: Convert KStreamImpl filters to new PAPI Key: KAFKA-10537 URL: https://issues.apache.org/jira/browse/KAFKA-10537 Project: Kafka Issue Type: Sub-task

[jira] [Created] (KAFKA-10536) KIP-478: Implement KStream changes

2020-09-29 Thread John Roesler (Jira)
John Roesler created KAFKA-10536: Summary: KIP-478: Implement KStream changes Key: KAFKA-10536 URL: https://issues.apache.org/jira/browse/KAFKA-10536 Project: Kafka Issue Type: Sub-task

[jira] [Created] (KAFKA-10535) KIP-478: Implement StateStoreContext and Record

2020-09-29 Thread John Roesler (Jira)
John Roesler created KAFKA-10535: Summary: KIP-478: Implement StateStoreContext and Record Key: KAFKA-10535 URL: https://issues.apache.org/jira/browse/KAFKA-10535 Project: Kafka Issue Type:

[jira] [Created] (KAFKA-10534) Modify the originals parameter type of the AbstractConfig class to avoid redundant judgments in the code

2020-09-29 Thread linenwei (Jira)
linenwei created KAFKA-10534: Summary: Modify the originals parameter type of the AbstractConfig class to avoid redundant judgments in the code Key: KAFKA-10534 URL: https://issues.apache.org/jira/browse/KAFKA-10534

[jira] [Created] (KAFKA-10533) Add log flush semantics to simulation test

2020-09-29 Thread Jason Gustafson (Jira)
Jason Gustafson created KAFKA-10533: --- Summary: Add log flush semantics to simulation test Key: KAFKA-10533 URL: https://issues.apache.org/jira/browse/KAFKA-10533 Project: Kafka Issue Type:

Re: [DISCUSS] KIP-671: Shutdown Streams Application when appropriate exception is thrown

2020-09-29 Thread Guozhang Wang
Thanks for the updates Walker. They all lgtm. On Tue, Sep 29, 2020 at 8:33 AM Walker Carlson wrote: > Thank you for the feedback Guozhang and Bruno. See the responses below. > > I have updated the kip accordingly > > Thanks, > Walker > > On Tue, Sep 29, 2020 at 1:59 AM Bruno Cadonna wrote: > >

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

2020-09-29 Thread Apache Jenkins Server
See Changes: [github] KAFKA-8360: Docs do not mention RequestQueueSize JMX metric (#9325) -- [...truncated 6.70 MB...]

[jira] [Resolved] (KAFKA-9546) Make FileStreamSourceTask extendable with generic streams

2020-09-29 Thread Randall Hauch (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9546?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Randall Hauch resolved KAFKA-9546. -- Resolution: Won't Fix I'm going to close this as WONTFIX, per my previous comment. > Make

Re: [DISCUSS] KIP-671: Shutdown Streams Application when appropriate exception is thrown

2020-09-29 Thread Walker Carlson
Thank you for the feedback Guozhang and Bruno. See the responses below. I have updated the kip accordingly Thanks, Walker On Tue, Sep 29, 2020 at 1:59 AM Bruno Cadonna wrote: > Hi Walker, > > Thanks for updating the KIP! > > 1. I would add response REPLACE_STREAM_THREAD to the >

Jenkins build is back to normal : Kafka » kafka-trunk-jdk11 #98

2020-09-29 Thread Apache Jenkins Server
See

[jira] [Resolved] (KAFKA-9514) The protocol generator generated useless condition when a field is made nullable and flexible version is used

2020-09-29 Thread David Jacot (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9514?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] David Jacot resolved KAFKA-9514. Fix Version/s: 2.7.0 Resolution: Fixed > The protocol generator generated useless condition

Re: [VOTE] KIP-663: API to Start and Shut Down Stream Threads and to Request Closing of Kafka Streams Clients

2020-09-29 Thread John Roesler
Thanks, Bruno, this sounds good to me. -John On Tue, Sep 29, 2020, at 03:13, Bruno Cadonna wrote: > Hi all, > > I did two minor modifications to the KIP. > > - I removed the rather strict guarantee "Dead stream threads are removed > from a Kafka Streams client at latest after the next call to

Re: [DISCUSS] KIP-671: Shutdown Streams Application when appropriate exception is thrown

2020-09-29 Thread Bruno Cadonna
Hi Walker, Thanks for updating the KIP! 1. I would add response REPLACE_STREAM_THREAD to the StreamsUncaughtExceptionHandlerResponse enum to start a new stream thread that replaces the failed one. I suspect you did not add it because it depends on KIP-663. A dependency to another unfinished

Re: [DISCUSS] KIP-671: Shutdown Streams Application when appropriate exception is thrown

2020-09-29 Thread Bruno Cadonna
Hi John, I totally agree with you and Walker. I also think that we should leave this as a problem for the future and that we should document this limitation. Best, Bruno On 24.09.20 16:51, John Roesler wrote: Hello all, Thanks for bringing this up, Bruno. It’s a really good point that a

Re: [VOTE] KIP-663: API to Start and Shut Down Stream Threads and to Request Closing of Kafka Streams Clients

2020-09-29 Thread Bruno Cadonna
Hi all, I did two minor modifications to the KIP. - I removed the rather strict guarantee "Dead stream threads are removed from a Kafka Streams client at latest after the next call to KafkaStreams#addStreamThread() or KafkaStreams#removeStreamThread() following the transition to state DEAD."

Build failed in Jenkins: Kafka » kafka-2.4-jdk8 #7

2020-09-29 Thread Apache Jenkins Server
See Changes: [Randall Hauch] KAFKA-10218: Stop reading config topic in every subsequent tick if catchup fails once (#8973) -- [...truncated 2.90 MB...]