[ 
https://issues.apache.org/jira/browse/KAFKA-8666?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16889058#comment-16889058
 ] 

Matthias J. Sax commented on KAFKA-8666:
----------------------------------------

{quote}we are throwing away information that the user already provided if they 
choose to build their object
{quote}
Not sure if I agree with this statement. Assume there would not be any static 
methods, but a public constructor for `Materialized`:

 
{code:java}
// current API
builder.table(..., 
Materialized.as("storeName").withKeySerde(keySerce).withValueSerde(valueSerde));
// or
builder.table(..., Materialized.with(keySerde, valueSerde));

// if we would not have static methods but public constructors it would be like 
this:
builder.table(..., new 
Materialized("storeName").withKeySerde(keySerce).withValueSerde(valueSerde));
// or
builder.table(..., new Materialized(keySerde, valueSerde));{code}
 

If you call 
{code:java}
builder.table(..., Materialized.as("storeName").with(keySerde, 
valueSerde));{code}
It's similar to call "new" twice and use the second object:
{code:java}
Materialized m;
m = new Materialized("storeName");
m = new Materialized(keySerde, valueSerde)
builder.table(..., m);{code}
Hence, KS does not "through away" information IMHO. For the "new" case it's 
obvious that the second object will not have a store name set – it's two 
objects after all. It might be somewhat subtle, but calling a static method in 
a builder like pattern is like calling "new" (and an modern IDE should actually 
give you a warning about it – a static method should never be called on an 
object, but only on the class).

 
{quote}Before going into solution mode I'd like to agree on something - we _do_ 
want the described functionality to be available, right? As you said we are 
using the `builder pattern` for that reason. Because in the commit message of 
the code says

_Add a `with(Serde keySerde, Serde valSerde)` to `Materialized` for cases where 
people don't care about the state store name._
{quote}
Yes. There are cased for which only passing in the Serdes is the best way and 
passing in a store name (that forces store materialization and disables store 
optimization) is undesired.
{quote}That leads me to believe that maybe that the code is working as it is 
supposed and we should focus and highlighting that to the developer rather than 
finding a way to comply with the request in this issue.
{quote}
Agreed.
{quote}If we do want continue `building` the the `Materialized` object please 
enlighten me what is the standard practise with Kafka APIs with regards to 
backwards compatibility? My intuition was that we shouldn't break code that 
relies on the current implementation - which will happen if we attempt to 
provide a fix for the way the current `with(keySerde, valueSerde)` is done. If 
that is not the case then a way to fix the problem is to remove the `static` 
identifier of the method. That will allows us to access previously provided 
`storeName`.
{quote}
Yes, code must be backward compatible. We could add a non-static 
`withStoreName(String)` method, that allows you to do:
{code:java}
Materialized.with(keySerde, valueSerde).withStoreName("storeName");{code}
But there woulds still be the problem that one could still do it wrong:
{code:java}
Materialized.with(keySerde, valueSerde).as("storeName");{code}
Therefore, I don't see a big advantage to add `withStoreName(String)` method. 
We can update the JavaDocs of course, but if people write "correct" JavaCode 
and don't call static methods on objects, they do it correctly automatically.

> Improve Documentation on usage of Materialized config object
> ------------------------------------------------------------
>
>                 Key: KAFKA-8666
>                 URL: https://issues.apache.org/jira/browse/KAFKA-8666
>             Project: Kafka
>          Issue Type: Improvement
>          Components: documentation, streams
>            Reporter: Bill Bejeck
>            Priority: Major
>              Labels: newbie
>
> When using the Materialized object if the user wants to name the statestore 
> with
> {code:java}
> Materialized.as("MyStoreName"){code}
> then subsequently provide the key and value serde the calls to do so must 
> take the form of 
> {code:java}
> Materialized.as("MyStoreName").withKeySerde(keySerde).withValueSerde(valueSerde)
> {code}
> If users do the following 
> {code:java}
> Materialized.as("MyStoreName").with(keySerde, valueSerde)
> {code}
> the Materialized instance created by the "as(storeName)" call is replaced by 
> a new Materialized instance resulting from the "with(...)" call and any 
> configuration on the first Materialized instance is lost.



--
This message was sent by Atlassian JIRA
(v7.6.14#76016)

Reply via email to