I for one use the consumer (Simple Consumer) without any deserialization. I
just take the ByteBuffer wrap it a preallocated flyweight and use it
without creating any objects. I'd ideally not have to wrap this logic in a
deserializer interface. For every one who does do this, it seems like a
very small step.

On Tue, Dec 2, 2014 at 5:12 PM, Joel Koshy <jjkosh...@gmail.com> wrote:

> > For (1), yes, but it's easier to make a config change than a code change.
> > If you are using a third party library, one may not be able to make any
> > code change.
>
> Doesn't that assume that all organizations have to already share the
> same underlying specific data type definition (e.g.,
> UniversalAvroRecord). If not, then wouldn't they have to anyway make a
> code change anyway to use the shared definition (since that is
> required in the parameterized type of the producerrecord and
> producer)?  And if they have already made the change to use the said
> shared definition then you could just as well have the serializer of
> UniversalAvroRecord configured in your application config and have
> that replaced if you wish by some other implementation of a serializer
> of UniversalAvroRecord (again via config).
>
> > For (2), it's just that if most consumers always do deserialization after
> > getting the raw bytes, perhaps it would be better to have these two steps
> > integrated.
>
> True, but it is just a marginal and very obvious step that shouldn't
> surprise any user.
>
> Thanks,
>
> Joel
>
> >
> > Thanks,
> >
> > Jun
> >
> > On Tue, Dec 2, 2014 at 2:05 PM, Joel Koshy <jjkosh...@gmail.com> wrote:
> >
> > > > The issue with a separate ser/deser library is that if it's not part
> of
> > > the
> > > > client API, (1) users may not use it or (2) different users may use
> it in
> > > > different ways. For example, you can imagine that two Avro
> > > implementations
> > > > have different ways of instantiation (since it's not enforced by the
> > > client
> > > > API). This makes sharing such kind of libraries harder.
> > >
> > > That is true - but that is also the point I think and it seems
> > > irrelevant to whether it is built-in to the producer's config or
> > > plugged in outside at the application-level. i.e., users will not use
> > > a common implementation if it does not fit their requirements. If a
> > > well-designed, full-featured and correctly implemented avro-or-other
> > > serializer/deserializer is made available there is no reason why that
> > > cannot be shared by different applications.
> > >
> > > > As for reason about the data types, take an example of the consumer
> > > > application. It needs to deal with objects at some point. So the
> earlier
> > > > that type information is revealed, the clearer it is to the
> application.
> > >
> > > Again for this, the only additional step is a call to deserialize. At
> > > some level the application _has_ to deal with the specific data type
> > > and it is thus reasonable to require that a consumed byte array needs
> > > to be deserialized to that type before being used.
> > >
> > > I suppose I don't see much benefit in pushing this into the core API
> > > of the producer at the expense of making these changes to the API.  At
> > > the same time, I should be clear that I don't think the proposal is in
> > > any way unreasonable which is why I'm definitely not opposed to it,
> > > but I'm also not convinced that it is necessary.
> > >
> > > Thanks,
> > >
> > > Joel
> > >
> > > >
> > > > On Tue, Dec 2, 2014 at 10:06 AM, Joel Koshy <jjkosh...@gmail.com>
> wrote:
> > > >
> > > > > Re: pushing complexity of dealing with objects: we're talking about
> > > > > just a call to a serialize method to convert the object to a byte
> > > > > array right? Or is there more to it? (To me) that seems less
> > > > > cumbersome than having to interact with parameterized types.
> Actually,
> > > > > can you explain more clearly what you mean by <q>reason about what
> > > > > type of data is being sent</q> in your original email? I have some
> > > > > notion of what that means but it is a bit vague and you might have
> > > > > meant something else.
> > > > >
> > > > > Thanks,
> > > > >
> > > > > Joel
> > > > >
> > > > > On Tue, Dec 02, 2014 at 09:15:19AM -0800, Jun Rao wrote:
> > > > > > Joel,
> > > > > >
> > > > > > Thanks for the feedback.
> > > > > >
> > > > > > Yes, the raw bytes interface is simpler than the Generic api.
> > > However, it
> > > > > > just pushes the complexity of dealing with the objects to the
> > > > > application.
> > > > > > We also thought about the layered approach. However, this may
> > > confuse the
> > > > > > users since there is no single entry point and it's not clear
> which
> > > > > layer a
> > > > > > user should be using.
> > > > > >
> > > > > > Jun
> > > > > >
> > > > > >
> > > > > > On Tue, Dec 2, 2014 at 12:34 AM, Joel Koshy <jjkosh...@gmail.com
> >
> > > wrote:
> > > > > >
> > > > > > > > makes it hard to reason about what type of data is being
> sent to
> > > > > Kafka
> > > > > > > and
> > > > > > > > also makes it hard to share an implementation of the
> serializer.
> > > For
> > > > > > > > example, to support Avro, the serialization logic could be
> quite
> > > > > involved
> > > > > > > > since it might need to register the Avro schema in some
> remote
> > > > > registry
> > > > > > > and
> > > > > > > > maintain a schema cache locally, etc. Without a serialization
> > > api,
> > > > > it's
> > > > > > > > impossible to share such an implementation so that people can
> > > easily
> > > > > > > reuse.
> > > > > > > > We sort of overlooked this implication during the initial
> > > discussion
> > > > > of
> > > > > > > the
> > > > > > > > producer api.
> > > > > > >
> > > > > > > Thanks for bringing this up and the patch.  My take on this is
> that
> > > > > > > any reasoning about the data itself is more appropriately
> handled
> > > > > > > outside of the core producer API. FWIW, I don't think this was
> > > > > > > _overlooked_ during the initial discussion of the producer API
> > > > > > > (especially since it was a significant change from the old
> > > producer).
> > > > > > > IIRC we believed at the time that there is elegance and
> > > flexibility in
> > > > > > > a simple API that deals with raw bytes. I think it is more
> > > accurate to
> > > > > > > say that this is a reversal of opinion for some (which is
> fine) but
> > > > > > > personally I'm still in the old camp :) i.e., I really like the
> > > > > > > simplicity of the current 0.8.2 producer API and find
> parameterized
> > > > > > > types/generics to be distracting and annoying; and IMO any
> > > > > > > data-specific handling is better absorbed at a higher-level
> than
> > > the
> > > > > > > core Kafka APIs - possibly by a (very thin) wrapper producer
> > > library.
> > > > > > > I don't quite see why it is difficult to share different
> wrapper
> > > > > > > implementations; or even ser-de libraries for that matter that
> > > people
> > > > > > > can invoke before sending to/reading from Kafka.
> > > > > > >
> > > > > > > That said I'm not opposed to the change - it's just that I
> prefer
> > > > > > > what's currently there. So I'm +0 on the proposal.
> > > > > > >
> > > > > > > Thanks,
> > > > > > >
> > > > > > > Joel
> > > > > > >
> > > > > > > On Mon, Nov 24, 2014 at 05:58:50PM -0800, Jun Rao wrote:
> > > > > > > > Hi, Everyone,
> > > > > > > >
> > > > > > > > I'd like to start a discussion on whether it makes sense to
> add
> > > the
> > > > > > > > serializer api back to the new java producer. Currently, the
> new
> > > java
> > > > > > > > producer takes a byte array for both the key and the value.
> While
> > > > > this
> > > > > > > api
> > > > > > > > is simple, it pushes the serialization logic into the
> > > application.
> > > > > This
> > > > > > > > makes it hard to reason about what type of data is being
> sent to
> > > > > Kafka
> > > > > > > and
> > > > > > > > also makes it hard to share an implementation of the
> serializer.
> > > For
> > > > > > > > example, to support Avro, the serialization logic could be
> quite
> > > > > involved
> > > > > > > > since it might need to register the Avro schema in some
> remote
> > > > > registry
> > > > > > > and
> > > > > > > > maintain a schema cache locally, etc. Without a serialization
> > > api,
> > > > > it's
> > > > > > > > impossible to share such an implementation so that people can
> > > easily
> > > > > > > reuse.
> > > > > > > > We sort of overlooked this implication during the initial
> > > discussion
> > > > > of
> > > > > > > the
> > > > > > > > producer api.
> > > > > > > >
> > > > > > > > So, I'd like to propose an api change to the new producer by
> > > adding
> > > > > back
> > > > > > > > the serializer api similar to what we had in the old
> producer.
> > > > > Specially,
> > > > > > > > the proposed api changes are the following.
> > > > > > > >
> > > > > > > > First, we change KafkaProducer to take generic types K and V
> for
> > > the
> > > > > key
> > > > > > > > and the value, respectively.
> > > > > > > >
> > > > > > > > public class KafkaProducer<K,V> implements Producer<K,V> {
> > > > > > > >
> > > > > > > >     public Future<RecordMetadata> send(ProducerRecord<K,V>
> > > record,
> > > > > > > Callback
> > > > > > > > callback);
> > > > > > > >
> > > > > > > >     public Future<RecordMetadata> send(ProducerRecord<K,V>
> > > record);
> > > > > > > > }
> > > > > > > >
> > > > > > > > Second, we add two new configs, one for the key serializer
> and
> > > > > another
> > > > > > > for
> > > > > > > > the value serializer. Both serializers will default to the
> byte
> > > array
> > > > > > > > implementation.
> > > > > > > >
> > > > > > > > public class ProducerConfig extends AbstractConfig {
> > > > > > > >
> > > > > > > >     .define(KEY_SERIALIZER_CLASS_CONFIG, Type.CLASS,
> > > > > > > > "org.apache.kafka.clients.producer.ByteArraySerializer",
> > > > > Importance.HIGH,
> > > > > > > > KEY_SERIALIZER_CLASS_DOC)
> > > > > > > >     .define(VALUE_SERIALIZER_CLASS_CONFIG, Type.CLASS,
> > > > > > > > "org.apache.kafka.clients.producer.ByteArraySerializer",
> > > > > Importance.HIGH,
> > > > > > > > VALUE_SERIALIZER_CLASS_DOC);
> > > > > > > > }
> > > > > > > >
> > > > > > > > Both serializers will implement the following interface.
> > > > > > > >
> > > > > > > > public interface Serializer<T> extends Configurable {
> > > > > > > >     public byte[] serialize(String topic, T data, boolean
> isKey);
> > > > > > > >
> > > > > > > >     public void close();
> > > > > > > > }
> > > > > > > >
> > > > > > > > This is more or less the same as what's in the old producer.
> The
> > > > > slight
> > > > > > > > differences are (1) the serializer now only requires a
> > > parameter-less
> > > > > > > > constructor; (2) the serializer has a configure() and a
> close()
> > > > > method
> > > > > > > for
> > > > > > > > initialization and cleanup, respectively; (3) the serialize()
> > > method
> > > > > > > > additionally takes the topic and an isKey indicator, both of
> > > which
> > > > > are
> > > > > > > > useful for things like schema registration.
> > > > > > > >
> > > > > > > > The detailed changes are included in KAFKA-1797. For
> > > completeness, I
> > > > > also
> > > > > > > > made the corresponding changes for the new java consumer api
> as
> > > well.
> > > > > > > >
> > > > > > > > Note that the proposed api changes are incompatible with
> what's
> > > in
> > > > > the
> > > > > > > > 0.8.2 branch. However, if those api changes are beneficial,
> it's
> > > > > probably
> > > > > > > > better to include them now in the 0.8.2 release, rather than
> > > later.
> > > > > > > >
> > > > > > > > I'd like to discuss mainly two things in this thread.
> > > > > > > > 1. Do people feel that the proposed api changes are
> reasonable?
> > > > > > > > 2. Are there any concerns of including the api changes in the
> > > 0.8.2
> > > > > final
> > > > > > > > release?
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > >
> > > > > > > > Jun
> > > > > > >
> > > > > > >
> > > > >
> > > > >
> > >
> > > --
> > > Joel
> > >
>
>

Reply via email to