Thanks Viktor. I think it would be good to verify that existing
ExtendedSerializer implementations work without recompiling. This could be
done as a manual test. If you agree, I suggest adding it to the testing
plan section.

Ismael

On Mon, Aug 27, 2018 at 7:57 AM Viktor Somogyi-Vass <viktorsomo...@gmail.com>
wrote:

> Thanks guys, I've updated my KIP with this info (so to keep solution #1).
> If you find it good enough, please vote as well or let me know if you think
> something is missing.
>
> On Sat, Aug 25, 2018 at 1:14 AM Ismael Juma <ism...@juma.me.uk> wrote:
>
> > I'm OK with 1 too. It makes me a bit sad that we don't have a path for
> > removing the method without headers, but it seems like the simplest and
> > least confusing option (I am assuming that headers are not needed in the
> > serializers in the common case).
> >
> > Ismael
> >
> > On Fri, Aug 24, 2018 at 2:42 PM Jason Gustafson <ja...@confluent.io>
> > wrote:
> >
> > > Hey Viktor,
> > >
> > > Good summary. I agree that option 1) seems like the simplest choice
> and,
> > as
> > > you note, we can always add the default implementation later. I'll
> leave
> > > Ismael to make a case for the circular forwarding approach ;)
> > >
> > > -Jason
> > >
> > > On Fri, Aug 24, 2018 at 3:02 AM, Viktor Somogyi-Vass <
> > > viktorsomo...@gmail.com> wrote:
> > >
> > > > I think in the first draft I didn't provide an implementation for
> them
> > as
> > > > it seemed very simple and straightforward. I looked up a couple of
> > > > implementations of the ExtendedSerializers on github and the general
> > > > behavior seems to be that they delegate to the 2 argument
> (headerless)
> > > > method:
> > > >
> > > > https://github.com/khoitnm/practice-spring-kafka-grpc/blob/
> > > > a6fc9b3395762c4889807baedd822f4653d5dcdd/kafka-common/src/
> > > > main/java/org/tnmk/common/kafka/serialization/protobuf/
> > > > ProtobufSerializer.java
> > > >
> https://github.com/hong-zhu/nxgen/blob/5cf1427d4e1a8f5c7fab47955af32a
> > > > 0d4f4873af/nxgen-kafka-client/src/main/java/nxgen/kafka/
> > > > client/event/serdes/EventSerializer.java
> > > > https://github.com/jerry-jx/spring-kafka/blob/
> > > > ac323ec5b8b9a0ca975db2f7322ff6878fce481a/spring-kafka/src/
> > > >
> > >
> >
> main/java/org/springframework/kafka/support/serializer/JsonSerializer.java
> > > > https://github.com/enzobonggio/nonblocking-kafka/blob/
> > > > bc1a379b2d9593b46cf9604063bc5b38e2785d19/src/main/java/com/
> > > > example/kafka/producer/CustomJsonSerializer.java
> > > >
> > > > Of course 4 example is not representative but it shows that these
> users
> > > > usually delegate to the "headerless" (2 argument) method. I've tried
> to
> > > > look it up on other code search sites but haven't had much luck so
> far.
> > > > Given these examples and the way they implement them I'd say it's
> more
> > > > common to delegate to the headerless method, that's why I think it's
> a
> > > good
> > > > approach for us too. Now having a default implementation for that is
> > > again
> > > > a good question. I think current use cases wouldn't change in either
> > case
> > > > (unless we deprecate the headerless one).
> > > > For the new use cases it depends what do we want to propagate going
> > > > forward. Do we want only one method to exist or two? As Ismael
> > > highlighted
> > > > it might be confusing if we have 2 methods, both with default
> > > > implementation and in this case we want to push the 3 argument one
> for
> > > > users.
> > > >
> > > > So I see three possible ways:
> > > > 1.) Don't provide a default implementation for the headerless method.
> > > This
> > > > supports the current implementations and encourages the delegation
> > style
> > > in
> > > > future implementations. This might be the simplest option.
> > > > 2.) Provide a default implementation for the headerless method. This
> > > would
> > > > be a bit confusing, so we'd likely push the use of the 3 parameter
> > method
> > > > and deprecate the headerless. This would however further litter the
> > code
> > > > base with deprecation warnings as we're using the headerless method
> in
> > a
> > > > lot of places (think of the current serializers/deserializers). So in
> > > this
> > > > case we would want to clean up the code base a little where we can
> and
> > > may
> > > > remove the headerless method entirely in Kafka 3. But they would hang
> > > > around until that point. I think in this case the implementation for
> > the
> > > > headerless is a detail question as that is deprecated so we don't
> > expect
> > > > new implementations to use that method.
> > > > If we decide to move this way, we have explored two options so far:
> > > > returning null / empty array or throwing exceptions. (And I honestly
> > > > started to like the latter as calling that with no real
> implementation
> > is
> > > > really a programming error.)
> > > > 3.) We can do it in multiple steps. In the first step we do 1 and
> later
> > > 2.
> > > > I think it would also make sense as the Kafka code base heavily uses
> > the
> > > > headerless method still (think of the existing
> > serializers/deserializers)
> > > > and it would give us time to eliminate/change those use cases.
> > > >
> > > > Cheers,
> > > > Viktor
> > > >
> > > > On Thu, Aug 23, 2018 at 11:55 PM Jason Gustafson <ja...@confluent.io
> >
> > > > wrote:
> > > >
> > > > > To clarify, what I am suggesting is to only remove the default
> > > > > implementation for these methods. So users would be required to
> > > implement
> > > > > serialize(topic, data) and deserialize(topic, data).
> > > > >
> > > > > -Jason
> > > > >
> > > > > On Thu, Aug 23, 2018 at 1:48 PM, Jason Gustafson <
> ja...@confluent.io
> > >
> > > > > wrote:
> > > > >
> > > > > > Hey Viktor,
> > > > > >
> > > > > > Thinking about it a little more, I wonder if we should just not
> > > > provide a
> > > > > > default method for serialize(topic, data) and deserialize(topic,
> > > data).
> > > > > > Implementing these methods is a trivial burden for users and it
> > feels
> > > > > like
> > > > > > there's no good solution which allows both methods to have
> default
> > > > > > implementations.
> > > > > >
> > > > > > Also, ack on KIP-331. Thanks for the pointer.
> > > > > >
> > > > > > -Jason
> > > > > >
> > > > > > On Thu, Aug 23, 2018 at 12:30 PM, Viktor Somogyi-Vass <
> > > > > > viktorsomo...@gmail.com> wrote:
> > > > > >
> > > > > >> Hi Ismael,
> > > > > >>
> > > > > >> Regarding the deprecation of the 2 parameter method: should we
> do
> > > this
> > > > > >> with
> > > > > >> the Serializer interface as well?
> > > > > >>
> > > > > >> I've updated the "Rejected Alternatives" with a few.
> > > > > >> I've added this circular reference one too but actually there's
> a
> > > way
> > > > > >> (pretty heavyweight) by adding a guard class that prevents
> > recursive
> > > > > >> invocation of either methods. I've tried this out but it seems
> to
> > me
> > > > an
> > > > > >> overshoot. So just for the sake of completeness I'll copy it
> here.
> > > :)
> > > > > >>
> > > > > >> public interface Deserializer<T> extends Closeable {
> > > > > >>
> > > > > >>     class Guard {
> > > > > >>
> > > > > >>         private Set<Object> objects =
> > > Collections.synchronizedSet(new
> > > > > >> HashSet<>()); // might as well use concurrent hashmap
> > > > > >>
> > > > > >>         private void methodCallInProgress(Object x) {
> > > > > >>             objects.add(x);
> > > > > >>         }
> > > > > >>
> > > > > >>         private boolean isMethodCallInProgress(Object x) {
> > > > > >>             return objects.contains(x);
> > > > > >>         }
> > > > > >>
> > > > > >>         private void clearMethodCallInProgress(Object x) {
> > > > > >>             objects.remove(x);
> > > > > >>         }
> > > > > >>
> > > > > >>         private <T> T guard(Supplier<T> supplier) {
> > > > > >>             if (GUARD.isMethodCallInProgress(this)) {
> > > > > >>                 throw new IllegalStateException("You must
> > implement
> > > > one
> > > > > of
> > > > > >> the deserialize methods");
> > > > > >>             } else {
> > > > > >>                 try {
> > > > > >>                     GUARD.methodCallInProgress(this);
> > > > > >>                     return supplier.get();
> > > > > >>                 } finally {
> > > > > >>                     GUARD.clearMethodCallInProgress(this);
> > > > > >>                 }
> > > > > >>             }
> > > > > >>         }
> > > > > >>     }
> > > > > >>
> > > > > >>     Guard GUARD = new Guard();
> > > > > >>
> > > > > >>     void configure(Map<String, ?> configs, boolean isKey);
> > > > > >>
> > > > > >>     default T deserialize(String topic, byte[] data) {
> > > > > >>         return GUARD.guard(() -> deserialize(topic, null,
> data));
> > > > > >>     }
> > > > > >>
> > > > > >>     default T deserialize(String topic, Headers headers, byte[]
> > > data)
> > > > {
> > > > > >>         return GUARD.guard(() -> deserialize(topic, data));
> > > > > >>     }
> > > > > >>
> > > > > >>     @Override
> > > > > >>     void close();
> > > > > >> }
> > > > > >>
> > > > > >>
> > > > > >> Cheers,
> > > > > >> Viktor
> > > > > >>
> > > > > >> On Thu, Aug 23, 2018 at 3:50 PM Ismael Juma <ism...@juma.me.uk>
> > > > wrote:
> > > > > >>
> > > > > >> > Also, we may consider deprecating the deserialize method that
> > does
> > > > not
> > > > > >> take
> > > > > >> > headers. Yes, it's a convenience, but it also adds confusion.
> > > > > >> >
> > > > > >> > Ismael
> > > > > >> >
> > > > > >> > On Thu, Aug 23, 2018 at 6:48 AM Ismael Juma <
> ism...@juma.me.uk>
> > > > > wrote:
> > > > > >> >
> > > > > >> > > I think the KIP needs the rejected alternatives section to
> > have
> > > > more
> > > > > >> > > detail. For example, another option would be something like
> > the
> > > > > >> > following,
> > > > > >> > > which works great as long as one overrides one of the
> methods,
> > > but
> > > > > >> pretty
> > > > > >> > > bad if one doesn't. :)
> > > > > >> > >
> > > > > >> > > default T deserialize(String topic, byte[] data) {
> > > > > >> > >     return deserialize(topic, null, data);
> > > > > >> > > }
> > > > > >> > >
> > > > > >> > > default T deserialize(String topic, Headers headers, byte[]
> > > data)
> > > > {
> > > > > //
> > > > > >> > > This is the new method
> > > > > >> > >     return deserialize(topic, data);
> > > > > >> > > }
> > > > > >> > >
> > > > > >> > >
> > > > > >> > > On Thu, Aug 23, 2018 at 3:57 AM Viktor Somogyi-Vass <
> > > > > >> > > viktorsomo...@gmail.com> wrote:
> > > > > >> > >
> > > > > >> > >> Hi Jason,
> > > > > >> > >>
> > > > > >> > >> Thanks for the feedback.
> > > > > >> > >> 1. I chose to return null here because according to the
> > > > > >> documentation it
> > > > > >> > >> may return null data, therefore the users of this methods
> are
> > > > > >> perpared
> > > > > >> > for
> > > > > >> > >> getting a null. Thinking of it though it may be better to
> > throw
> > > > an
> > > > > >> > >> exception by default because it'd indicate a programming
> > error.
> > > > > >> However,
> > > > > >> > >> would that be a backward incompatible change? I'm simply
> > > thinking
> > > > > of
> > > > > >> > this
> > > > > >> > >> because this is a new behavior that we'd introduce but I'm
> > not
> > > > sure
> > > > > >> yet
> > > > > >> > if
> > > > > >> > >> it'd cause problems.
> > > > > >> > >> Do you think it'd make sense to do the same in `serialize`?
> > > > > >> > >> 2. Yes, I believe that is covered in KIP-331:
> > > > > >> > >>
> > > > > >> > >>
> > > > > >> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-331+
> > > > > >> Add+default+implementation+to+close%28%29+and+configure%28%
> > > > > >> 29+for+Serializer%2C+Deserializer+and+Serde
> > > > > >> > >>
> > > > > >> > >> Cheers,
> > > > > >> > >> Viktor
> > > > > >> > >>
> > > > > >> > >> On Wed, Aug 22, 2018 at 6:11 PM Jason Gustafson <
> > > > > ja...@confluent.io>
> > > > > >> > >> wrote:
> > > > > >> > >>
> > > > > >> > >> > Hey Viktor,
> > > > > >> > >> >
> > > > > >> > >> > This is a nice cleanup. Just a couple quick questions:
> > > > > >> > >> >
> > > > > >> > >> > 1. Rather than returning null for the default
> > > > `deserialize(topic,
> > > > > >> > >> data)`,
> > > > > >> > >> > would it be better to throw
> UnsupportedOperationException?
> > I
> > > > > assume
> > > > > >> > that
> > > > > >> > >> > internally we'll always invoke the api which takes
> headers.
> > > > > >> Similarly
> > > > > >> > >> for
> > > > > >> > >> > `serialize(topic, data)`.
> > > > > >> > >> > 2. Would it make sense to have default no-op
> > implementations
> > > > for
> > > > > >> > >> > `configure` and `close`?
> > > > > >> > >> >
> > > > > >> > >> > Thanks,
> > > > > >> > >> > Jason
> > > > > >> > >> >
> > > > > >> > >> > On Wed, Aug 22, 2018 at 5:27 AM, Satish Duggana <
> > > > > >> > >> satish.dugg...@gmail.com>
> > > > > >> > >> > wrote:
> > > > > >> > >> >
> > > > > >> > >> > > +1
> > > > > >> > >> > >
> > > > > >> > >> > > On Wed, Aug 22, 2018 at 4:45 PM, Ted Yu <
> > > yuzhih...@gmail.com
> > > > >
> > > > > >> > wrote:
> > > > > >> > >> > >
> > > > > >> > >> > > > +1
> > > > > >> > >> > > > -------- Original message --------From: Kamal
> > > > Chandraprakash
> > > > > <
> > > > > >> > >> > > > kamal.chandraprak...@gmail.com> Date: 8/22/18  3:19
> AM
> > > > > >> > (GMT-08:00)
> > > > > >> > >> > To:
> > > > > >> > >> > > > dev@kafka.apache.org Subject: Re: [VOTE] KIP-336:
> > > > > Consolidate
> > > > > >> > >> > > > ExtendedSerializer/Serializer and
> > > > > >> > ExtendedDeserializer/Deserializer
> > > > > >> > >> > > > +1
> > > > > >> > >> > > >
> > > > > >> > >> > > > Thanks for the KIP!
> > > > > >> > >> > > >
> > > > > >> > >> > > > On Wed, Aug 22, 2018 at 2:48 PM Viktor Somogyi-Vass <
> > > > > >> > >> > > > viktorsomo...@gmail.com>
> > > > > >> > >> > > > wrote:
> > > > > >> > >> > > >
> > > > > >> > >> > > > > Hi All,
> > > > > >> > >> > > > >
> > > > > >> > >> > > > > I'd like to start a vote on this KIP (
> > > > > >> > >> > > > > https://cwiki.apache.org/confluence/pages/viewpage
> .
> > > > > >> > >> > > > action?pageId=87298242)
> > > > > >> > >> > > > > which aims to refactor
> ExtendedSerializer/Serializer
> > > and
> > > > > >> > >> > > > > ExtendedDeserializer/Deserializer.
> > > > > >> > >> > > > >
> > > > > >> > >> > > > > To summarize what's the motivation:
> > > > > >> > >> > > > >
> > > > > >> > >> > > > > When headers were introduced by KIP-82 the
> > > > > ExtendedSerializer
> > > > > >> > and
> > > > > >> > >> > > > > ExtendedDeserializer classes were created in order
> to
> > > > keep
> > > > > >> > >> interface
> > > > > >> > >> > > > > compatibility but still add `T deserialize(String
> > > topic,
> > > > > >> Headers
> > > > > >> > >> > > headers,
> > > > > >> > >> > > > > byte[] data);` and `byte[] serialize(String topic,
> > > > Headers
> > > > > >> > >> headers, T
> > > > > >> > >> > > > > data);` methods that consume the headers for
> > > > > >> > >> > > > serialization/deserialization.
> > > > > >> > >> > > > > The reason for doing so was that Kafka at that time
> > > > needed
> > > > > be
> > > > > >> > >> > > compatbile
> > > > > >> > >> > > > > with Java 7. Since we're not compiling on Java 7
> > > anymore
> > > > > >> > >> (KAFKA-4423)
> > > > > >> > >> > > > we'll
> > > > > >> > >> > > > > try consolidate the way we're using these in a
> > backward
> > > > > >> > compatible
> > > > > >> > >> > > > fashion:
> > > > > >> > >> > > > > deprecating the Extended* classes and moving the
> > > > > >> aforementioned
> > > > > >> > >> > methods
> > > > > >> > >> > > > up
> > > > > >> > >> > > > > in the class hierarchy.
> > > > > >> > >> > > > >
> > > > > >> > >> > > > > I'd be happy to get votes or additional feedback on
> > > this.
> > > > > >> > >> > > > >
> > > > > >> > >> > > > > Viktor
> > > > > >> > >> > > > >
> > > > > >> > >> > > >
> > > > > >> > >> > >
> > > > > >> > >> >
> > > > > >> > >>
> > > > > >> > >
> > > > > >> >
> > > > > >>
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Reply via email to