Re: [VOTE] KIP-599: Throttle Create Topic, Create Partition and Delete Topic Operations

2020-05-30 Thread Gwen Shapira
+1 (binding)

Looks great. Thank you for the in-depth design and discussion.

On Fri, May 29, 2020 at 7:58 AM David Jacot  wrote:

> Hi folks,
>
> I'd like to start the vote for KIP-599 which proposes a new quota to
> throttle create topic, create partition, and delete topics operations to
> protect the Kafka controller:
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-599%3A+Throttle+Create+Topic%2C+Create+Partition+and+Delete+Topic+Operations
>
> Please, let me know what you think.
>
> Cheers,
> David
>


-- 
Gwen Shapira
Engineering Manager | Confluent
650.450.2760 | @gwenshap
Follow us: Twitter | blog


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

2020-05-30 Thread Apache Jenkins Server
See 




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

2020-05-30 Thread Apache Jenkins Server
See 




Re: [DISCUSS] KIP-614: Add Prefix Scan support for State Stores

2020-05-30 Thread John Roesler
Hi Sagar,

Thanks for the response. Your use case makes sense to me; I figured it must be 
something like that. 

On a pragmatic level, in the near term, you might consider basically doing the 
same thing we did in KIP-213. If you swap out the store types for Byte/byte[] 
and “manually” invoke the serdes in your own logic, then you can use the same 
algorithm we did to derive the range scan boundaries from your desired prefix.

For the actual KIP, it seems like we would need significant design improvements 
to be able to do any mixins, so I think we should favor proposing either to 
just add to the existing interfaces or to create brand new interfaces, as 
appropriate, for now. Given that prefix can be converted to a range query at a 
low level, I think we can probably explore adding prefix to the existing 
interfaces with a default implementation. 

It seems like that just leaves the question of how to define the type of the 
prefix. To be honest, I don’t have any great ideas here. Are you able to 
generate some creative solutions, Sagar?

Thanks,
John

On Tue, May 26, 2020, at 06:42, Sagar wrote:
> Hi John,
> 
> Thanks for the detailed reply. I was a bit crammed with work last week so
> couldn't respond earlier so apologies for that.
> 
> First of all, thanks for the context that both you and Adam have
> provided me on the issues faced previously. As I can clearly see, while I
> was able to cut some corners while writing some test cases or benchmarks,
> to be able to stitch together a store with prefix scan into an actual
> topology needs more work. I am sorry for the half baked tests that I wrote
> without realising and you have rightly put it when you said these
> challenges aren't obvious up front.
> 
> Now, coming back to the other points, I spent some time going through the
> KIP-213 and also some of the code snippets that are talked about in that
> KIP. With the detailed explanation that you provided, it is now obvious to
> me that keeping a generic type for keys like K won't work oob and hence a
> decision was made to use Bytes as the key type.
> 
> I just had another thought on this though. I looked at the range function
> that was added in the ReadOnlyKeyValueStore. While the Key and the Value
> mentioned in that method is generic, internally almost all queries end up
> querying using Bytes in some or the other form. I looked at not just
> RocksDb Store but other stores like InMemory store or MemoryLRU and this
> seems to be the pattern. I think this stems from the fact that these stores
> while implementing KeyValueStore pass Bytes, byte[] as the K and V values.
> Classes like MeteredKeyValueStore which don't do this, still use Bytes.wrap
> to wrap the passed keys and values and invoke the range method.
> 
> So, the point I am trying to make is, with the same behaviour - and
> ignoring for a moment that it's a separate interface which I am trying to
> "mix-in"- the issues with the key types could be resolved. I may be wrong
> though so would like to know your thoughts on this. Infact unknowingly the
> interface implementation of PrefixSeekableType in RockDBStateStore also
> passes Bytes and bytes[] as K and V.
> 
> The second part of exposing it via the publically accessible interfaces to
> which we downcast while building the topology (like KeyValueStore), I can
> clearly see now that mixing-in the way I tried to won't work. My intention
> all along was not to hamper the flow of those stores which don't support
> prefix scan as yet and hence the separate interface. But, I agree that for
> this to work, it needs to be part of some pre-defined store types like
> KVStore etc. Right now, I don't have an answer to this but mostly it would
> have to be moved there and implemented across all stores(if we see the
> worth in prefix scans :) )
> 
> Regarding the motivation, I am sorry if I wasn't clear. This originated
> from one of my own use cases with kafka streams where i needed to find some
> keys based upon certain prefix. Infact it's similar to the
> RangeScanCombinedKeyUsage diagram in KIP-213 where the otherTable tries to
> find entries in the state store based upon the FK. I was using
> KevValueStore to be precise. I also remember having a slack conversation on
> this, and I was told that this isn't supported right now, but some other
> users shared their experiences on how with some hacks they are able to
> perform prefix scans even though their use case fits the bill for a prefix
> scan. That kind of motivated me to take a stab at it. Unfortunately, I have
> lost the slack chat because of some cleanup at the slack channel level. I
> will try and update the ambiguous motivation statement in the near future.
> 
> Lastly, I would like to point out, that your response was not at all
> discouraging. On the contrary it was really insightful and it's always good
> to learn/discover new things :)
> 
> Thanks!
> Sagar.
> 
> On Fri, May 15, 2020 at 7:37 AM John Roesler  wrote:
> 
> > Hi, Sagar!
> >
> 

Re: [DISCUSS] Apache Kafka 2.6.0 release

2020-05-30 Thread Randall Hauch
Hi, Kowshik,

Thanks for the update on KIP-584. This is listed on the "Postponed" section
of the AK 2.6.0 release plan (
https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=152113430).

Best regards,

Randall

On Fri, May 29, 2020 at 4:51 PM Kowshik Prakasam 
wrote:

> Hi Randall,
>
> We have to remove KIP-584 from the release plan, as this item will not be
> completed for 2.6 release (although KIP is accepted). We plan to include it
> in a next release.
>
>
> Cheers,
> Kowshik
>
>
> On Fri, May 29, 2020 at 11:43 AM Maulin Vasavada <
> maulin.vasav...@gmail.com>
> wrote:
>
> > Hi Randall Hauch
> >
> > Can we add KIP-519 to 2.6? It was merged to Trunk already in April -
> >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=128650952
> > .
> >
> > Thanks
> > Maulin
> >
> > On Fri, May 29, 2020 at 11:01 AM Randall Hauch  wrote:
> >
> > > Here's an update on the AK 2.6.0 release.
> > >
> > > Code freeze was Wednesday, and the release plan [1] has been updated to
> > > reflect all of the KIPs that made the release. We've also cut the `2.6`
> > > branch that we'll use for the release; see separate email announcing
> the
> > > new branch.
> > >
> > > The next important date for the 2.6.0 release is CODE FREEZE on JUNE
> 10,
> > > and until that date all bug fixes are still welcome on the release
> > branch.
> > > But after that, only blocker bugs can be merged to the release branch.
> > >
> > > If you have any questions or concerns, please contact me or (better
> yet)
> > > reply to this thread.
> > >
> > > Thanks, and best regards!
> > >
> > > Randall
> > >
> > > [1] AK 2.6.0 Release Plan:
> > >
> >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=152113430
> > >
> > >
> > > On Wed, May 27, 2020 at 5:53 PM Matthias J. Sax 
> > wrote:
> > >
> > > > Thanks Randall!
> > > >
> > > > I added missing KIP-594.
> > > >
> > > >
> > > > For the postponed KIP section: I removed KIP-441 and KIP-444 as both
> > are
> > > > completed.
> > > >
> > > >
> > > > -Matthias
> > > >
> > > > On 5/27/20 2:31 PM, Randall Hauch wrote:
> > > > > Hey everyone, just a quick update on the 2.6.0 release.
> > > > >
> > > > > Based on the release plan (
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=152113430
> > > > ),
> > > > > today (May 27) is feature freeze. Any major feature work that is
> not
> > > > > already complete will need to push out to the next release (either
> > 2.7
> > > or
> > > > > 3.0). There are a few PRs for KIPs that are nearing completion, and
> > > we're
> > > > > having some Jenkins build issues. I will send another email later
> > today
> > > > or
> > > > > early tomorrow with an update, and I plan to cut the release branch
> > > > shortly
> > > > > thereafter.
> > > > >
> > > > > I have also updated the list of planned KIPs on the release plan
> > page (
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=152113430
> > > > ),
> > > > > and I've moved to the "Postponed" table any KIP that looks like it
> is
> > > not
> > > > > going to be complete today. If any KIP is in the wrong table,
> please
> > > let
> > > > me
> > > > > know.
> > > > >
> > > > > If you have any questions or concerns, please feel free to reply to
> > > this
> > > > > thread.
> > > > >
> > > > > Thanks, and best regards!
> > > > >
> > > > > Randall
> > > > >
> > > > > On Wed, May 20, 2020 at 2:16 PM Sophie Blee-Goldman <
> > > sop...@confluent.io
> > > > >
> > > > > wrote:
> > > > >
> > > > >> Hey Randall,
> > > > >>
> > > > >> Can you also add KIP-613 which was accepted yesterday?
> > > > >>
> > > > >> Thanks!
> > > > >> Sophie
> > > > >>
> > > > >> On Wed, May 20, 2020 at 6:47 AM Randall Hauch 
> > > wrote:
> > > > >>
> > > > >>> Hi, Tom. I saw last night that the KIP had enough votes before
> > > today’s
> > > > >>> deadline and I will add it to the roadmap today. Thanks for
> driving
> > > > this!
> > > > >>>
> > > > >>> On Wed, May 20, 2020 at 6:18 AM Tom Bentley  >
> > > > wrote:
> > > > >>>
> > > >  Hi Randall,
> > > > 
> > > >  Can we add KIP-585? (I'm not quite sure of the protocol here,
> but
> > > > >> thought
> > > >  it better to ask than to just add it myself).
> > > > 
> > > >  Thanks,
> > > > 
> > > >  Tom
> > > > 
> > > >  On Tue, May 5, 2020 at 6:54 PM Randall Hauch  >
> > > > >> wrote:
> > > > 
> > > > > Greetings!
> > > > >
> > > > > I'd like to volunteer to be release manager for the next
> > time-based
> > > >  feature
> > > > > release which will be 2.6.0. I've published a release plan at
> > > > >
> > > > 
> > > > >>>
> > > > >>
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=152113430
> > > > > ,
> > > > > and have included all of the KIPs that are currently approved
> or
> > > > >>> actively
> > > > > in discussion (though I'm happy to adjust as necessary).
> > > 

Re: [DISCUSS] Apache Kafka 2.6.0 release

2020-05-30 Thread Randall Hauch
Hi, Maulin.

Thanks for pointing out that KIP-519 was already merged in April. I've
corrected the
https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
page
to reflect this, and have added it to the AK 2.6.0 release plan.

Best regards,

Randall

On Fri, May 29, 2020 at 1:43 PM Maulin Vasavada 
wrote:

> Hi Randall Hauch
>
> Can we add KIP-519 to 2.6? It was merged to Trunk already in April -
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=128650952
> .
>
> Thanks
> Maulin
>
> On Fri, May 29, 2020 at 11:01 AM Randall Hauch  wrote:
>
> > Here's an update on the AK 2.6.0 release.
> >
> > Code freeze was Wednesday, and the release plan [1] has been updated to
> > reflect all of the KIPs that made the release. We've also cut the `2.6`
> > branch that we'll use for the release; see separate email announcing the
> > new branch.
> >
> > The next important date for the 2.6.0 release is CODE FREEZE on JUNE 10,
> > and until that date all bug fixes are still welcome on the release
> branch.
> > But after that, only blocker bugs can be merged to the release branch.
> >
> > If you have any questions or concerns, please contact me or (better yet)
> > reply to this thread.
> >
> > Thanks, and best regards!
> >
> > Randall
> >
> > [1] AK 2.6.0 Release Plan:
> >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=152113430
> >
> >
> > On Wed, May 27, 2020 at 5:53 PM Matthias J. Sax 
> wrote:
> >
> > > Thanks Randall!
> > >
> > > I added missing KIP-594.
> > >
> > >
> > > For the postponed KIP section: I removed KIP-441 and KIP-444 as both
> are
> > > completed.
> > >
> > >
> > > -Matthias
> > >
> > > On 5/27/20 2:31 PM, Randall Hauch wrote:
> > > > Hey everyone, just a quick update on the 2.6.0 release.
> > > >
> > > > Based on the release plan (
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=152113430
> > > ),
> > > > today (May 27) is feature freeze. Any major feature work that is not
> > > > already complete will need to push out to the next release (either
> 2.7
> > or
> > > > 3.0). There are a few PRs for KIPs that are nearing completion, and
> > we're
> > > > having some Jenkins build issues. I will send another email later
> today
> > > or
> > > > early tomorrow with an update, and I plan to cut the release branch
> > > shortly
> > > > thereafter.
> > > >
> > > > I have also updated the list of planned KIPs on the release plan
> page (
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=152113430
> > > ),
> > > > and I've moved to the "Postponed" table any KIP that looks like it is
> > not
> > > > going to be complete today. If any KIP is in the wrong table, please
> > let
> > > me
> > > > know.
> > > >
> > > > If you have any questions or concerns, please feel free to reply to
> > this
> > > > thread.
> > > >
> > > > Thanks, and best regards!
> > > >
> > > > Randall
> > > >
> > > > On Wed, May 20, 2020 at 2:16 PM Sophie Blee-Goldman <
> > sop...@confluent.io
> > > >
> > > > wrote:
> > > >
> > > >> Hey Randall,
> > > >>
> > > >> Can you also add KIP-613 which was accepted yesterday?
> > > >>
> > > >> Thanks!
> > > >> Sophie
> > > >>
> > > >> On Wed, May 20, 2020 at 6:47 AM Randall Hauch 
> > wrote:
> > > >>
> > > >>> Hi, Tom. I saw last night that the KIP had enough votes before
> > today’s
> > > >>> deadline and I will add it to the roadmap today. Thanks for driving
> > > this!
> > > >>>
> > > >>> On Wed, May 20, 2020 at 6:18 AM Tom Bentley 
> > > wrote:
> > > >>>
> > >  Hi Randall,
> > > 
> > >  Can we add KIP-585? (I'm not quite sure of the protocol here, but
> > > >> thought
> > >  it better to ask than to just add it myself).
> > > 
> > >  Thanks,
> > > 
> > >  Tom
> > > 
> > >  On Tue, May 5, 2020 at 6:54 PM Randall Hauch 
> > > >> wrote:
> > > 
> > > > Greetings!
> > > >
> > > > I'd like to volunteer to be release manager for the next
> time-based
> > >  feature
> > > > release which will be 2.6.0. I've published a release plan at
> > > >
> > > 
> > > >>>
> > > >>
> > >
> >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=152113430
> > > > ,
> > > > and have included all of the KIPs that are currently approved or
> > > >>> actively
> > > > in discussion (though I'm happy to adjust as necessary).
> > > >
> > > > To stay on our time-based cadence, the KIP freeze is on May 20
> > with a
> > > > target release date of June 24.
> > > >
> > > > Let me know if there are any objections.
> > > >
> > > > Thanks,
> > > > Randall Hauch
> > > >
> > > 
> > > >>>
> > > >>
> > > >
> > >
> > >
> >
>


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

2020-05-30 Thread Apache Jenkins Server
See 


Changes:

[github] Remove redundant `containsKey` call in KafkaProducer (#8761)


--
[...truncated 6.28 MB...]
org.apache.kafka.streams.TopologyTestDriverTest > 
shouldProcessRecordForTopic[Eos enabled = true] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldProcessRecordForTopic[Eos enabled = true] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldForwardRecordsFromSubtopologyToSubtopology[Eos enabled = true] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldForwardRecordsFromSubtopologyToSubtopology[Eos enabled = true] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldNotUpdateStoreForSmallerValue[Eos enabled = true] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldNotUpdateStoreForSmallerValue[Eos enabled = true] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldCreateStateDirectoryForStatefulTopology[Eos enabled = true] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldCreateStateDirectoryForStatefulTopology[Eos enabled = true] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldPunctuateIfWallClockTimeAdvances[Eos enabled = true] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldPunctuateIfWallClockTimeAdvances[Eos enabled = true] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > shouldCloseProcessor[Eos 
enabled = false] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > shouldCloseProcessor[Eos 
enabled = false] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldFeedStoreFromGlobalKTable[Eos enabled = false] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldFeedStoreFromGlobalKTable[Eos enabled = false] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldCleanUpPersistentStateStoresOnClose[Eos enabled = false] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldCleanUpPersistentStateStoresOnClose[Eos enabled = false] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldThrowForUnknownTopicDeprecated[Eos enabled = false] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldThrowForUnknownTopicDeprecated[Eos enabled = false] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldThrowPatternNotValidForTopicNameException[Eos enabled = false] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldThrowPatternNotValidForTopicNameException[Eos enabled = false] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldEnqueueLaterOutputsAfterEarlierOnes[Eos enabled = false] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldEnqueueLaterOutputsAfterEarlierOnes[Eos enabled = false] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldUseSourceSpecificDeserializersDeprecated[Eos enabled = false] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldUseSourceSpecificDeserializersDeprecated[Eos enabled = false] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldPunctuateIfEvenTimeAdvances[Eos enabled = false] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldPunctuateIfEvenTimeAdvances[Eos enabled = false] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > shouldInitProcessor[Eos 
enabled = false] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > shouldInitProcessor[Eos 
enabled = false] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldThrowForUnknownTopic[Eos enabled = false] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldThrowForUnknownTopic[Eos enabled = false] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldPunctuateOnStreamsTime[Eos enabled = false] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldPunctuateOnStreamsTime[Eos enabled = false] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldCaptureGlobalTopicNameIfWrittenInto[Eos enabled = false] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldCaptureGlobalTopicNameIfWrittenInto[Eos enabled = false] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldThrowIfInMemoryBuiltInStoreIsAccessedWithUntypedMethod[Eos enabled = 
false] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldThrowIfInMemoryBuiltInStoreIsAccessedWithUntypedMethod[Eos enabled = 
false] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldProcessFromSourcesThatMatchMultiplePattern[Eos enabled = false] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldProcessFromSourcesThatMatchMultiplePattern[Eos enabled = false] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > shouldPopulateGlobalStore[Eos 
enabled = false] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > shouldPopulateGlobalStore[Eos 
enabled = false] PASSED


Re: [DISCUSS] KIP-616: Rename implicit Serdes instances in kafka-streams-scala

2020-05-30 Thread John Roesler
Hi Yuriy,

I was just looking back at KIP-513, and I’m wondering if there’s any overlap we 
should consider here, or if they are just orthogonal.

Thanks,
-John

On Thu, May 28, 2020, at 21:36, Yuriy Badalyantc wrote:
> At the current moment, I think John's plan is better than the original plan
> described in the KIP. I think we should create a new `Serdes` in another
> package. The old one will be deprecated.
> 
> - Yuriy
> 
> On Fri, May 29, 2020 at 8:58 AM John Roesler  wrote:
> 
> > Thanks, Matthias,
> >
> > If we go with the approach Yuriy and I agreed on, to deprecate and replace
> > the whole class and not just a few of the methods, then the timeline is
> > less of a concern. Under that plan, Yuriy can just write the new class
> > exactly the way he wants and people can cleanly swap over to the new
> > pattern when they are ready.
> >
> > The timeline was more significant if we were just going to deprecate some
> > methods and add new methods to the existing class. That plan requires two
> > implementation phases, where we first deprecate the existing methods and
> > later swap the implicits at the same time we remove the deprecated members.
> > Aside from the complexity of that approach, it’s not a breakage free path,
> > as some users would be forced to continue using the deprecated members
> > until a future release drops them, breaking their source code, and only
> > then can they update their code.
> >
> > That wouldn’t be the end of the world, and we’ve had to do the same thing
> > in the past with the implicit conversations, but this is a much wider
> > scope, since it’s all the serdes. I’m happy with the new plan, since it’s
> > not only one step, but also it provides everyone a breakage-free path.
> >
> > We can still consider dropping the deprecated class in 3.0; I just wanted
> > to clarify how the timeline issue has changed.
> >
> > Thanks,
> > John
> >
> > On Thu, May 28, 2020, at 20:34, Matthias J. Sax wrote:
> > > I am not a Scale person, so I cannot really contribute much. However for
> > > the deprecation period, if we get the change into 2.7, it might be ok to
> > > remove the deprecated classed in 3.0.
> > >
> > > It would only be one minor release in between what is a little bit short
> > > (we usually prefer at least two minor released, better three), but if we
> > > have a good reason for it, it might be ok.
> > >
> > > If we cannot remove it in 3.0, it seems there would be a 4.0 in about a
> > > year(?) when ZK removal is finished and we can remove the deprecated
> > > code than.
> > >
> > >
> > > -Matthias
> > >
> > > On 5/28/20 7:39 AM, John Roesler wrote:
> > > > Hi Yuriy,
> > > >
> > > > Sounds good to me! I had a feeling we were bringing different context
> > > > to the discussion; thanks for sticking with the conversation until we
> > got
> > > > it hashed out.
> > > >
> > > > I'm glad you prefer Serde*s*, since having multiple different classes
> > with
> > > > the same name leads to all kinds of trouble. "Serdes" seems relatively
> > > > safe because people in the Scala lib won't be using the Java Serdes
> > class,
> > > > and they won't be using the deprecated and non-deprecated one at the
> > > > same time.
> > > >
> > > > Thank again,
> > > > -John
> > > >
> > > > On Thu, May 28, 2020, at 02:21, Yuriy Badalyantc wrote:
> > > >> Ok, I understood you, John. I wasn't sure about kafka deprecation
> > policy
> > > >> and thought that the full cycle could be done with 2.7 version.
> > Waiting for
> > > >> 3.0 is too much, I agree with it.
> > > >>
> > > >> So, I think creating one more `Serdes` in another package is our way.
> > I
> > > >> suggest one of the following:
> > > >> 1. `org.apache.kafka.streams.scala.serde.Serdes`
> > > >> 2. `org.apache.kafka.streams.scala.serialization.Serdes`
> > > >>
> > > >> About `Serde` vs `Serdes`. I'm strongly against `Serde` because it
> > would
> > > >> lead to a new name clash with the
> > > >> `org.apache.kafka.common.serialization.Serde`.
> > > >>
> > > >> - Yuriy
> > > >>
> > > >> On Thu, May 28, 2020 at 11:12 AM John Roesler 
> > wrote:
> > > >>
> > > >>> Hi Yuriy,
> > > >>>
> > > >>> Thanks for the clarification.
> > > >>>
> > > >>> I guess my concern is twofold:
> > > >>> 1. We typically leave deprecated methods in place for at least a
> > major
> > > >>> release cycle before removing them, so it would seem abrupt to have a
> > > >>> deprecation period of only one minor release. If we follow the same
> > pattern
> > > >>> here, it would take over a year to finish this KIP.
> > > >>> 2. It doesn’t seem like there is a nonbreaking deprecation path at
> > all if
> > > >>> people enumerate their imports (if they don’t use a wildcard). In
> > that
> > > >>> case, they would have no path to implicitly use the newly named
> > serdes, and
> > > >>> therefore they would have no way to avoid continuing to use the
> > deprecated
> > > >>> ones.
> > > >>>
> > > >>> Since you mentioned that your reason is mainly the preference for
> > 

Re: [DISCUSS] KIP-513: Distinguish between Key and Value serdes in scala wrapper library for kafka streams

2020-05-30 Thread John Roesler
Hi Mykhailo,

Wow, I really dropped the ball here. I have just looked over your KIP again, 
and now I see how you don’t need to change every dsl method, only Consumed, 
Materialized, etc. 

I think this would be a good addition. Yuriy has just proposed KIP-616 to fix 
some other problems with the implicit serdes. I’m wondering if these two kips 
have any joint opportunities we should consider, or if it’s better to continue 
to consider them separately. 

Thanks,
John

On Wed, Jan 22, 2020, at 16:18, Михаил Ерёменко wrote:
> Hi, John! 
> 
> Sorry for the late reply. I am not really familiar with this mail list 
> discussions, so I have not seen your mails.
> 
> Regarding your question:
> >   I guess what
>   I'm struggling with is why you actually want to have different key and
>   serdes for the same type
> 
> I think good example will be (and it is actually what we do in ours 
> project) using confluent schema registry in conjunction with kafka 
> streams. Some models can be used as keys as well as values. When we 
> define schema registry compatible serde, we have to specify is it for 
> key or not. We can of course create two serdes for the same model, but 
> in this case implicit semantic will not work because scala doesn’t know 
> which implicit to pick. And things become even more complicated in case 
> if you will try to derive your serdes (we derive serdes in ours 
> project).
> 
> One more thing:
> > every method in the streams-scala DSL.
> 
> So far we've just changed 
> org.apache.kafka.streams.scala.ImplicitConversions and 
> org.apache.kafka.streams.scala.kstream.Materialized and it works for 
> us. Also we did introduce default serdes for primitive types. 
> 
> Regards,
> Mykhailo


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

2020-05-30 Thread Apache Jenkins Server
See 


Changes:

[github] Remove redundant `containsKey` call in KafkaProducer (#8761)


--
[...truncated 3.14 MB...]

org.apache.kafka.streams.TestTopicsTest > testRecordsToList PASSED

org.apache.kafka.streams.TestTopicsTest > testKeyValueListDuration STARTED

org.apache.kafka.streams.TestTopicsTest > testKeyValueListDuration PASSED

org.apache.kafka.streams.TestTopicsTest > testInputToString STARTED

org.apache.kafka.streams.TestTopicsTest > testInputToString PASSED

org.apache.kafka.streams.TestTopicsTest > testTimestamp STARTED

org.apache.kafka.streams.TestTopicsTest > testTimestamp PASSED

org.apache.kafka.streams.TestTopicsTest > testWithHeaders STARTED

org.apache.kafka.streams.TestTopicsTest > testWithHeaders PASSED

org.apache.kafka.streams.TestTopicsTest > testKeyValue STARTED

org.apache.kafka.streams.TestTopicsTest > testKeyValue PASSED

org.apache.kafka.streams.TestTopicsTest > 
shouldNotAllowToCreateTopicWithNullTopicName STARTED

org.apache.kafka.streams.TestTopicsTest > 
shouldNotAllowToCreateTopicWithNullTopicName PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldCreateConsumerRecordsFromKeyValuePairs STARTED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldCreateConsumerRecordsFromKeyValuePairs PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldNotAllowToCreateTopicWithNullTopicNameWithNullKeyAndDefaultTimestamp 
STARTED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldNotAllowToCreateTopicWithNullTopicNameWithNullKeyAndDefaultTimestamp 
PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldRequireCustomTopicNameIfNotDefaultFactoryTopicNameWithDefaultTimestamp 
STARTED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldRequireCustomTopicNameIfNotDefaultFactoryTopicNameWithDefaultTimestamp 
PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldNotAllowToCreateTopicWithNullTopicNameWithNullKey STARTED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldNotAllowToCreateTopicWithNullTopicNameWithNullKey PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldCreateNullKeyConsumerRecordWithOtherTopicNameAndTimestampWithTimetamp 
STARTED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldCreateNullKeyConsumerRecordWithOtherTopicNameAndTimestampWithTimetamp 
PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldCreateConsumerRecordWithTimestamp STARTED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldCreateConsumerRecordWithTimestamp PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldNotAllowToCreateTopicWithNullHeaders STARTED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldNotAllowToCreateTopicWithNullHeaders PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldNotAllowToCreateTopicWithNullTopicNameWithDefaultTimestamp STARTED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldNotAllowToCreateTopicWithNullTopicNameWithDefaultTimestamp PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldRequireCustomTopicNameIfNotDefaultFactoryTopicName STARTED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldRequireCustomTopicNameIfNotDefaultFactoryTopicName PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldRequireCustomTopicNameIfNotDefaultFactoryTopicNameWithNullKey STARTED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldRequireCustomTopicNameIfNotDefaultFactoryTopicNameWithNullKey PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldCreateConsumerRecord STARTED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldCreateConsumerRecord PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldCreateNullKeyConsumerRecord STARTED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldCreateNullKeyConsumerRecord PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldCreateConsumerRecordWithOtherTopicName STARTED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldCreateConsumerRecordWithOtherTopicName PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > shouldAdvanceTime 
STARTED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > shouldAdvanceTime 
PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldNotAllowToCreateTopicWithNullTopicNameWithKeyValuePairs STARTED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldNotAllowToCreateTopicWithNullTopicNameWithKeyValuePairs PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldRequireCustomTopicNameIfNotDefaultFactoryTopicNameWithKeyValuePairsAndCustomTimestamps
 STARTED


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

2020-05-30 Thread Apache Jenkins Server
See 


Changes:

[github] KAFKA-9494; Include additional metadata information in DescribeConfig


--
[...truncated 6.28 MB...]
org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldCreateConsumerRecordsFromKeyValuePairs PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldNotAllowToCreateTopicWithNullTopicNameWithNullKeyAndDefaultTimestamp 
STARTED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldNotAllowToCreateTopicWithNullTopicNameWithNullKeyAndDefaultTimestamp 
PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldRequireCustomTopicNameIfNotDefaultFactoryTopicNameWithDefaultTimestamp 
STARTED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldRequireCustomTopicNameIfNotDefaultFactoryTopicNameWithDefaultTimestamp 
PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldNotAllowToCreateTopicWithNullTopicNameWithNullKey STARTED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldNotAllowToCreateTopicWithNullTopicNameWithNullKey PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldCreateNullKeyConsumerRecordWithOtherTopicNameAndTimestampWithTimetamp 
STARTED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldCreateNullKeyConsumerRecordWithOtherTopicNameAndTimestampWithTimetamp 
PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldCreateConsumerRecordWithTimestamp STARTED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldCreateConsumerRecordWithTimestamp PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldNotAllowToCreateTopicWithNullHeaders STARTED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldNotAllowToCreateTopicWithNullHeaders PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldNotAllowToCreateTopicWithNullTopicNameWithDefaultTimestamp STARTED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldNotAllowToCreateTopicWithNullTopicNameWithDefaultTimestamp PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldRequireCustomTopicNameIfNotDefaultFactoryTopicName STARTED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldRequireCustomTopicNameIfNotDefaultFactoryTopicName PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldRequireCustomTopicNameIfNotDefaultFactoryTopicNameWithNullKey STARTED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldRequireCustomTopicNameIfNotDefaultFactoryTopicNameWithNullKey PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldCreateConsumerRecord STARTED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldCreateConsumerRecord PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldCreateNullKeyConsumerRecord STARTED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldCreateNullKeyConsumerRecord PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldCreateConsumerRecordWithOtherTopicName STARTED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldCreateConsumerRecordWithOtherTopicName PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > shouldAdvanceTime 
STARTED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > shouldAdvanceTime 
PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldNotAllowToCreateTopicWithNullTopicNameWithKeyValuePairs STARTED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldNotAllowToCreateTopicWithNullTopicNameWithKeyValuePairs PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldRequireCustomTopicNameIfNotDefaultFactoryTopicNameWithKeyValuePairsAndCustomTimestamps
 STARTED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldRequireCustomTopicNameIfNotDefaultFactoryTopicNameWithKeyValuePairsAndCustomTimestamps
 PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldRequireCustomTopicNameIfNotDefaultFactoryTopicNameWithKeyValuePairs 
STARTED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldRequireCustomTopicNameIfNotDefaultFactoryTopicNameWithKeyValuePairs PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldCreateConsumerRecordWithOtherTopicNameAndTimestamp STARTED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldCreateConsumerRecordWithOtherTopicNameAndTimestamp PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldNotAllowToCreateTopicWithNullTopicNameWithKeyValuePairsAndCustomTimestamps
 STARTED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 
shouldNotAllowToCreateTopicWithNullTopicNameWithKeyValuePairsAndCustomTimestamps
 PASSED

org.apache.kafka.streams.test.ConsumerRecordFactoryTest > 

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

2020-05-30 Thread Apache Jenkins Server
See