You are viewing a plain text version of this content. The canonical link for it is here.
Posted to users@kafka.apache.org by Jun Rao <ju...@gmail.com> on 2014/11/25 02:58:50 UTC

[DISCUSSION] adding the serializer api back to the new java producer

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

Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Jonathan Weeks <jo...@gmail.com>.
+1 on this change — APIs are forever. As much as we’d love to see 0.8.2 release ASAP, it is important to get this right.

-JW

> On Nov 24, 2014, at 5:58 PM, Jun Rao <ju...@gmail.com> 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


Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Jay Kreps <ja...@gmail.com>.
I agree that having the new Producer<K, V>(KeySerializer<K>,
ValueSerializer<V>) interface would be useful.

People suggested cases where you want to mix and match serialization types.
The ByteArraySerializer is a no-op that would give the current behavior so
any odd case where you need to mix and match serialization or opt out
entirely are totally possible and won't have any overhead other than the
syntactic burden of declaring the parametric type <byte[], byte[]>. However
the expectation is that these cases are rare.

I really really think we should avoid having a second producer interface
like KafkaSerializedProducer. KafkaProducer<byte[], byte[]> will give the
serialization free behavior. I think our experience has been that surface
area really matters with these things so let's not have two. That sounds
like a compromise but is actually the worst of all worlds since it
duplicates everything over a fairly minor matter.

-Jay



On Thu, Dec 4, 2014 at 10:33 AM, Jiangjie Qin <jq...@linkedin.com.invalid>
wrote:

>
> I'm just thinking instead of binding serialization with producer, another
> option is to bind serializer/deserializer with
> ProducerRecord/ConsumerRecord (please see the detail proposal below.)
>            The arguments for this option is:
>         A. A single producer could send different message types. There are
> several use cases in LinkedIn for per record serializer
>         - In Samza, there are some in-stream order-sensitive control
> messages
> having different deserializer from other messages.
>         - There are use cases which need support for sending both Avro
> messages
> and raw bytes.
>         - Some use cases needs to deserialize some Avro messages into
> generic
> record and some other messages into specific record.
>         B. In current proposal, the serializer/deserilizer is instantiated
> according to config. Compared with that, binding serializer with
> ProducerRecord and ConsumerRecord is less error prone.
>
>
>         This option includes the following changes:
>         A. Add serializer and deserializer interfaces to replace serializer
> instance from config.
>                 Public interface Serializer <K, V> {
>                         public byte[] serializeKey(K key);
>                         public byte[] serializeValue(V value);
>                 }
>                 Public interface deserializer <K, V> {
>                         Public K deserializeKey(byte[] key);
>                         public V deserializeValue(byte[] value);
>                 }
>
>         B. Make ProducerRecord and ConsumerRecord abstract class
> implementing
> Serializer <K, V> and Deserializer <K, V> respectively.
>                 Public abstract class ProducerRecord <K, V> implements
> Serializer <K, V>
> {...}
>                 Public abstract class ConsumerRecord <K, V> implements
> Deserializer <K,
> V> {...}
>
>         C. Instead of instantiate the serializer/Deserializer from config,
> let
> concrete ProducerRecord/ConsumerRecord extends the abstract class and
> override the serialize/deserialize methods.
>
>                 Public class AvroProducerRecord extends ProducerRecord
> <String,
> GenericRecord> {
>                         ...
>                         @Override
>                         Public byte[] serializeKey(String key) {Š}
>                         @Override
>                         public byte[] serializeValue(GenericRecord value);
>                 }
>
>                 Public class AvroConsumerRecord extends ConsumerRecord
> <String,
> GenericRecord> {
>                         ...
>                         @Override
>                         Public K deserializeKey(byte[] key) {Š}
>                         @Override
>                         public V deserializeValue(byte[] value);
>                 }
>
>         D. The producer API changes to
>                 Public class KafkaProducer {
>                         ...
>
>                         Future<RecordMetadata> send (ProducerRecord <K, V>
> record) {
>                                 ...
>                                 K key = record.serializeKey(record.key);
>                                 V value =
> record.serializedValue(record.value);
>                                 BytesProducerRecord bytesProducerRecord =
> new
> BytesProducerRecord(topic, partition, key, value);
>                                 ...
>                         }
>                         ...
>                 }
>
>
>
> We also had some brainstorm in LinkedIn and here are the feedbacks:
>
> If the community decide to add the serialization back to new producer,
> besides current proposal which changes new producer API to be a template,
> there are some other options raised during our discussion:
>         1) Rather than change current new producer API, we can provide a
> wrapper
> of current new producer (e.g. KafkaSerializedProducer) and make it
> available to users. As there is value in the simplicity of current API.
>
>         2) If we decide to go with tempalated new producer API, according
> to
> experience in LinkedIn, it might worth considering to instantiate the
> serializer in code instead of from config so we can avoid runtime errors
> due to dynamic instantiation from config, which is more error prone. If
> that is the case, the producer API could be changed to something like:
>                 producer = new Producer<K, V>(KeySerializer<K>,
> ValueSerializer<V>)
>
> --Jiangjie (Becket) Qin
>
>
> On 11/24/14, 5:58 PM, "Jun Rao" <ju...@gmail.com> 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
>
>

Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Jay Kreps <ja...@gmail.com>.
I agree that having the new Producer<K, V>(KeySerializer<K>,
ValueSerializer<V>) interface would be useful.

People suggested cases where you want to mix and match serialization types.
The ByteArraySerializer is a no-op that would give the current behavior so
any odd case where you need to mix and match serialization or opt out
entirely are totally possible and won't have any overhead other than the
syntactic burden of declaring the parametric type <byte[], byte[]>. However
the expectation is that these cases are rare.

I really really think we should avoid having a second producer interface
like KafkaSerializedProducer. KafkaProducer<byte[], byte[]> will give the
serialization free behavior. I think our experience has been that surface
area really matters with these things so let's not have two. That sounds
like a compromise but is actually the worst of all worlds since it
duplicates everything over a fairly minor matter.

-Jay



On Thu, Dec 4, 2014 at 10:33 AM, Jiangjie Qin <jq...@linkedin.com.invalid>
wrote:

>
> I'm just thinking instead of binding serialization with producer, another
> option is to bind serializer/deserializer with
> ProducerRecord/ConsumerRecord (please see the detail proposal below.)
>            The arguments for this option is:
>         A. A single producer could send different message types. There are
> several use cases in LinkedIn for per record serializer
>         - In Samza, there are some in-stream order-sensitive control
> messages
> having different deserializer from other messages.
>         - There are use cases which need support for sending both Avro
> messages
> and raw bytes.
>         - Some use cases needs to deserialize some Avro messages into
> generic
> record and some other messages into specific record.
>         B. In current proposal, the serializer/deserilizer is instantiated
> according to config. Compared with that, binding serializer with
> ProducerRecord and ConsumerRecord is less error prone.
>
>
>         This option includes the following changes:
>         A. Add serializer and deserializer interfaces to replace serializer
> instance from config.
>                 Public interface Serializer <K, V> {
>                         public byte[] serializeKey(K key);
>                         public byte[] serializeValue(V value);
>                 }
>                 Public interface deserializer <K, V> {
>                         Public K deserializeKey(byte[] key);
>                         public V deserializeValue(byte[] value);
>                 }
>
>         B. Make ProducerRecord and ConsumerRecord abstract class
> implementing
> Serializer <K, V> and Deserializer <K, V> respectively.
>                 Public abstract class ProducerRecord <K, V> implements
> Serializer <K, V>
> {...}
>                 Public abstract class ConsumerRecord <K, V> implements
> Deserializer <K,
> V> {...}
>
>         C. Instead of instantiate the serializer/Deserializer from config,
> let
> concrete ProducerRecord/ConsumerRecord extends the abstract class and
> override the serialize/deserialize methods.
>
>                 Public class AvroProducerRecord extends ProducerRecord
> <String,
> GenericRecord> {
>                         ...
>                         @Override
>                         Public byte[] serializeKey(String key) {Š}
>                         @Override
>                         public byte[] serializeValue(GenericRecord value);
>                 }
>
>                 Public class AvroConsumerRecord extends ConsumerRecord
> <String,
> GenericRecord> {
>                         ...
>                         @Override
>                         Public K deserializeKey(byte[] key) {Š}
>                         @Override
>                         public V deserializeValue(byte[] value);
>                 }
>
>         D. The producer API changes to
>                 Public class KafkaProducer {
>                         ...
>
>                         Future<RecordMetadata> send (ProducerRecord <K, V>
> record) {
>                                 ...
>                                 K key = record.serializeKey(record.key);
>                                 V value =
> record.serializedValue(record.value);
>                                 BytesProducerRecord bytesProducerRecord =
> new
> BytesProducerRecord(topic, partition, key, value);
>                                 ...
>                         }
>                         ...
>                 }
>
>
>
> We also had some brainstorm in LinkedIn and here are the feedbacks:
>
> If the community decide to add the serialization back to new producer,
> besides current proposal which changes new producer API to be a template,
> there are some other options raised during our discussion:
>         1) Rather than change current new producer API, we can provide a
> wrapper
> of current new producer (e.g. KafkaSerializedProducer) and make it
> available to users. As there is value in the simplicity of current API.
>
>         2) If we decide to go with tempalated new producer API, according
> to
> experience in LinkedIn, it might worth considering to instantiate the
> serializer in code instead of from config so we can avoid runtime errors
> due to dynamic instantiation from config, which is more error prone. If
> that is the case, the producer API could be changed to something like:
>                 producer = new Producer<K, V>(KeySerializer<K>,
> ValueSerializer<V>)
>
> --Jiangjie (Becket) Qin
>
>
> On 11/24/14, 5:58 PM, "Jun Rao" <ju...@gmail.com> 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
>
>

Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Jun Rao <ju...@confluent.io>.
Ok, based on all the feedbacks that we have heard, I plan to do the
following.

1. Keep the generic api in KAFKA-1797.
2. Add a new constructor in Producer/Consumer that takes the key and the
value serializer instance.
3. Have KAFKA-1797 reviewed and checked into 0.8.2 and trunk.

This will make it easy for people to reuse common serializers while at the
same time allow people to use the byte array api if one chooses to do so.

I plan to make those changes in the next couple of days unless someone
strongly objects.

Thanks,

Jun


On Fri, Dec 5, 2014 at 5:46 PM, Jiangjie Qin <jq...@linkedin.com.invalid>
wrote:

> Hi Jun,
>
> Thanks for pointing out this. Yes, putting serialization/deserialization
> code into record does lose some flexibility. Some more thinking, I think
> no matter what we do to bind the producer and serializer/deserializer, we
> can always to the same thing on Record, i.e. We can also have some
> constructor like ProducerRecor<Serializer<K, V>, Deserializer<K, V>>. The
> downside of this is that we could potentially have a
> serializer/deserializer instance for each record (that's actually the very
> reason that I propose to put the code in record). This problem could be
> addressed by either using a singleton class or factory for
> serializer/deserializer library. But it might be a little bit complicated
> and we are not able to enforce that to external library either. So it
> seems only make sense if we really want to:
> 1. Have a single simple producer interface.
> AND
> 2. use a single producer send all type of messages
>
> I'm not sure if these requirement are strong enough to make us take the
> complexity of singleton/factory class serializer/deserializer library.
>
> Thanks.
>
> Jiangjie (Becket) Qin
>
> On 12/5/14, 3:16 PM, "Jun Rao" <ju...@confluent.io> wrote:
>
> >Jiangjie,
> >
> >The issue with adding the serializer in ProducerRecord is that you need to
> >implement all combinations of serializers for key and value. So, instead
> >of
> >just implementing int and string serializers, you will have to implement
> >all 4 combinations.
> >
> >Adding a new producer constructor like Producer<K, V>(KeySerializer<K>,
> >ValueSerializer<V>, Properties properties) can be useful.
> >
> >Thanks,
> >
> >Jun
> >
> >On Thu, Dec 4, 2014 at 10:33 AM, Jiangjie Qin <jq...@linkedin.com.invalid>
> >wrote:
> >
> >>
> >> I'm just thinking instead of binding serialization with producer,
> >>another
> >> option is to bind serializer/deserializer with
> >> ProducerRecord/ConsumerRecord (please see the detail proposal below.)
> >>            The arguments for this option is:
> >>         A. A single producer could send different message types. There
> >>are
> >> several use cases in LinkedIn for per record serializer
> >>         - In Samza, there are some in-stream order-sensitive control
> >> messages
> >> having different deserializer from other messages.
> >>         - There are use cases which need support for sending both Avro
> >> messages
> >> and raw bytes.
> >>         - Some use cases needs to deserialize some Avro messages into
> >> generic
> >> record and some other messages into specific record.
> >>         B. In current proposal, the serializer/deserilizer is
> >>instantiated
> >> according to config. Compared with that, binding serializer with
> >> ProducerRecord and ConsumerRecord is less error prone.
> >>
> >>
> >>         This option includes the following changes:
> >>         A. Add serializer and deserializer interfaces to replace
> >>serializer
> >> instance from config.
> >>                 Public interface Serializer <K, V> {
> >>                         public byte[] serializeKey(K key);
> >>                         public byte[] serializeValue(V value);
> >>                 }
> >>                 Public interface deserializer <K, V> {
> >>                         Public K deserializeKey(byte[] key);
> >>                         public V deserializeValue(byte[] value);
> >>                 }
> >>
> >>         B. Make ProducerRecord and ConsumerRecord abstract class
> >> implementing
> >> Serializer <K, V> and Deserializer <K, V> respectively.
> >>                 Public abstract class ProducerRecord <K, V> implements
> >> Serializer <K, V>
> >> {...}
> >>                 Public abstract class ConsumerRecord <K, V> implements
> >> Deserializer <K,
> >> V> {...}
> >>
> >>         C. Instead of instantiate the serializer/Deserializer from
> >>config,
> >> let
> >> concrete ProducerRecord/ConsumerRecord extends the abstract class and
> >> override the serialize/deserialize methods.
> >>
> >>                 Public class AvroProducerRecord extends ProducerRecord
> >> <String,
> >> GenericRecord> {
> >>                         ...
> >>                         @Override
> >>                         Public byte[] serializeKey(String key) {Š}
> >>                         @Override
> >>                         public byte[] serializeValue(GenericRecord
> >>value);
> >>                 }
> >>
> >>                 Public class AvroConsumerRecord extends ConsumerRecord
> >> <String,
> >> GenericRecord> {
> >>                         ...
> >>                         @Override
> >>                         Public K deserializeKey(byte[] key) {Š}
> >>                         @Override
> >>                         public V deserializeValue(byte[] value);
> >>                 }
> >>
> >>         D. The producer API changes to
> >>                 Public class KafkaProducer {
> >>                         ...
> >>
> >>                         Future<RecordMetadata> send (ProducerRecord <K,
> >>V>
> >> record) {
> >>                                 ...
> >>                                 K key = record.serializeKey(record.key);
> >>                                 V value =
> >> record.serializedValue(record.value);
> >>                                 BytesProducerRecord bytesProducerRecord
> >>=
> >> new
> >> BytesProducerRecord(topic, partition, key, value);
> >>                                 ...
> >>                         }
> >>                         ...
> >>                 }
> >>
> >>
> >>
> >> We also had some brainstorm in LinkedIn and here are the feedbacks:
> >>
> >> If the community decide to add the serialization back to new producer,
> >> besides current proposal which changes new producer API to be a
> >>template,
> >> there are some other options raised during our discussion:
> >>         1) Rather than change current new producer API, we can provide a
> >> wrapper
> >> of current new producer (e.g. KafkaSerializedProducer) and make it
> >> available to users. As there is value in the simplicity of current API.
> >>
> >>         2) If we decide to go with tempalated new producer API,
> >>according
> >> to
> >> experience in LinkedIn, it might worth considering to instantiate the
> >> serializer in code instead of from config so we can avoid runtime errors
> >> due to dynamic instantiation from config, which is more error prone. If
> >> that is the case, the producer API could be changed to something like:
> >>                 producer = new Producer<K, V>(KeySerializer<K>,
> >> ValueSerializer<V>)
> >>
> >> --Jiangjie (Becket) Qin
> >>
> >>
> >> On 11/24/14, 5:58 PM, "Jun Rao" <ju...@gmail.com> 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
> >>
> >>
>
>

Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Jun Rao <ju...@confluent.io>.
Ok, based on all the feedbacks that we have heard, I plan to do the
following.

1. Keep the generic api in KAFKA-1797.
2. Add a new constructor in Producer/Consumer that takes the key and the
value serializer instance.
3. Have KAFKA-1797 reviewed and checked into 0.8.2 and trunk.

This will make it easy for people to reuse common serializers while at the
same time allow people to use the byte array api if one chooses to do so.

I plan to make those changes in the next couple of days unless someone
strongly objects.

Thanks,

Jun


On Fri, Dec 5, 2014 at 5:46 PM, Jiangjie Qin <jq...@linkedin.com.invalid>
wrote:

> Hi Jun,
>
> Thanks for pointing out this. Yes, putting serialization/deserialization
> code into record does lose some flexibility. Some more thinking, I think
> no matter what we do to bind the producer and serializer/deserializer, we
> can always to the same thing on Record, i.e. We can also have some
> constructor like ProducerRecor<Serializer<K, V>, Deserializer<K, V>>. The
> downside of this is that we could potentially have a
> serializer/deserializer instance for each record (that's actually the very
> reason that I propose to put the code in record). This problem could be
> addressed by either using a singleton class or factory for
> serializer/deserializer library. But it might be a little bit complicated
> and we are not able to enforce that to external library either. So it
> seems only make sense if we really want to:
> 1. Have a single simple producer interface.
> AND
> 2. use a single producer send all type of messages
>
> I'm not sure if these requirement are strong enough to make us take the
> complexity of singleton/factory class serializer/deserializer library.
>
> Thanks.
>
> Jiangjie (Becket) Qin
>
> On 12/5/14, 3:16 PM, "Jun Rao" <ju...@confluent.io> wrote:
>
> >Jiangjie,
> >
> >The issue with adding the serializer in ProducerRecord is that you need to
> >implement all combinations of serializers for key and value. So, instead
> >of
> >just implementing int and string serializers, you will have to implement
> >all 4 combinations.
> >
> >Adding a new producer constructor like Producer<K, V>(KeySerializer<K>,
> >ValueSerializer<V>, Properties properties) can be useful.
> >
> >Thanks,
> >
> >Jun
> >
> >On Thu, Dec 4, 2014 at 10:33 AM, Jiangjie Qin <jq...@linkedin.com.invalid>
> >wrote:
> >
> >>
> >> I'm just thinking instead of binding serialization with producer,
> >>another
> >> option is to bind serializer/deserializer with
> >> ProducerRecord/ConsumerRecord (please see the detail proposal below.)
> >>            The arguments for this option is:
> >>         A. A single producer could send different message types. There
> >>are
> >> several use cases in LinkedIn for per record serializer
> >>         - In Samza, there are some in-stream order-sensitive control
> >> messages
> >> having different deserializer from other messages.
> >>         - There are use cases which need support for sending both Avro
> >> messages
> >> and raw bytes.
> >>         - Some use cases needs to deserialize some Avro messages into
> >> generic
> >> record and some other messages into specific record.
> >>         B. In current proposal, the serializer/deserilizer is
> >>instantiated
> >> according to config. Compared with that, binding serializer with
> >> ProducerRecord and ConsumerRecord is less error prone.
> >>
> >>
> >>         This option includes the following changes:
> >>         A. Add serializer and deserializer interfaces to replace
> >>serializer
> >> instance from config.
> >>                 Public interface Serializer <K, V> {
> >>                         public byte[] serializeKey(K key);
> >>                         public byte[] serializeValue(V value);
> >>                 }
> >>                 Public interface deserializer <K, V> {
> >>                         Public K deserializeKey(byte[] key);
> >>                         public V deserializeValue(byte[] value);
> >>                 }
> >>
> >>         B. Make ProducerRecord and ConsumerRecord abstract class
> >> implementing
> >> Serializer <K, V> and Deserializer <K, V> respectively.
> >>                 Public abstract class ProducerRecord <K, V> implements
> >> Serializer <K, V>
> >> {...}
> >>                 Public abstract class ConsumerRecord <K, V> implements
> >> Deserializer <K,
> >> V> {...}
> >>
> >>         C. Instead of instantiate the serializer/Deserializer from
> >>config,
> >> let
> >> concrete ProducerRecord/ConsumerRecord extends the abstract class and
> >> override the serialize/deserialize methods.
> >>
> >>                 Public class AvroProducerRecord extends ProducerRecord
> >> <String,
> >> GenericRecord> {
> >>                         ...
> >>                         @Override
> >>                         Public byte[] serializeKey(String key) {Š}
> >>                         @Override
> >>                         public byte[] serializeValue(GenericRecord
> >>value);
> >>                 }
> >>
> >>                 Public class AvroConsumerRecord extends ConsumerRecord
> >> <String,
> >> GenericRecord> {
> >>                         ...
> >>                         @Override
> >>                         Public K deserializeKey(byte[] key) {Š}
> >>                         @Override
> >>                         public V deserializeValue(byte[] value);
> >>                 }
> >>
> >>         D. The producer API changes to
> >>                 Public class KafkaProducer {
> >>                         ...
> >>
> >>                         Future<RecordMetadata> send (ProducerRecord <K,
> >>V>
> >> record) {
> >>                                 ...
> >>                                 K key = record.serializeKey(record.key);
> >>                                 V value =
> >> record.serializedValue(record.value);
> >>                                 BytesProducerRecord bytesProducerRecord
> >>=
> >> new
> >> BytesProducerRecord(topic, partition, key, value);
> >>                                 ...
> >>                         }
> >>                         ...
> >>                 }
> >>
> >>
> >>
> >> We also had some brainstorm in LinkedIn and here are the feedbacks:
> >>
> >> If the community decide to add the serialization back to new producer,
> >> besides current proposal which changes new producer API to be a
> >>template,
> >> there are some other options raised during our discussion:
> >>         1) Rather than change current new producer API, we can provide a
> >> wrapper
> >> of current new producer (e.g. KafkaSerializedProducer) and make it
> >> available to users. As there is value in the simplicity of current API.
> >>
> >>         2) If we decide to go with tempalated new producer API,
> >>according
> >> to
> >> experience in LinkedIn, it might worth considering to instantiate the
> >> serializer in code instead of from config so we can avoid runtime errors
> >> due to dynamic instantiation from config, which is more error prone. If
> >> that is the case, the producer API could be changed to something like:
> >>                 producer = new Producer<K, V>(KeySerializer<K>,
> >> ValueSerializer<V>)
> >>
> >> --Jiangjie (Becket) Qin
> >>
> >>
> >> On 11/24/14, 5:58 PM, "Jun Rao" <ju...@gmail.com> 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
> >>
> >>
>
>

Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Jiangjie Qin <jq...@linkedin.com.INVALID>.
Hi Jun,

Thanks for pointing out this. Yes, putting serialization/deserialization
code into record does lose some flexibility. Some more thinking, I think
no matter what we do to bind the producer and serializer/deserializer, we
can always to the same thing on Record, i.e. We can also have some
constructor like ProducerRecor<Serializer<K, V>, Deserializer<K, V>>. The
downside of this is that we could potentially have a
serializer/deserializer instance for each record (that's actually the very
reason that I propose to put the code in record). This problem could be
addressed by either using a singleton class or factory for
serializer/deserializer library. But it might be a little bit complicated
and we are not able to enforce that to external library either. So it
seems only make sense if we really want to:
1. Have a single simple producer interface.
AND
2. use a single producer send all type of messages

I'm not sure if these requirement are strong enough to make us take the
complexity of singleton/factory class serializer/deserializer library.

Thanks.

Jiangjie (Becket) Qin

On 12/5/14, 3:16 PM, "Jun Rao" <ju...@confluent.io> wrote:

>Jiangjie,
>
>The issue with adding the serializer in ProducerRecord is that you need to
>implement all combinations of serializers for key and value. So, instead
>of
>just implementing int and string serializers, you will have to implement
>all 4 combinations.
>
>Adding a new producer constructor like Producer<K, V>(KeySerializer<K>,
>ValueSerializer<V>, Properties properties) can be useful.
>
>Thanks,
>
>Jun
>
>On Thu, Dec 4, 2014 at 10:33 AM, Jiangjie Qin <jq...@linkedin.com.invalid>
>wrote:
>
>>
>> I'm just thinking instead of binding serialization with producer,
>>another
>> option is to bind serializer/deserializer with
>> ProducerRecord/ConsumerRecord (please see the detail proposal below.)
>>            The arguments for this option is:
>>         A. A single producer could send different message types. There
>>are
>> several use cases in LinkedIn for per record serializer
>>         - In Samza, there are some in-stream order-sensitive control
>> messages
>> having different deserializer from other messages.
>>         - There are use cases which need support for sending both Avro
>> messages
>> and raw bytes.
>>         - Some use cases needs to deserialize some Avro messages into
>> generic
>> record and some other messages into specific record.
>>         B. In current proposal, the serializer/deserilizer is
>>instantiated
>> according to config. Compared with that, binding serializer with
>> ProducerRecord and ConsumerRecord is less error prone.
>>
>>
>>         This option includes the following changes:
>>         A. Add serializer and deserializer interfaces to replace
>>serializer
>> instance from config.
>>                 Public interface Serializer <K, V> {
>>                         public byte[] serializeKey(K key);
>>                         public byte[] serializeValue(V value);
>>                 }
>>                 Public interface deserializer <K, V> {
>>                         Public K deserializeKey(byte[] key);
>>                         public V deserializeValue(byte[] value);
>>                 }
>>
>>         B. Make ProducerRecord and ConsumerRecord abstract class
>> implementing
>> Serializer <K, V> and Deserializer <K, V> respectively.
>>                 Public abstract class ProducerRecord <K, V> implements
>> Serializer <K, V>
>> {...}
>>                 Public abstract class ConsumerRecord <K, V> implements
>> Deserializer <K,
>> V> {...}
>>
>>         C. Instead of instantiate the serializer/Deserializer from
>>config,
>> let
>> concrete ProducerRecord/ConsumerRecord extends the abstract class and
>> override the serialize/deserialize methods.
>>
>>                 Public class AvroProducerRecord extends ProducerRecord
>> <String,
>> GenericRecord> {
>>                         ...
>>                         @Override
>>                         Public byte[] serializeKey(String key) {Š}
>>                         @Override
>>                         public byte[] serializeValue(GenericRecord
>>value);
>>                 }
>>
>>                 Public class AvroConsumerRecord extends ConsumerRecord
>> <String,
>> GenericRecord> {
>>                         ...
>>                         @Override
>>                         Public K deserializeKey(byte[] key) {Š}
>>                         @Override
>>                         public V deserializeValue(byte[] value);
>>                 }
>>
>>         D. The producer API changes to
>>                 Public class KafkaProducer {
>>                         ...
>>
>>                         Future<RecordMetadata> send (ProducerRecord <K,
>>V>
>> record) {
>>                                 ...
>>                                 K key = record.serializeKey(record.key);
>>                                 V value =
>> record.serializedValue(record.value);
>>                                 BytesProducerRecord bytesProducerRecord
>>=
>> new
>> BytesProducerRecord(topic, partition, key, value);
>>                                 ...
>>                         }
>>                         ...
>>                 }
>>
>>
>>
>> We also had some brainstorm in LinkedIn and here are the feedbacks:
>>
>> If the community decide to add the serialization back to new producer,
>> besides current proposal which changes new producer API to be a
>>template,
>> there are some other options raised during our discussion:
>>         1) Rather than change current new producer API, we can provide a
>> wrapper
>> of current new producer (e.g. KafkaSerializedProducer) and make it
>> available to users. As there is value in the simplicity of current API.
>>
>>         2) If we decide to go with tempalated new producer API,
>>according
>> to
>> experience in LinkedIn, it might worth considering to instantiate the
>> serializer in code instead of from config so we can avoid runtime errors
>> due to dynamic instantiation from config, which is more error prone. If
>> that is the case, the producer API could be changed to something like:
>>                 producer = new Producer<K, V>(KeySerializer<K>,
>> ValueSerializer<V>)
>>
>> --Jiangjie (Becket) Qin
>>
>>
>> On 11/24/14, 5:58 PM, "Jun Rao" <ju...@gmail.com> 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
>>
>>


Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Jun Rao <ju...@confluent.io>.
We still have a few blockers to fix in 0.8.2. When that's done, we can
discuss whether to do another 0.8.2 beta or just do the 0.8.2 final release.

Thanks,

Jun

On Wed, Dec 17, 2014 at 5:29 PM, Shannon Lloyd <sh...@gmail.com> wrote:
>
> Are you guys planning another beta for everyone to try out the changes
> before you cut 0.8.2 final?
>
> Cheers,
> Shannon
>
> On 18 December 2014 at 11:24, Rajiv Kurian <ra...@signalfuse.com> wrote:
> >
> > Has the mvn repo been updated too?
> >
> > On Wed, Dec 17, 2014 at 4:31 PM, Jun Rao <ju...@confluent.io> wrote:
> > >
> > > Thanks everyone for the feedback and the discussion. The proposed
> changes
> > > have been checked into both 0.8.2 and trunk.
> > >
> > > Jun
> > >
> > > On Tue, Dec 16, 2014 at 10:43 PM, Joel Koshy <jj...@gmail.com>
> > wrote:
> > > >
> > > > Jun,
> > > >
> > > > Thanks for summarizing this - it helps confirm for me that I did not
> > > > misunderstand anything in this thread so far; and that I disagree
> with
> > > > the premise that the steps in using the current byte-oriented API is
> > > > cumbersome or inflexible. It involves instantiating the K-V
> > > > serializers in code (as opposed to config) and a extra (but explicit
> > > > - i.e., making it very clear to the user) but simple call to
> serialize
> > > > before sending.
> > > >
> > > > The point about downstream queries breaking can happen just as well
> > > > with the implicit serializers/deserializers - since ultimately people
> > > > have to instantiate the specific type in their code and if they want
> > > > to send it they will.
> > > >
> > > > I think adoption is also equivalent since people will just
> instantiate
> > > > whatever serializer/deserializer they want in one line. Plugging in a
> > > > new serializer implementation does require a code change, but that
> can
> > > > also be avoided via a config driven factory.
> > > >
> > > > So I'm still +0 on the change but I'm definitely not against moving
> > > > forward with the changes. i.e., unless there is any strong -1 on the
> > > > proposal from anyone else.
> > > >
> > > > Thanks,
> > > >
> > > > Joel
> > > >
> > > > > With a byte array interface, of course there is nothing that one
> > can't
> > > > do.
> > > > > However, the real question is that whether we want to encourage
> > people
> > > to
> > > > > use it this way or not. Being able to flow just bytes is definitely
> > > > easier
> > > > > to get started. That's why many early adopters choose to do it that
> > > way.
> > > > > However, it's often the case that they start feeling the pain later
> > > when
> > > > > some producers change the data format. Their Hive/Pig queries start
> > to
> > > > > break and it's a painful process to have the issue fixed. So, the
> > > purpose
> > > > > of this api change is really to encourage people to standardize on
> a
> > > > single
> > > > > serializer/deserializer that supports things like data validation
> and
> > > > > schema evolution upstream in the producer. Now, suppose there is an
> > > Avro
> > > > > serializer/deserializer implementation. How do we make it easy for
> > > people
> > > > > to adopt? If the serializer is part of the api, we can just say,
> wire
> > > in
> > > > > the Avro serializer for key and/or value in the config and then you
> > can
> > > > > start sending Avro records to the producer. If the serializer is
> not
> > > part
> > > > > of the api, we have to say, first instantiate a key and/or value
> > > > serializer
> > > > > this way, send the key to the key serializer to get the key bytes,
> > send
> > > > the
> > > > > value to the value serializer to get the value bytes, and finally
> > send
> > > > the
> > > > > bytes to the producer. The former will be simpler and likely makes
> > the
> > > > > adoption easier.
> > > > >
> > > > > Thanks,
> > > > >
> > > > > Jun
> > > > >
> > > > > On Mon, Dec 15, 2014 at 7:20 PM, Joel Koshy <jj...@gmail.com>
> > > wrote:
> > > > > >
> > > > > > Documentation is inevitable even if the serializer/deserializer
> is
> > > > > > part of the API - since the user has to set it up in the configs.
> > So
> > > > > > again, you can only encourage people to use it through
> > documentation.
> > > > > > The simpler byte-oriented API seems clearer to me because anyone
> > who
> > > > > > needs to send (or receive) a specific data type will _be forced
> to_
> > > > > > (or actually, _intuitively_) select a serializer (or
> deserializer)
> > > and
> > > > > > will definitely pick an already available implementation if a
> good
> > > one
> > > > > > already exists.
> > > > > >
> > > > > > Sorry I still don't get it and this is really the only sticking
> > point
> > > > > > for me, albeit a minor one (which is why I have been +0 all along
> > on
> > > > > > the change). I (and I think many others) would appreciate it if
> > > > > > someone can help me understand this better.  So I will repeat the
> > > > > > question: What "usage pattern" cannot be supported by easily by
> the
> > > > > > simpler API without adding burden on the user?
> > > > > >
> > > > > > Thanks,
> > > > > >
> > > > > > Joel
> > > > > >
> > > > > > On Mon, Dec 15, 2014 at 11:59:34AM -0800, Jun Rao wrote:
> > > > > > > Joel,
> > > > > > >
> > > > > > > It's just that if the serializer/deserializer is not part of
> the
> > > > API, you
> > > > > > > can only encourage people to use it through documentation.
> > However,
> > > > not
> > > > > > > everyone will read the documentation if it's not directly used
> in
> > > the
> > > > > > API.
> > > > > > >
> > > > > > > Thanks,
> > > > > > >
> > > > > > > Jun
> > > > > > >
> > > > > > > On Mon, Dec 15, 2014 at 2:11 AM, Joel Koshy <
> jjkoshy.w@gmail.com
> > >
> > > > wrote:
> > > > > > >
> > > > > > > > (sorry about the late follow-up late - I'm traveling most of
> > this
> > > > > > > > month)
> > > > > > > >
> > > > > > > > I'm likely missing something obvious, but I find the
> following
> > to
> > > > be a
> > > > > > > > somewhat vague point that has been mentioned more than once
> in
> > > this
> > > > > > > > thread without a clear explanation. i.e., why is it hard to
> > > share a
> > > > > > > > serializer/deserializer implementation and just have the
> > clients
> > > > call
> > > > > > > > it before a send/receive? What "usage pattern" cannot be
> > > supported
> > > > by
> > > > > > > > the simpler API?
> > > > > > > >
> > > > > > > > > 1. Can we keep the serialization semantics outside the
> > Producer
> > > > > > interface
> > > > > > > > > and have simple bytes in / bytes out for the interface
> (This
> > is
> > > > what
> > > > > > we
> > > > > > > > > have today).
> > > > > > > > >
> > > > > > > > > The points for this is to keep the interface simple and
> usage
> > > > easy to
> > > > > > > > > understand. The points against this is that it gets hard to
> > > share
> > > > > > common
> > > > > > > > > usage patterns around serialization/message validations for
> > the
> > > > > > future.
> > > > > > > >
> > > > > > > >
> > > > > > > > On Tue, Dec 09, 2014 at 03:51:08AM +0000, Sriram Subramanian
> > > wrote:
> > > > > > > > > Thank you Jay. I agree with the issue that you point w.r.t
> > > paired
> > > > > > > > > serializers. I also think having mix serialization types is
> > > > rare. To
> > > > > > get
> > > > > > > > > the current behavior, one can simply use a
> > ByteArraySerializer.
> > > > This
> > > > > > is
> > > > > > > > > best understood by talking with many customers and you seem
> > to
> > > > have
> > > > > > done
> > > > > > > > > that. I am convinced about the change.
> > > > > > > > >
> > > > > > > > > For the rest who gave -1 or 0 for this proposal, does the
> > > answers
> > > > > > for the
> > > > > > > > > three points(updated) below seem reasonable? Are these
> > > > explanations
> > > > > > > > > convincing?
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > 1. Can we keep the serialization semantics outside the
> > Producer
> > > > > > interface
> > > > > > > > > and have simple bytes in / bytes out for the interface
> (This
> > is
> > > > what
> > > > > > we
> > > > > > > > > have today).
> > > > > > > > >
> > > > > > > > > The points for this is to keep the interface simple and
> usage
> > > > easy to
> > > > > > > > > understand. The points against this is that it gets hard to
> > > share
> > > > > > common
> > > > > > > > > usage patterns around serialization/message validations for
> > the
> > > > > > future.
> > > > > > > > >
> > > > > > > > > 2. Can we create a wrapper producer that does the
> > serialization
> > > > and
> > > > > > have
> > > > > > > > > different variants of it for different data formats?
> > > > > > > > >
> > > > > > > > > The points for this is again to keep the main API clean.
> The
> > > > points
> > > > > > > > > against this is that it duplicates the API, increases the
> > > surface
> > > > > > area
> > > > > > > > and
> > > > > > > > > creates redundancy for a minor addition.
> > > > > > > > >
> > > > > > > > > 3. Do we need to support different data types per record?
> The
> > > > current
> > > > > > > > > interface (bytes in/bytes out) lets you instantiate one
> > > producer
> > > > and
> > > > > > use
> > > > > > > > > it to send multiple data formats. There seems to be some
> > valid
> > > > use
> > > > > > cases
> > > > > > > > > for this.
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > Mixed serialization types are rare based on interactions
> with
> > > > > > customers.
> > > > > > > > > To get the current behavior, one can simply use a
> > > > > > ByteArraySerializer.
> > > > > > > > >
> > > > > > > > > On 12/5/14 5:00 PM, "Jay Kreps" <ja...@confluent.io> wrote:
> > > > > > > > >
> > > > > > > > > >Hey Sriram,
> > > > > > > > > >
> > > > > > > > > >Thanks! I think this is a very helpful summary.
> > > > > > > > > >
> > > > > > > > > >Let me try to address your point about passing in the
> serde
> > at
> > > > send
> > > > > > > > time.
> > > > > > > > > >
> > > > > > > > > >I think the first objection is really to the paired
> > key/value
> > > > > > serializer
> > > > > > > > > >interfaces. This leads to kind of a weird combinatorial
> > thing
> > > > where
> > > > > > you
> > > > > > > > > >would have an avro/avro serializer a string/avro
> > serializer, a
> > > > pb/pb
> > > > > > > > > >serializer, and a string/pb serializer, and so on. But
> your
> > > > proposal
> > > > > > > > would
> > > > > > > > > >work as well with separate serializers for key and value.
> > > > > > > > > >
> > > > > > > > > >I think the downside is just the one you call out--that
> this
> > > is
> > > > a
> > > > > > corner
> > > > > > > > > >case and you end up with two versions of all the apis to
> > > > support it.
> > > > > > > > This
> > > > > > > > > >also makes the serializer api more annoying to implement.
> I
> > > > think
> > > > > > the
> > > > > > > > > >alternative solution to this case and any other we can
> give
> > > > people
> > > > > > is
> > > > > > > > just
> > > > > > > > > >configuring ByteArraySerializer which gives you basically
> > the
> > > > api
> > > > > > that
> > > > > > > > you
> > > > > > > > > >have now with byte arrays. If this is incredibly common
> then
> > > > this
> > > > > > would
> > > > > > > > be
> > > > > > > > > >a silly solution, but I guess the belief is that these
> cases
> > > are
> > > > > > rare
> > > > > > > > and
> > > > > > > > > >a
> > > > > > > > > >really well implemented avro or json serializer should be
> > 100%
> > > > of
> > > > > > what
> > > > > > > > > >most
> > > > > > > > > >people need.
> > > > > > > > > >
> > > > > > > > > >In practice the cases that actually mix serialization
> types
> > > in a
> > > > > > single
> > > > > > > > > >stream are pretty rare I think just because the consumer
> > then
> > > > has
> > > > > > the
> > > > > > > > > >problem of guessing how to deserialize, so most of these
> > will
> > > > end up
> > > > > > > > with
> > > > > > > > > >at least some marker or schema id or whatever that tells
> you
> > > > how to
> > > > > > read
> > > > > > > > > >the data. Arguable this mixed serialization with marker is
> > > > itself a
> > > > > > > > > >serializer type and should have a serializer of its own...
> > > > > > > > > >
> > > > > > > > > >-Jay
> > > > > > > > > >
> > > > > > > > > >On Fri, Dec 5, 2014 at 3:48 PM, Sriram Subramanian <
> > > > > > > > > >srsubramanian@linkedin.com.invalid> wrote:
> > > > > > > > > >
> > > > > > > > > >> This thread has diverged multiple times now and it would
> > be
> > > > worth
> > > > > > > > > >> summarizing them.
> > > > > > > > > >>
> > > > > > > > > >> There seems to be the following points of discussion -
> > > > > > > > > >>
> > > > > > > > > >> 1. Can we keep the serialization semantics outside the
> > > > Producer
> > > > > > > > > >>interface
> > > > > > > > > >> and have simple bytes in / bytes out for the interface
> > (This
> > > > is
> > > > > > what
> > > > > > > > we
> > > > > > > > > >> have today).
> > > > > > > > > >>
> > > > > > > > > >> The points for this is to keep the interface simple and
> > > usage
> > > > > > easy to
> > > > > > > > > >> understand. The points against this is that it gets hard
> > to
> > > > share
> > > > > > > > common
> > > > > > > > > >> usage patterns around serialization/message validations
> > for
> > > > the
> > > > > > > > future.
> > > > > > > > > >>
> > > > > > > > > >> 2. Can we create a wrapper producer that does the
> > > > serialization
> > > > > > and
> > > > > > > > have
> > > > > > > > > >> different variants of it for different data formats?
> > > > > > > > > >>
> > > > > > > > > >> The points for this is again to keep the main API clean.
> > The
> > > > > > points
> > > > > > > > > >> against this is that it duplicates the API, increases
> the
> > > > surface
> > > > > > area
> > > > > > > > > >>and
> > > > > > > > > >> creates redundancy for a minor addition.
> > > > > > > > > >>
> > > > > > > > > >> 3. Do we need to support different data types per
> record?
> > > The
> > > > > > current
> > > > > > > > > >> interface (bytes in/bytes out) lets you instantiate one
> > > > producer
> > > > > > and
> > > > > > > > use
> > > > > > > > > >> it to send multiple data formats. There seems to be some
> > > > valid use
> > > > > > > > cases
> > > > > > > > > >> for this.
> > > > > > > > > >>
> > > > > > > > > >> I have still not seen a strong argument against not
> having
> > > > this
> > > > > > > > > >> functionality. Can someone provide their views on why we
> > > don't
> > > > > > need
> > > > > > > > this
> > > > > > > > > >> support that is possible with the current API?
> > > > > > > > > >>
> > > > > > > > > >> One possible approach for the per record serialization
> > would
> > > > be to
> > > > > > > > > >>define
> > > > > > > > > >>
> > > > > > > > > >> public interface SerDe<K,V> {
> > > > > > > > > >>   public byte[] serializeKey();
> > > > > > > > > >>
> > > > > > > > > >>   public K deserializeKey();
> > > > > > > > > >>
> > > > > > > > > >>   public byte[] serializeValue();
> > > > > > > > > >>
> > > > > > > > > >>   public V deserializeValue();
> > > > > > > > > >> }
> > > > > > > > > >>
> > > > > > > > > >> This would be used by both the Producer and the
> Consumer.
> > > > > > > > > >>
> > > > > > > > > >> The send APIs can then be
> > > > > > > > > >>
> > > > > > > > > >> public Future<RecordMetadata> send(ProducerRecord<K,V>
> > > > record);
> > > > > > > > > >> public Future<RecordMetadata> send(ProducerRecord<K,V>
> > > record,
> > > > > > > > Callback
> > > > > > > > > >> callback);
> > > > > > > > > >>
> > > > > > > > > >>
> > > > > > > > > >> public Future<RecordMetadata> send(ProducerRecord<K,V>
> > > record,
> > > > > > > > > >>SerDe<K,V>
> > > > > > > > > >> serde);
> > > > > > > > > >>
> > > > > > > > > >> public Future<RecordMetadata> send(ProducerRecord<K,V>
> > > record,
> > > > > > > > > >>SerDe<K,V>
> > > > > > > > > >> serde, Callback callback);
> > > > > > > > > >>
> > > > > > > > > >>
> > > > > > > > > >> A default SerDe can be set in the config. The producer
> > would
> > > > use
> > > > > > the
> > > > > > > > > >> default from the config if the non-serde send APIs are
> > used.
> > > > The
> > > > > > > > > >>downside
> > > > > > > > > >> to this approach is that we would need to have four
> > variants
> > > > of
> > > > > > Send
> > > > > > > > API
> > > > > > > > > >> for the Producer.
> > > > > > > > > >>
> > > > > > > > > >>
> > > > > > > > > >>
> > > > > > > > > >>
> > > > > > > > > >>
> > > > > > > > > >>
> > > > > > > > > >> On 12/5/14 3:16 PM, "Jun Rao" <ju...@confluent.io> wrote:
> > > > > > > > > >>
> > > > > > > > > >> >Jiangjie,
> > > > > > > > > >> >
> > > > > > > > > >> >The issue with adding the serializer in ProducerRecord
> is
> > > > that
> > > > > > you
> > > > > > > > > >>need to
> > > > > > > > > >> >implement all combinations of serializers for key and
> > > value.
> > > > So,
> > > > > > > > > >>instead
> > > > > > > > > >> >of
> > > > > > > > > >> >just implementing int and string serializers, you will
> > have
> > > > to
> > > > > > > > > >>implement
> > > > > > > > > >> >all 4 combinations.
> > > > > > > > > >> >
> > > > > > > > > >> >Adding a new producer constructor like Producer<K,
> > > > > > > > V>(KeySerializer<K>,
> > > > > > > > > >> >ValueSerializer<V>, Properties properties) can be
> useful.
> > > > > > > > > >> >
> > > > > > > > > >> >Thanks,
> > > > > > > > > >> >
> > > > > > > > > >> >Jun
> > > > > > > > > >> >
> > > > > > > > > >> >On Thu, Dec 4, 2014 at 10:33 AM, Jiangjie Qin
> > > > > > > > > >><jq...@linkedin.com.invalid>
> > > > > > > > > >> >wrote:
> > > > > > > > > >> >
> > > > > > > > > >> >>
> > > > > > > > > >> >> I'm just thinking instead of binding serialization
> with
> > > > > > producer,
> > > > > > > > > >> >>another
> > > > > > > > > >> >> option is to bind serializer/deserializer with
> > > > > > > > > >> >> ProducerRecord/ConsumerRecord (please see the detail
> > > > proposal
> > > > > > > > below.)
> > > > > > > > > >> >>            The arguments for this option is:
> > > > > > > > > >> >>         A. A single producer could send different
> > message
> > > > > > types.
> > > > > > > > > >>There
> > > > > > > > > >> >>are
> > > > > > > > > >> >> several use cases in LinkedIn for per record
> serializer
> > > > > > > > > >> >>         - In Samza, there are some in-stream
> > > > order-sensitive
> > > > > > > > control
> > > > > > > > > >> >> messages
> > > > > > > > > >> >> having different deserializer from other messages.
> > > > > > > > > >> >>         - There are use cases which need support for
> > > > sending
> > > > > > both
> > > > > > > > > >>Avro
> > > > > > > > > >> >> messages
> > > > > > > > > >> >> and raw bytes.
> > > > > > > > > >> >>         - Some use cases needs to deserialize some
> Avro
> > > > > > messages
> > > > > > > > into
> > > > > > > > > >> >> generic
> > > > > > > > > >> >> record and some other messages into specific record.
> > > > > > > > > >> >>         B. In current proposal, the
> > > serializer/deserilizer
> > > > is
> > > > > > > > > >> >>instantiated
> > > > > > > > > >> >> according to config. Compared with that, binding
> > > serializer
> > > > > > with
> > > > > > > > > >> >> ProducerRecord and ConsumerRecord is less error
> prone.
> > > > > > > > > >> >>
> > > > > > > > > >> >>
> > > > > > > > > >> >>         This option includes the following changes:
> > > > > > > > > >> >>         A. Add serializer and deserializer interfaces
> > to
> > > > > > replace
> > > > > > > > > >> >>serializer
> > > > > > > > > >> >> instance from config.
> > > > > > > > > >> >>                 Public interface Serializer <K, V> {
> > > > > > > > > >> >>                         public byte[] serializeKey(K
> > > key);
> > > > > > > > > >> >>                         public byte[]
> serializeValue(V
> > > > value);
> > > > > > > > > >> >>                 }
> > > > > > > > > >> >>                 Public interface deserializer <K, V>
> {
> > > > > > > > > >> >>                         Public K
> deserializeKey(byte[]
> > > > key);
> > > > > > > > > >> >>                         public V
> > deserializeValue(byte[]
> > > > > > value);
> > > > > > > > > >> >>                 }
> > > > > > > > > >> >>
> > > > > > > > > >> >>         B. Make ProducerRecord and ConsumerRecord
> > > abstract
> > > > > > class
> > > > > > > > > >> >> implementing
> > > > > > > > > >> >> Serializer <K, V> and Deserializer <K, V>
> respectively.
> > > > > > > > > >> >>                 Public abstract class ProducerRecord
> > <K,
> > > V>
> > > > > > > > > >>implements
> > > > > > > > > >> >> Serializer <K, V>
> > > > > > > > > >> >> {...}
> > > > > > > > > >> >>                 Public abstract class ConsumerRecord
> > <K,
> > > V>
> > > > > > > > > >>implements
> > > > > > > > > >> >> Deserializer <K,
> > > > > > > > > >> >> V> {...}
> > > > > > > > > >> >>
> > > > > > > > > >> >>         C. Instead of instantiate the
> > > > serializer/Deserializer
> > > > > > from
> > > > > > > > > >> >>config,
> > > > > > > > > >> >> let
> > > > > > > > > >> >> concrete ProducerRecord/ConsumerRecord extends the
> > > abstract
> > > > > > class
> > > > > > > > and
> > > > > > > > > >> >> override the serialize/deserialize methods.
> > > > > > > > > >> >>
> > > > > > > > > >> >>                 Public class AvroProducerRecord
> extends
> > > > > > > > > >>ProducerRecord
> > > > > > > > > >> >> <String,
> > > > > > > > > >> >> GenericRecord> {
> > > > > > > > > >> >>                         ...
> > > > > > > > > >> >>                         @Override
> > > > > > > > > >> >>                         Public byte[]
> > serializeKey(String
> > > > key)
> > > > > > {Š}
> > > > > > > > > >> >>                         @Override
> > > > > > > > > >> >>                         public byte[]
> > > > > > serializeValue(GenericRecord
> > > > > > > > > >> >>value);
> > > > > > > > > >> >>                 }
> > > > > > > > > >> >>
> > > > > > > > > >> >>                 Public class AvroConsumerRecord
> extends
> > > > > > > > > >>ConsumerRecord
> > > > > > > > > >> >> <String,
> > > > > > > > > >> >> GenericRecord> {
> > > > > > > > > >> >>                         ...
> > > > > > > > > >> >>                         @Override
> > > > > > > > > >> >>                         Public K
> deserializeKey(byte[]
> > > > key) {Š}
> > > > > > > > > >> >>                         @Override
> > > > > > > > > >> >>                         public V
> > deserializeValue(byte[]
> > > > > > value);
> > > > > > > > > >> >>                 }
> > > > > > > > > >> >>
> > > > > > > > > >> >>         D. The producer API changes to
> > > > > > > > > >> >>                 Public class KafkaProducer {
> > > > > > > > > >> >>                         ...
> > > > > > > > > >> >>
> > > > > > > > > >> >>                         Future<RecordMetadata> send
> > > > > > (ProducerRecord
> > > > > > > > > >><K,
> > > > > > > > > >> >>V>
> > > > > > > > > >> >> record) {
> > > > > > > > > >> >>                                 ...
> > > > > > > > > >> >>                                 K key =
> > > > > > > > > >>record.serializeKey(record.key);
> > > > > > > > > >> >>                                 V value =
> > > > > > > > > >> >> record.serializedValue(record.value);
> > > > > > > > > >> >>                                 BytesProducerRecord
> > > > > > > > > >>bytesProducerRecord
> > > > > > > > > >> >>=
> > > > > > > > > >> >> new
> > > > > > > > > >> >> BytesProducerRecord(topic, partition, key, value);
> > > > > > > > > >> >>                                 ...
> > > > > > > > > >> >>                         }
> > > > > > > > > >> >>                         ...
> > > > > > > > > >> >>                 }
> > > > > > > > > >> >>
> > > > > > > > > >> >>
> > > > > > > > > >> >>
> > > > > > > > > >> >> We also had some brainstorm in LinkedIn and here are
> > the
> > > > > > feedbacks:
> > > > > > > > > >> >>
> > > > > > > > > >> >> If the community decide to add the serialization back
> > to
> > > > new
> > > > > > > > > >>producer,
> > > > > > > > > >> >> besides current proposal which changes new producer
> API
> > > to
> > > > be a
> > > > > > > > > >> >>template,
> > > > > > > > > >> >> there are some other options raised during our
> > > discussion:
> > > > > > > > > >> >>         1) Rather than change current new producer
> API,
> > > we
> > > > can
> > > > > > > > > >>provide a
> > > > > > > > > >> >> wrapper
> > > > > > > > > >> >> of current new producer (e.g.
> KafkaSerializedProducer)
> > > and
> > > > > > make it
> > > > > > > > > >> >> available to users. As there is value in the
> simplicity
> > > of
> > > > > > current
> > > > > > > > > >>API.
> > > > > > > > > >> >>
> > > > > > > > > >> >>         2) If we decide to go with tempalated new
> > > producer
> > > > API,
> > > > > > > > > >> >>according
> > > > > > > > > >> >> to
> > > > > > > > > >> >> experience in LinkedIn, it might worth considering to
> > > > > > instantiate
> > > > > > > > the
> > > > > > > > > >> >> serializer in code instead of from config so we can
> > avoid
> > > > > > runtime
> > > > > > > > > >>errors
> > > > > > > > > >> >> due to dynamic instantiation from config, which is
> more
> > > > error
> > > > > > > > prone.
> > > > > > > > > >>If
> > > > > > > > > >> >> that is the case, the producer API could be changed
> to
> > > > > > something
> > > > > > > > > >>like:
> > > > > > > > > >> >>                 producer = new Producer<K,
> > > > V>(KeySerializer<K>,
> > > > > > > > > >> >> ValueSerializer<V>)
> > > > > > > > > >> >>
> > > > > > > > > >> >> --Jiangjie (Becket) Qin
> > > > > > > > > >> >>
> > > > > > > > > >> >>
> > > > > > > > > >> >> On 11/24/14, 5:58 PM, "Jun Rao" <ju...@gmail.com>
> > > 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
> > > > > > > > > >> >>
> > > > > > > > > >> >>
> > > > > > > > > >>
> > > > > > > > > >>
> > > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > >
> > > > > >
> > > >
> > > >
> > >
> >
>

Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Shannon Lloyd <sh...@gmail.com>.
Are you guys planning another beta for everyone to try out the changes
before you cut 0.8.2 final?

Cheers,
Shannon

On 18 December 2014 at 11:24, Rajiv Kurian <ra...@signalfuse.com> wrote:
>
> Has the mvn repo been updated too?
>
> On Wed, Dec 17, 2014 at 4:31 PM, Jun Rao <ju...@confluent.io> wrote:
> >
> > Thanks everyone for the feedback and the discussion. The proposed changes
> > have been checked into both 0.8.2 and trunk.
> >
> > Jun
> >
> > On Tue, Dec 16, 2014 at 10:43 PM, Joel Koshy <jj...@gmail.com>
> wrote:
> > >
> > > Jun,
> > >
> > > Thanks for summarizing this - it helps confirm for me that I did not
> > > misunderstand anything in this thread so far; and that I disagree with
> > > the premise that the steps in using the current byte-oriented API is
> > > cumbersome or inflexible. It involves instantiating the K-V
> > > serializers in code (as opposed to config) and a extra (but explicit
> > > - i.e., making it very clear to the user) but simple call to serialize
> > > before sending.
> > >
> > > The point about downstream queries breaking can happen just as well
> > > with the implicit serializers/deserializers - since ultimately people
> > > have to instantiate the specific type in their code and if they want
> > > to send it they will.
> > >
> > > I think adoption is also equivalent since people will just instantiate
> > > whatever serializer/deserializer they want in one line. Plugging in a
> > > new serializer implementation does require a code change, but that can
> > > also be avoided via a config driven factory.
> > >
> > > So I'm still +0 on the change but I'm definitely not against moving
> > > forward with the changes. i.e., unless there is any strong -1 on the
> > > proposal from anyone else.
> > >
> > > Thanks,
> > >
> > > Joel
> > >
> > > > With a byte array interface, of course there is nothing that one
> can't
> > > do.
> > > > However, the real question is that whether we want to encourage
> people
> > to
> > > > use it this way or not. Being able to flow just bytes is definitely
> > > easier
> > > > to get started. That's why many early adopters choose to do it that
> > way.
> > > > However, it's often the case that they start feeling the pain later
> > when
> > > > some producers change the data format. Their Hive/Pig queries start
> to
> > > > break and it's a painful process to have the issue fixed. So, the
> > purpose
> > > > of this api change is really to encourage people to standardize on a
> > > single
> > > > serializer/deserializer that supports things like data validation and
> > > > schema evolution upstream in the producer. Now, suppose there is an
> > Avro
> > > > serializer/deserializer implementation. How do we make it easy for
> > people
> > > > to adopt? If the serializer is part of the api, we can just say, wire
> > in
> > > > the Avro serializer for key and/or value in the config and then you
> can
> > > > start sending Avro records to the producer. If the serializer is not
> > part
> > > > of the api, we have to say, first instantiate a key and/or value
> > > serializer
> > > > this way, send the key to the key serializer to get the key bytes,
> send
> > > the
> > > > value to the value serializer to get the value bytes, and finally
> send
> > > the
> > > > bytes to the producer. The former will be simpler and likely makes
> the
> > > > adoption easier.
> > > >
> > > > Thanks,
> > > >
> > > > Jun
> > > >
> > > > On Mon, Dec 15, 2014 at 7:20 PM, Joel Koshy <jj...@gmail.com>
> > wrote:
> > > > >
> > > > > Documentation is inevitable even if the serializer/deserializer is
> > > > > part of the API - since the user has to set it up in the configs.
> So
> > > > > again, you can only encourage people to use it through
> documentation.
> > > > > The simpler byte-oriented API seems clearer to me because anyone
> who
> > > > > needs to send (or receive) a specific data type will _be forced to_
> > > > > (or actually, _intuitively_) select a serializer (or deserializer)
> > and
> > > > > will definitely pick an already available implementation if a good
> > one
> > > > > already exists.
> > > > >
> > > > > Sorry I still don't get it and this is really the only sticking
> point
> > > > > for me, albeit a minor one (which is why I have been +0 all along
> on
> > > > > the change). I (and I think many others) would appreciate it if
> > > > > someone can help me understand this better.  So I will repeat the
> > > > > question: What "usage pattern" cannot be supported by easily by the
> > > > > simpler API without adding burden on the user?
> > > > >
> > > > > Thanks,
> > > > >
> > > > > Joel
> > > > >
> > > > > On Mon, Dec 15, 2014 at 11:59:34AM -0800, Jun Rao wrote:
> > > > > > Joel,
> > > > > >
> > > > > > It's just that if the serializer/deserializer is not part of the
> > > API, you
> > > > > > can only encourage people to use it through documentation.
> However,
> > > not
> > > > > > everyone will read the documentation if it's not directly used in
> > the
> > > > > API.
> > > > > >
> > > > > > Thanks,
> > > > > >
> > > > > > Jun
> > > > > >
> > > > > > On Mon, Dec 15, 2014 at 2:11 AM, Joel Koshy <jjkoshy.w@gmail.com
> >
> > > wrote:
> > > > > >
> > > > > > > (sorry about the late follow-up late - I'm traveling most of
> this
> > > > > > > month)
> > > > > > >
> > > > > > > I'm likely missing something obvious, but I find the following
> to
> > > be a
> > > > > > > somewhat vague point that has been mentioned more than once in
> > this
> > > > > > > thread without a clear explanation. i.e., why is it hard to
> > share a
> > > > > > > serializer/deserializer implementation and just have the
> clients
> > > call
> > > > > > > it before a send/receive? What "usage pattern" cannot be
> > supported
> > > by
> > > > > > > the simpler API?
> > > > > > >
> > > > > > > > 1. Can we keep the serialization semantics outside the
> Producer
> > > > > interface
> > > > > > > > and have simple bytes in / bytes out for the interface (This
> is
> > > what
> > > > > we
> > > > > > > > have today).
> > > > > > > >
> > > > > > > > The points for this is to keep the interface simple and usage
> > > easy to
> > > > > > > > understand. The points against this is that it gets hard to
> > share
> > > > > common
> > > > > > > > usage patterns around serialization/message validations for
> the
> > > > > future.
> > > > > > >
> > > > > > >
> > > > > > > On Tue, Dec 09, 2014 at 03:51:08AM +0000, Sriram Subramanian
> > wrote:
> > > > > > > > Thank you Jay. I agree with the issue that you point w.r.t
> > paired
> > > > > > > > serializers. I also think having mix serialization types is
> > > rare. To
> > > > > get
> > > > > > > > the current behavior, one can simply use a
> ByteArraySerializer.
> > > This
> > > > > is
> > > > > > > > best understood by talking with many customers and you seem
> to
> > > have
> > > > > done
> > > > > > > > that. I am convinced about the change.
> > > > > > > >
> > > > > > > > For the rest who gave -1 or 0 for this proposal, does the
> > answers
> > > > > for the
> > > > > > > > three points(updated) below seem reasonable? Are these
> > > explanations
> > > > > > > > convincing?
> > > > > > > >
> > > > > > > >
> > > > > > > > 1. Can we keep the serialization semantics outside the
> Producer
> > > > > interface
> > > > > > > > and have simple bytes in / bytes out for the interface (This
> is
> > > what
> > > > > we
> > > > > > > > have today).
> > > > > > > >
> > > > > > > > The points for this is to keep the interface simple and usage
> > > easy to
> > > > > > > > understand. The points against this is that it gets hard to
> > share
> > > > > common
> > > > > > > > usage patterns around serialization/message validations for
> the
> > > > > future.
> > > > > > > >
> > > > > > > > 2. Can we create a wrapper producer that does the
> serialization
> > > and
> > > > > have
> > > > > > > > different variants of it for different data formats?
> > > > > > > >
> > > > > > > > The points for this is again to keep the main API clean. The
> > > points
> > > > > > > > against this is that it duplicates the API, increases the
> > surface
> > > > > area
> > > > > > > and
> > > > > > > > creates redundancy for a minor addition.
> > > > > > > >
> > > > > > > > 3. Do we need to support different data types per record? The
> > > current
> > > > > > > > interface (bytes in/bytes out) lets you instantiate one
> > producer
> > > and
> > > > > use
> > > > > > > > it to send multiple data formats. There seems to be some
> valid
> > > use
> > > > > cases
> > > > > > > > for this.
> > > > > > > >
> > > > > > > >
> > > > > > > > Mixed serialization types are rare based on interactions with
> > > > > customers.
> > > > > > > > To get the current behavior, one can simply use a
> > > > > ByteArraySerializer.
> > > > > > > >
> > > > > > > > On 12/5/14 5:00 PM, "Jay Kreps" <ja...@confluent.io> wrote:
> > > > > > > >
> > > > > > > > >Hey Sriram,
> > > > > > > > >
> > > > > > > > >Thanks! I think this is a very helpful summary.
> > > > > > > > >
> > > > > > > > >Let me try to address your point about passing in the serde
> at
> > > send
> > > > > > > time.
> > > > > > > > >
> > > > > > > > >I think the first objection is really to the paired
> key/value
> > > > > serializer
> > > > > > > > >interfaces. This leads to kind of a weird combinatorial
> thing
> > > where
> > > > > you
> > > > > > > > >would have an avro/avro serializer a string/avro
> serializer, a
> > > pb/pb
> > > > > > > > >serializer, and a string/pb serializer, and so on. But your
> > > proposal
> > > > > > > would
> > > > > > > > >work as well with separate serializers for key and value.
> > > > > > > > >
> > > > > > > > >I think the downside is just the one you call out--that this
> > is
> > > a
> > > > > corner
> > > > > > > > >case and you end up with two versions of all the apis to
> > > support it.
> > > > > > > This
> > > > > > > > >also makes the serializer api more annoying to implement. I
> > > think
> > > > > the
> > > > > > > > >alternative solution to this case and any other we can give
> > > people
> > > > > is
> > > > > > > just
> > > > > > > > >configuring ByteArraySerializer which gives you basically
> the
> > > api
> > > > > that
> > > > > > > you
> > > > > > > > >have now with byte arrays. If this is incredibly common then
> > > this
> > > > > would
> > > > > > > be
> > > > > > > > >a silly solution, but I guess the belief is that these cases
> > are
> > > > > rare
> > > > > > > and
> > > > > > > > >a
> > > > > > > > >really well implemented avro or json serializer should be
> 100%
> > > of
> > > > > what
> > > > > > > > >most
> > > > > > > > >people need.
> > > > > > > > >
> > > > > > > > >In practice the cases that actually mix serialization types
> > in a
> > > > > single
> > > > > > > > >stream are pretty rare I think just because the consumer
> then
> > > has
> > > > > the
> > > > > > > > >problem of guessing how to deserialize, so most of these
> will
> > > end up
> > > > > > > with
> > > > > > > > >at least some marker or schema id or whatever that tells you
> > > how to
> > > > > read
> > > > > > > > >the data. Arguable this mixed serialization with marker is
> > > itself a
> > > > > > > > >serializer type and should have a serializer of its own...
> > > > > > > > >
> > > > > > > > >-Jay
> > > > > > > > >
> > > > > > > > >On Fri, Dec 5, 2014 at 3:48 PM, Sriram Subramanian <
> > > > > > > > >srsubramanian@linkedin.com.invalid> wrote:
> > > > > > > > >
> > > > > > > > >> This thread has diverged multiple times now and it would
> be
> > > worth
> > > > > > > > >> summarizing them.
> > > > > > > > >>
> > > > > > > > >> There seems to be the following points of discussion -
> > > > > > > > >>
> > > > > > > > >> 1. Can we keep the serialization semantics outside the
> > > Producer
> > > > > > > > >>interface
> > > > > > > > >> and have simple bytes in / bytes out for the interface
> (This
> > > is
> > > > > what
> > > > > > > we
> > > > > > > > >> have today).
> > > > > > > > >>
> > > > > > > > >> The points for this is to keep the interface simple and
> > usage
> > > > > easy to
> > > > > > > > >> understand. The points against this is that it gets hard
> to
> > > share
> > > > > > > common
> > > > > > > > >> usage patterns around serialization/message validations
> for
> > > the
> > > > > > > future.
> > > > > > > > >>
> > > > > > > > >> 2. Can we create a wrapper producer that does the
> > > serialization
> > > > > and
> > > > > > > have
> > > > > > > > >> different variants of it for different data formats?
> > > > > > > > >>
> > > > > > > > >> The points for this is again to keep the main API clean.
> The
> > > > > points
> > > > > > > > >> against this is that it duplicates the API, increases the
> > > surface
> > > > > area
> > > > > > > > >>and
> > > > > > > > >> creates redundancy for a minor addition.
> > > > > > > > >>
> > > > > > > > >> 3. Do we need to support different data types per record?
> > The
> > > > > current
> > > > > > > > >> interface (bytes in/bytes out) lets you instantiate one
> > > producer
> > > > > and
> > > > > > > use
> > > > > > > > >> it to send multiple data formats. There seems to be some
> > > valid use
> > > > > > > cases
> > > > > > > > >> for this.
> > > > > > > > >>
> > > > > > > > >> I have still not seen a strong argument against not having
> > > this
> > > > > > > > >> functionality. Can someone provide their views on why we
> > don't
> > > > > need
> > > > > > > this
> > > > > > > > >> support that is possible with the current API?
> > > > > > > > >>
> > > > > > > > >> One possible approach for the per record serialization
> would
> > > be to
> > > > > > > > >>define
> > > > > > > > >>
> > > > > > > > >> public interface SerDe<K,V> {
> > > > > > > > >>   public byte[] serializeKey();
> > > > > > > > >>
> > > > > > > > >>   public K deserializeKey();
> > > > > > > > >>
> > > > > > > > >>   public byte[] serializeValue();
> > > > > > > > >>
> > > > > > > > >>   public V deserializeValue();
> > > > > > > > >> }
> > > > > > > > >>
> > > > > > > > >> This would be used by both the Producer and the Consumer.
> > > > > > > > >>
> > > > > > > > >> The send APIs can then be
> > > > > > > > >>
> > > > > > > > >> public Future<RecordMetadata> send(ProducerRecord<K,V>
> > > record);
> > > > > > > > >> public Future<RecordMetadata> send(ProducerRecord<K,V>
> > record,
> > > > > > > Callback
> > > > > > > > >> callback);
> > > > > > > > >>
> > > > > > > > >>
> > > > > > > > >> public Future<RecordMetadata> send(ProducerRecord<K,V>
> > record,
> > > > > > > > >>SerDe<K,V>
> > > > > > > > >> serde);
> > > > > > > > >>
> > > > > > > > >> public Future<RecordMetadata> send(ProducerRecord<K,V>
> > record,
> > > > > > > > >>SerDe<K,V>
> > > > > > > > >> serde, Callback callback);
> > > > > > > > >>
> > > > > > > > >>
> > > > > > > > >> A default SerDe can be set in the config. The producer
> would
> > > use
> > > > > the
> > > > > > > > >> default from the config if the non-serde send APIs are
> used.
> > > The
> > > > > > > > >>downside
> > > > > > > > >> to this approach is that we would need to have four
> variants
> > > of
> > > > > Send
> > > > > > > API
> > > > > > > > >> for the Producer.
> > > > > > > > >>
> > > > > > > > >>
> > > > > > > > >>
> > > > > > > > >>
> > > > > > > > >>
> > > > > > > > >>
> > > > > > > > >> On 12/5/14 3:16 PM, "Jun Rao" <ju...@confluent.io> wrote:
> > > > > > > > >>
> > > > > > > > >> >Jiangjie,
> > > > > > > > >> >
> > > > > > > > >> >The issue with adding the serializer in ProducerRecord is
> > > that
> > > > > you
> > > > > > > > >>need to
> > > > > > > > >> >implement all combinations of serializers for key and
> > value.
> > > So,
> > > > > > > > >>instead
> > > > > > > > >> >of
> > > > > > > > >> >just implementing int and string serializers, you will
> have
> > > to
> > > > > > > > >>implement
> > > > > > > > >> >all 4 combinations.
> > > > > > > > >> >
> > > > > > > > >> >Adding a new producer constructor like Producer<K,
> > > > > > > V>(KeySerializer<K>,
> > > > > > > > >> >ValueSerializer<V>, Properties properties) can be useful.
> > > > > > > > >> >
> > > > > > > > >> >Thanks,
> > > > > > > > >> >
> > > > > > > > >> >Jun
> > > > > > > > >> >
> > > > > > > > >> >On Thu, Dec 4, 2014 at 10:33 AM, Jiangjie Qin
> > > > > > > > >><jq...@linkedin.com.invalid>
> > > > > > > > >> >wrote:
> > > > > > > > >> >
> > > > > > > > >> >>
> > > > > > > > >> >> I'm just thinking instead of binding serialization with
> > > > > producer,
> > > > > > > > >> >>another
> > > > > > > > >> >> option is to bind serializer/deserializer with
> > > > > > > > >> >> ProducerRecord/ConsumerRecord (please see the detail
> > > proposal
> > > > > > > below.)
> > > > > > > > >> >>            The arguments for this option is:
> > > > > > > > >> >>         A. A single producer could send different
> message
> > > > > types.
> > > > > > > > >>There
> > > > > > > > >> >>are
> > > > > > > > >> >> several use cases in LinkedIn for per record serializer
> > > > > > > > >> >>         - In Samza, there are some in-stream
> > > order-sensitive
> > > > > > > control
> > > > > > > > >> >> messages
> > > > > > > > >> >> having different deserializer from other messages.
> > > > > > > > >> >>         - There are use cases which need support for
> > > sending
> > > > > both
> > > > > > > > >>Avro
> > > > > > > > >> >> messages
> > > > > > > > >> >> and raw bytes.
> > > > > > > > >> >>         - Some use cases needs to deserialize some Avro
> > > > > messages
> > > > > > > into
> > > > > > > > >> >> generic
> > > > > > > > >> >> record and some other messages into specific record.
> > > > > > > > >> >>         B. In current proposal, the
> > serializer/deserilizer
> > > is
> > > > > > > > >> >>instantiated
> > > > > > > > >> >> according to config. Compared with that, binding
> > serializer
> > > > > with
> > > > > > > > >> >> ProducerRecord and ConsumerRecord is less error prone.
> > > > > > > > >> >>
> > > > > > > > >> >>
> > > > > > > > >> >>         This option includes the following changes:
> > > > > > > > >> >>         A. Add serializer and deserializer interfaces
> to
> > > > > replace
> > > > > > > > >> >>serializer
> > > > > > > > >> >> instance from config.
> > > > > > > > >> >>                 Public interface Serializer <K, V> {
> > > > > > > > >> >>                         public byte[] serializeKey(K
> > key);
> > > > > > > > >> >>                         public byte[] serializeValue(V
> > > value);
> > > > > > > > >> >>                 }
> > > > > > > > >> >>                 Public interface deserializer <K, V> {
> > > > > > > > >> >>                         Public K deserializeKey(byte[]
> > > key);
> > > > > > > > >> >>                         public V
> deserializeValue(byte[]
> > > > > value);
> > > > > > > > >> >>                 }
> > > > > > > > >> >>
> > > > > > > > >> >>         B. Make ProducerRecord and ConsumerRecord
> > abstract
> > > > > class
> > > > > > > > >> >> implementing
> > > > > > > > >> >> Serializer <K, V> and Deserializer <K, V> respectively.
> > > > > > > > >> >>                 Public abstract class ProducerRecord
> <K,
> > V>
> > > > > > > > >>implements
> > > > > > > > >> >> Serializer <K, V>
> > > > > > > > >> >> {...}
> > > > > > > > >> >>                 Public abstract class ConsumerRecord
> <K,
> > V>
> > > > > > > > >>implements
> > > > > > > > >> >> Deserializer <K,
> > > > > > > > >> >> V> {...}
> > > > > > > > >> >>
> > > > > > > > >> >>         C. Instead of instantiate the
> > > serializer/Deserializer
> > > > > from
> > > > > > > > >> >>config,
> > > > > > > > >> >> let
> > > > > > > > >> >> concrete ProducerRecord/ConsumerRecord extends the
> > abstract
> > > > > class
> > > > > > > and
> > > > > > > > >> >> override the serialize/deserialize methods.
> > > > > > > > >> >>
> > > > > > > > >> >>                 Public class AvroProducerRecord extends
> > > > > > > > >>ProducerRecord
> > > > > > > > >> >> <String,
> > > > > > > > >> >> GenericRecord> {
> > > > > > > > >> >>                         ...
> > > > > > > > >> >>                         @Override
> > > > > > > > >> >>                         Public byte[]
> serializeKey(String
> > > key)
> > > > > {Š}
> > > > > > > > >> >>                         @Override
> > > > > > > > >> >>                         public byte[]
> > > > > serializeValue(GenericRecord
> > > > > > > > >> >>value);
> > > > > > > > >> >>                 }
> > > > > > > > >> >>
> > > > > > > > >> >>                 Public class AvroConsumerRecord extends
> > > > > > > > >>ConsumerRecord
> > > > > > > > >> >> <String,
> > > > > > > > >> >> GenericRecord> {
> > > > > > > > >> >>                         ...
> > > > > > > > >> >>                         @Override
> > > > > > > > >> >>                         Public K deserializeKey(byte[]
> > > key) {Š}
> > > > > > > > >> >>                         @Override
> > > > > > > > >> >>                         public V
> deserializeValue(byte[]
> > > > > value);
> > > > > > > > >> >>                 }
> > > > > > > > >> >>
> > > > > > > > >> >>         D. The producer API changes to
> > > > > > > > >> >>                 Public class KafkaProducer {
> > > > > > > > >> >>                         ...
> > > > > > > > >> >>
> > > > > > > > >> >>                         Future<RecordMetadata> send
> > > > > (ProducerRecord
> > > > > > > > >><K,
> > > > > > > > >> >>V>
> > > > > > > > >> >> record) {
> > > > > > > > >> >>                                 ...
> > > > > > > > >> >>                                 K key =
> > > > > > > > >>record.serializeKey(record.key);
> > > > > > > > >> >>                                 V value =
> > > > > > > > >> >> record.serializedValue(record.value);
> > > > > > > > >> >>                                 BytesProducerRecord
> > > > > > > > >>bytesProducerRecord
> > > > > > > > >> >>=
> > > > > > > > >> >> new
> > > > > > > > >> >> BytesProducerRecord(topic, partition, key, value);
> > > > > > > > >> >>                                 ...
> > > > > > > > >> >>                         }
> > > > > > > > >> >>                         ...
> > > > > > > > >> >>                 }
> > > > > > > > >> >>
> > > > > > > > >> >>
> > > > > > > > >> >>
> > > > > > > > >> >> We also had some brainstorm in LinkedIn and here are
> the
> > > > > feedbacks:
> > > > > > > > >> >>
> > > > > > > > >> >> If the community decide to add the serialization back
> to
> > > new
> > > > > > > > >>producer,
> > > > > > > > >> >> besides current proposal which changes new producer API
> > to
> > > be a
> > > > > > > > >> >>template,
> > > > > > > > >> >> there are some other options raised during our
> > discussion:
> > > > > > > > >> >>         1) Rather than change current new producer API,
> > we
> > > can
> > > > > > > > >>provide a
> > > > > > > > >> >> wrapper
> > > > > > > > >> >> of current new producer (e.g. KafkaSerializedProducer)
> > and
> > > > > make it
> > > > > > > > >> >> available to users. As there is value in the simplicity
> > of
> > > > > current
> > > > > > > > >>API.
> > > > > > > > >> >>
> > > > > > > > >> >>         2) If we decide to go with tempalated new
> > producer
> > > API,
> > > > > > > > >> >>according
> > > > > > > > >> >> to
> > > > > > > > >> >> experience in LinkedIn, it might worth considering to
> > > > > instantiate
> > > > > > > the
> > > > > > > > >> >> serializer in code instead of from config so we can
> avoid
> > > > > runtime
> > > > > > > > >>errors
> > > > > > > > >> >> due to dynamic instantiation from config, which is more
> > > error
> > > > > > > prone.
> > > > > > > > >>If
> > > > > > > > >> >> that is the case, the producer API could be changed to
> > > > > something
> > > > > > > > >>like:
> > > > > > > > >> >>                 producer = new Producer<K,
> > > V>(KeySerializer<K>,
> > > > > > > > >> >> ValueSerializer<V>)
> > > > > > > > >> >>
> > > > > > > > >> >> --Jiangjie (Becket) Qin
> > > > > > > > >> >>
> > > > > > > > >> >>
> > > > > > > > >> >> On 11/24/14, 5:58 PM, "Jun Rao" <ju...@gmail.com>
> > 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
> > > > > > > > >> >>
> > > > > > > > >> >>
> > > > > > > > >>
> > > > > > > > >>
> > > > > > > >
> > > > > > >
> > > > > > >
> > > > >
> > > > >
> > >
> > >
> >
>

Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Jun Rao <ju...@confluent.io>.
Not yet. It will be when 0.8.2 is release.

Thanks,

Jun

On Wed, Dec 17, 2014 at 5:24 PM, Rajiv Kurian <ra...@signalfuse.com> wrote:
>
> Has the mvn repo been updated too?
>
> On Wed, Dec 17, 2014 at 4:31 PM, Jun Rao <ju...@confluent.io> wrote:
> >
> > Thanks everyone for the feedback and the discussion. The proposed changes
> > have been checked into both 0.8.2 and trunk.
> >
> > Jun
> >
> > On Tue, Dec 16, 2014 at 10:43 PM, Joel Koshy <jj...@gmail.com>
> wrote:
> > >
> > > Jun,
> > >
> > > Thanks for summarizing this - it helps confirm for me that I did not
> > > misunderstand anything in this thread so far; and that I disagree with
> > > the premise that the steps in using the current byte-oriented API is
> > > cumbersome or inflexible. It involves instantiating the K-V
> > > serializers in code (as opposed to config) and a extra (but explicit
> > > - i.e., making it very clear to the user) but simple call to serialize
> > > before sending.
> > >
> > > The point about downstream queries breaking can happen just as well
> > > with the implicit serializers/deserializers - since ultimately people
> > > have to instantiate the specific type in their code and if they want
> > > to send it they will.
> > >
> > > I think adoption is also equivalent since people will just instantiate
> > > whatever serializer/deserializer they want in one line. Plugging in a
> > > new serializer implementation does require a code change, but that can
> > > also be avoided via a config driven factory.
> > >
> > > So I'm still +0 on the change but I'm definitely not against moving
> > > forward with the changes. i.e., unless there is any strong -1 on the
> > > proposal from anyone else.
> > >
> > > Thanks,
> > >
> > > Joel
> > >
> > > > With a byte array interface, of course there is nothing that one
> can't
> > > do.
> > > > However, the real question is that whether we want to encourage
> people
> > to
> > > > use it this way or not. Being able to flow just bytes is definitely
> > > easier
> > > > to get started. That's why many early adopters choose to do it that
> > way.
> > > > However, it's often the case that they start feeling the pain later
> > when
> > > > some producers change the data format. Their Hive/Pig queries start
> to
> > > > break and it's a painful process to have the issue fixed. So, the
> > purpose
> > > > of this api change is really to encourage people to standardize on a
> > > single
> > > > serializer/deserializer that supports things like data validation and
> > > > schema evolution upstream in the producer. Now, suppose there is an
> > Avro
> > > > serializer/deserializer implementation. How do we make it easy for
> > people
> > > > to adopt? If the serializer is part of the api, we can just say, wire
> > in
> > > > the Avro serializer for key and/or value in the config and then you
> can
> > > > start sending Avro records to the producer. If the serializer is not
> > part
> > > > of the api, we have to say, first instantiate a key and/or value
> > > serializer
> > > > this way, send the key to the key serializer to get the key bytes,
> send
> > > the
> > > > value to the value serializer to get the value bytes, and finally
> send
> > > the
> > > > bytes to the producer. The former will be simpler and likely makes
> the
> > > > adoption easier.
> > > >
> > > > Thanks,
> > > >
> > > > Jun
> > > >
> > > > On Mon, Dec 15, 2014 at 7:20 PM, Joel Koshy <jj...@gmail.com>
> > wrote:
> > > > >
> > > > > Documentation is inevitable even if the serializer/deserializer is
> > > > > part of the API - since the user has to set it up in the configs.
> So
> > > > > again, you can only encourage people to use it through
> documentation.
> > > > > The simpler byte-oriented API seems clearer to me because anyone
> who
> > > > > needs to send (or receive) a specific data type will _be forced to_
> > > > > (or actually, _intuitively_) select a serializer (or deserializer)
> > and
> > > > > will definitely pick an already available implementation if a good
> > one
> > > > > already exists.
> > > > >
> > > > > Sorry I still don't get it and this is really the only sticking
> point
> > > > > for me, albeit a minor one (which is why I have been +0 all along
> on
> > > > > the change). I (and I think many others) would appreciate it if
> > > > > someone can help me understand this better.  So I will repeat the
> > > > > question: What "usage pattern" cannot be supported by easily by the
> > > > > simpler API without adding burden on the user?
> > > > >
> > > > > Thanks,
> > > > >
> > > > > Joel
> > > > >
> > > > > On Mon, Dec 15, 2014 at 11:59:34AM -0800, Jun Rao wrote:
> > > > > > Joel,
> > > > > >
> > > > > > It's just that if the serializer/deserializer is not part of the
> > > API, you
> > > > > > can only encourage people to use it through documentation.
> However,
> > > not
> > > > > > everyone will read the documentation if it's not directly used in
> > the
> > > > > API.
> > > > > >
> > > > > > Thanks,
> > > > > >
> > > > > > Jun
> > > > > >
> > > > > > On Mon, Dec 15, 2014 at 2:11 AM, Joel Koshy <jjkoshy.w@gmail.com
> >
> > > wrote:
> > > > > >
> > > > > > > (sorry about the late follow-up late - I'm traveling most of
> this
> > > > > > > month)
> > > > > > >
> > > > > > > I'm likely missing something obvious, but I find the following
> to
> > > be a
> > > > > > > somewhat vague point that has been mentioned more than once in
> > this
> > > > > > > thread without a clear explanation. i.e., why is it hard to
> > share a
> > > > > > > serializer/deserializer implementation and just have the
> clients
> > > call
> > > > > > > it before a send/receive? What "usage pattern" cannot be
> > supported
> > > by
> > > > > > > the simpler API?
> > > > > > >
> > > > > > > > 1. Can we keep the serialization semantics outside the
> Producer
> > > > > interface
> > > > > > > > and have simple bytes in / bytes out for the interface (This
> is
> > > what
> > > > > we
> > > > > > > > have today).
> > > > > > > >
> > > > > > > > The points for this is to keep the interface simple and usage
> > > easy to
> > > > > > > > understand. The points against this is that it gets hard to
> > share
> > > > > common
> > > > > > > > usage patterns around serialization/message validations for
> the
> > > > > future.
> > > > > > >
> > > > > > >
> > > > > > > On Tue, Dec 09, 2014 at 03:51:08AM +0000, Sriram Subramanian
> > wrote:
> > > > > > > > Thank you Jay. I agree with the issue that you point w.r.t
> > paired
> > > > > > > > serializers. I also think having mix serialization types is
> > > rare. To
> > > > > get
> > > > > > > > the current behavior, one can simply use a
> ByteArraySerializer.
> > > This
> > > > > is
> > > > > > > > best understood by talking with many customers and you seem
> to
> > > have
> > > > > done
> > > > > > > > that. I am convinced about the change.
> > > > > > > >
> > > > > > > > For the rest who gave -1 or 0 for this proposal, does the
> > answers
> > > > > for the
> > > > > > > > three points(updated) below seem reasonable? Are these
> > > explanations
> > > > > > > > convincing?
> > > > > > > >
> > > > > > > >
> > > > > > > > 1. Can we keep the serialization semantics outside the
> Producer
> > > > > interface
> > > > > > > > and have simple bytes in / bytes out for the interface (This
> is
> > > what
> > > > > we
> > > > > > > > have today).
> > > > > > > >
> > > > > > > > The points for this is to keep the interface simple and usage
> > > easy to
> > > > > > > > understand. The points against this is that it gets hard to
> > share
> > > > > common
> > > > > > > > usage patterns around serialization/message validations for
> the
> > > > > future.
> > > > > > > >
> > > > > > > > 2. Can we create a wrapper producer that does the
> serialization
> > > and
> > > > > have
> > > > > > > > different variants of it for different data formats?
> > > > > > > >
> > > > > > > > The points for this is again to keep the main API clean. The
> > > points
> > > > > > > > against this is that it duplicates the API, increases the
> > surface
> > > > > area
> > > > > > > and
> > > > > > > > creates redundancy for a minor addition.
> > > > > > > >
> > > > > > > > 3. Do we need to support different data types per record? The
> > > current
> > > > > > > > interface (bytes in/bytes out) lets you instantiate one
> > producer
> > > and
> > > > > use
> > > > > > > > it to send multiple data formats. There seems to be some
> valid
> > > use
> > > > > cases
> > > > > > > > for this.
> > > > > > > >
> > > > > > > >
> > > > > > > > Mixed serialization types are rare based on interactions with
> > > > > customers.
> > > > > > > > To get the current behavior, one can simply use a
> > > > > ByteArraySerializer.
> > > > > > > >
> > > > > > > > On 12/5/14 5:00 PM, "Jay Kreps" <ja...@confluent.io> wrote:
> > > > > > > >
> > > > > > > > >Hey Sriram,
> > > > > > > > >
> > > > > > > > >Thanks! I think this is a very helpful summary.
> > > > > > > > >
> > > > > > > > >Let me try to address your point about passing in the serde
> at
> > > send
> > > > > > > time.
> > > > > > > > >
> > > > > > > > >I think the first objection is really to the paired
> key/value
> > > > > serializer
> > > > > > > > >interfaces. This leads to kind of a weird combinatorial
> thing
> > > where
> > > > > you
> > > > > > > > >would have an avro/avro serializer a string/avro
> serializer, a
> > > pb/pb
> > > > > > > > >serializer, and a string/pb serializer, and so on. But your
> > > proposal
> > > > > > > would
> > > > > > > > >work as well with separate serializers for key and value.
> > > > > > > > >
> > > > > > > > >I think the downside is just the one you call out--that this
> > is
> > > a
> > > > > corner
> > > > > > > > >case and you end up with two versions of all the apis to
> > > support it.
> > > > > > > This
> > > > > > > > >also makes the serializer api more annoying to implement. I
> > > think
> > > > > the
> > > > > > > > >alternative solution to this case and any other we can give
> > > people
> > > > > is
> > > > > > > just
> > > > > > > > >configuring ByteArraySerializer which gives you basically
> the
> > > api
> > > > > that
> > > > > > > you
> > > > > > > > >have now with byte arrays. If this is incredibly common then
> > > this
> > > > > would
> > > > > > > be
> > > > > > > > >a silly solution, but I guess the belief is that these cases
> > are
> > > > > rare
> > > > > > > and
> > > > > > > > >a
> > > > > > > > >really well implemented avro or json serializer should be
> 100%
> > > of
> > > > > what
> > > > > > > > >most
> > > > > > > > >people need.
> > > > > > > > >
> > > > > > > > >In practice the cases that actually mix serialization types
> > in a
> > > > > single
> > > > > > > > >stream are pretty rare I think just because the consumer
> then
> > > has
> > > > > the
> > > > > > > > >problem of guessing how to deserialize, so most of these
> will
> > > end up
> > > > > > > with
> > > > > > > > >at least some marker or schema id or whatever that tells you
> > > how to
> > > > > read
> > > > > > > > >the data. Arguable this mixed serialization with marker is
> > > itself a
> > > > > > > > >serializer type and should have a serializer of its own...
> > > > > > > > >
> > > > > > > > >-Jay
> > > > > > > > >
> > > > > > > > >On Fri, Dec 5, 2014 at 3:48 PM, Sriram Subramanian <
> > > > > > > > >srsubramanian@linkedin.com.invalid> wrote:
> > > > > > > > >
> > > > > > > > >> This thread has diverged multiple times now and it would
> be
> > > worth
> > > > > > > > >> summarizing them.
> > > > > > > > >>
> > > > > > > > >> There seems to be the following points of discussion -
> > > > > > > > >>
> > > > > > > > >> 1. Can we keep the serialization semantics outside the
> > > Producer
> > > > > > > > >>interface
> > > > > > > > >> and have simple bytes in / bytes out for the interface
> (This
> > > is
> > > > > what
> > > > > > > we
> > > > > > > > >> have today).
> > > > > > > > >>
> > > > > > > > >> The points for this is to keep the interface simple and
> > usage
> > > > > easy to
> > > > > > > > >> understand. The points against this is that it gets hard
> to
> > > share
> > > > > > > common
> > > > > > > > >> usage patterns around serialization/message validations
> for
> > > the
> > > > > > > future.
> > > > > > > > >>
> > > > > > > > >> 2. Can we create a wrapper producer that does the
> > > serialization
> > > > > and
> > > > > > > have
> > > > > > > > >> different variants of it for different data formats?
> > > > > > > > >>
> > > > > > > > >> The points for this is again to keep the main API clean.
> The
> > > > > points
> > > > > > > > >> against this is that it duplicates the API, increases the
> > > surface
> > > > > area
> > > > > > > > >>and
> > > > > > > > >> creates redundancy for a minor addition.
> > > > > > > > >>
> > > > > > > > >> 3. Do we need to support different data types per record?
> > The
> > > > > current
> > > > > > > > >> interface (bytes in/bytes out) lets you instantiate one
> > > producer
> > > > > and
> > > > > > > use
> > > > > > > > >> it to send multiple data formats. There seems to be some
> > > valid use
> > > > > > > cases
> > > > > > > > >> for this.
> > > > > > > > >>
> > > > > > > > >> I have still not seen a strong argument against not having
> > > this
> > > > > > > > >> functionality. Can someone provide their views on why we
> > don't
> > > > > need
> > > > > > > this
> > > > > > > > >> support that is possible with the current API?
> > > > > > > > >>
> > > > > > > > >> One possible approach for the per record serialization
> would
> > > be to
> > > > > > > > >>define
> > > > > > > > >>
> > > > > > > > >> public interface SerDe<K,V> {
> > > > > > > > >>   public byte[] serializeKey();
> > > > > > > > >>
> > > > > > > > >>   public K deserializeKey();
> > > > > > > > >>
> > > > > > > > >>   public byte[] serializeValue();
> > > > > > > > >>
> > > > > > > > >>   public V deserializeValue();
> > > > > > > > >> }
> > > > > > > > >>
> > > > > > > > >> This would be used by both the Producer and the Consumer.
> > > > > > > > >>
> > > > > > > > >> The send APIs can then be
> > > > > > > > >>
> > > > > > > > >> public Future<RecordMetadata> send(ProducerRecord<K,V>
> > > record);
> > > > > > > > >> public Future<RecordMetadata> send(ProducerRecord<K,V>
> > record,
> > > > > > > Callback
> > > > > > > > >> callback);
> > > > > > > > >>
> > > > > > > > >>
> > > > > > > > >> public Future<RecordMetadata> send(ProducerRecord<K,V>
> > record,
> > > > > > > > >>SerDe<K,V>
> > > > > > > > >> serde);
> > > > > > > > >>
> > > > > > > > >> public Future<RecordMetadata> send(ProducerRecord<K,V>
> > record,
> > > > > > > > >>SerDe<K,V>
> > > > > > > > >> serde, Callback callback);
> > > > > > > > >>
> > > > > > > > >>
> > > > > > > > >> A default SerDe can be set in the config. The producer
> would
> > > use
> > > > > the
> > > > > > > > >> default from the config if the non-serde send APIs are
> used.
> > > The
> > > > > > > > >>downside
> > > > > > > > >> to this approach is that we would need to have four
> variants
> > > of
> > > > > Send
> > > > > > > API
> > > > > > > > >> for the Producer.
> > > > > > > > >>
> > > > > > > > >>
> > > > > > > > >>
> > > > > > > > >>
> > > > > > > > >>
> > > > > > > > >>
> > > > > > > > >> On 12/5/14 3:16 PM, "Jun Rao" <ju...@confluent.io> wrote:
> > > > > > > > >>
> > > > > > > > >> >Jiangjie,
> > > > > > > > >> >
> > > > > > > > >> >The issue with adding the serializer in ProducerRecord is
> > > that
> > > > > you
> > > > > > > > >>need to
> > > > > > > > >> >implement all combinations of serializers for key and
> > value.
> > > So,
> > > > > > > > >>instead
> > > > > > > > >> >of
> > > > > > > > >> >just implementing int and string serializers, you will
> have
> > > to
> > > > > > > > >>implement
> > > > > > > > >> >all 4 combinations.
> > > > > > > > >> >
> > > > > > > > >> >Adding a new producer constructor like Producer<K,
> > > > > > > V>(KeySerializer<K>,
> > > > > > > > >> >ValueSerializer<V>, Properties properties) can be useful.
> > > > > > > > >> >
> > > > > > > > >> >Thanks,
> > > > > > > > >> >
> > > > > > > > >> >Jun
> > > > > > > > >> >
> > > > > > > > >> >On Thu, Dec 4, 2014 at 10:33 AM, Jiangjie Qin
> > > > > > > > >><jq...@linkedin.com.invalid>
> > > > > > > > >> >wrote:
> > > > > > > > >> >
> > > > > > > > >> >>
> > > > > > > > >> >> I'm just thinking instead of binding serialization with
> > > > > producer,
> > > > > > > > >> >>another
> > > > > > > > >> >> option is to bind serializer/deserializer with
> > > > > > > > >> >> ProducerRecord/ConsumerRecord (please see the detail
> > > proposal
> > > > > > > below.)
> > > > > > > > >> >>            The arguments for this option is:
> > > > > > > > >> >>         A. A single producer could send different
> message
> > > > > types.
> > > > > > > > >>There
> > > > > > > > >> >>are
> > > > > > > > >> >> several use cases in LinkedIn for per record serializer
> > > > > > > > >> >>         - In Samza, there are some in-stream
> > > order-sensitive
> > > > > > > control
> > > > > > > > >> >> messages
> > > > > > > > >> >> having different deserializer from other messages.
> > > > > > > > >> >>         - There are use cases which need support for
> > > sending
> > > > > both
> > > > > > > > >>Avro
> > > > > > > > >> >> messages
> > > > > > > > >> >> and raw bytes.
> > > > > > > > >> >>         - Some use cases needs to deserialize some Avro
> > > > > messages
> > > > > > > into
> > > > > > > > >> >> generic
> > > > > > > > >> >> record and some other messages into specific record.
> > > > > > > > >> >>         B. In current proposal, the
> > serializer/deserilizer
> > > is
> > > > > > > > >> >>instantiated
> > > > > > > > >> >> according to config. Compared with that, binding
> > serializer
> > > > > with
> > > > > > > > >> >> ProducerRecord and ConsumerRecord is less error prone.
> > > > > > > > >> >>
> > > > > > > > >> >>
> > > > > > > > >> >>         This option includes the following changes:
> > > > > > > > >> >>         A. Add serializer and deserializer interfaces
> to
> > > > > replace
> > > > > > > > >> >>serializer
> > > > > > > > >> >> instance from config.
> > > > > > > > >> >>                 Public interface Serializer <K, V> {
> > > > > > > > >> >>                         public byte[] serializeKey(K
> > key);
> > > > > > > > >> >>                         public byte[] serializeValue(V
> > > value);
> > > > > > > > >> >>                 }
> > > > > > > > >> >>                 Public interface deserializer <K, V> {
> > > > > > > > >> >>                         Public K deserializeKey(byte[]
> > > key);
> > > > > > > > >> >>                         public V
> deserializeValue(byte[]
> > > > > value);
> > > > > > > > >> >>                 }
> > > > > > > > >> >>
> > > > > > > > >> >>         B. Make ProducerRecord and ConsumerRecord
> > abstract
> > > > > class
> > > > > > > > >> >> implementing
> > > > > > > > >> >> Serializer <K, V> and Deserializer <K, V> respectively.
> > > > > > > > >> >>                 Public abstract class ProducerRecord
> <K,
> > V>
> > > > > > > > >>implements
> > > > > > > > >> >> Serializer <K, V>
> > > > > > > > >> >> {...}
> > > > > > > > >> >>                 Public abstract class ConsumerRecord
> <K,
> > V>
> > > > > > > > >>implements
> > > > > > > > >> >> Deserializer <K,
> > > > > > > > >> >> V> {...}
> > > > > > > > >> >>
> > > > > > > > >> >>         C. Instead of instantiate the
> > > serializer/Deserializer
> > > > > from
> > > > > > > > >> >>config,
> > > > > > > > >> >> let
> > > > > > > > >> >> concrete ProducerRecord/ConsumerRecord extends the
> > abstract
> > > > > class
> > > > > > > and
> > > > > > > > >> >> override the serialize/deserialize methods.
> > > > > > > > >> >>
> > > > > > > > >> >>                 Public class AvroProducerRecord extends
> > > > > > > > >>ProducerRecord
> > > > > > > > >> >> <String,
> > > > > > > > >> >> GenericRecord> {
> > > > > > > > >> >>                         ...
> > > > > > > > >> >>                         @Override
> > > > > > > > >> >>                         Public byte[]
> serializeKey(String
> > > key)
> > > > > {Š}
> > > > > > > > >> >>                         @Override
> > > > > > > > >> >>                         public byte[]
> > > > > serializeValue(GenericRecord
> > > > > > > > >> >>value);
> > > > > > > > >> >>                 }
> > > > > > > > >> >>
> > > > > > > > >> >>                 Public class AvroConsumerRecord extends
> > > > > > > > >>ConsumerRecord
> > > > > > > > >> >> <String,
> > > > > > > > >> >> GenericRecord> {
> > > > > > > > >> >>                         ...
> > > > > > > > >> >>                         @Override
> > > > > > > > >> >>                         Public K deserializeKey(byte[]
> > > key) {Š}
> > > > > > > > >> >>                         @Override
> > > > > > > > >> >>                         public V
> deserializeValue(byte[]
> > > > > value);
> > > > > > > > >> >>                 }
> > > > > > > > >> >>
> > > > > > > > >> >>         D. The producer API changes to
> > > > > > > > >> >>                 Public class KafkaProducer {
> > > > > > > > >> >>                         ...
> > > > > > > > >> >>
> > > > > > > > >> >>                         Future<RecordMetadata> send
> > > > > (ProducerRecord
> > > > > > > > >><K,
> > > > > > > > >> >>V>
> > > > > > > > >> >> record) {
> > > > > > > > >> >>                                 ...
> > > > > > > > >> >>                                 K key =
> > > > > > > > >>record.serializeKey(record.key);
> > > > > > > > >> >>                                 V value =
> > > > > > > > >> >> record.serializedValue(record.value);
> > > > > > > > >> >>                                 BytesProducerRecord
> > > > > > > > >>bytesProducerRecord
> > > > > > > > >> >>=
> > > > > > > > >> >> new
> > > > > > > > >> >> BytesProducerRecord(topic, partition, key, value);
> > > > > > > > >> >>                                 ...
> > > > > > > > >> >>                         }
> > > > > > > > >> >>                         ...
> > > > > > > > >> >>                 }
> > > > > > > > >> >>
> > > > > > > > >> >>
> > > > > > > > >> >>
> > > > > > > > >> >> We also had some brainstorm in LinkedIn and here are
> the
> > > > > feedbacks:
> > > > > > > > >> >>
> > > > > > > > >> >> If the community decide to add the serialization back
> to
> > > new
> > > > > > > > >>producer,
> > > > > > > > >> >> besides current proposal which changes new producer API
> > to
> > > be a
> > > > > > > > >> >>template,
> > > > > > > > >> >> there are some other options raised during our
> > discussion:
> > > > > > > > >> >>         1) Rather than change current new producer API,
> > we
> > > can
> > > > > > > > >>provide a
> > > > > > > > >> >> wrapper
> > > > > > > > >> >> of current new producer (e.g. KafkaSerializedProducer)
> > and
> > > > > make it
> > > > > > > > >> >> available to users. As there is value in the simplicity
> > of
> > > > > current
> > > > > > > > >>API.
> > > > > > > > >> >>
> > > > > > > > >> >>         2) If we decide to go with tempalated new
> > producer
> > > API,
> > > > > > > > >> >>according
> > > > > > > > >> >> to
> > > > > > > > >> >> experience in LinkedIn, it might worth considering to
> > > > > instantiate
> > > > > > > the
> > > > > > > > >> >> serializer in code instead of from config so we can
> avoid
> > > > > runtime
> > > > > > > > >>errors
> > > > > > > > >> >> due to dynamic instantiation from config, which is more
> > > error
> > > > > > > prone.
> > > > > > > > >>If
> > > > > > > > >> >> that is the case, the producer API could be changed to
> > > > > something
> > > > > > > > >>like:
> > > > > > > > >> >>                 producer = new Producer<K,
> > > V>(KeySerializer<K>,
> > > > > > > > >> >> ValueSerializer<V>)
> > > > > > > > >> >>
> > > > > > > > >> >> --Jiangjie (Becket) Qin
> > > > > > > > >> >>
> > > > > > > > >> >>
> > > > > > > > >> >> On 11/24/14, 5:58 PM, "Jun Rao" <ju...@gmail.com>
> > 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
> > > > > > > > >> >>
> > > > > > > > >> >>
> > > > > > > > >>
> > > > > > > > >>
> > > > > > > >
> > > > > > >
> > > > > > >
> > > > >
> > > > >
> > >
> > >
> >
>

Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Rajiv Kurian <ra...@signalfuse.com>.
Has the mvn repo been updated too?

On Wed, Dec 17, 2014 at 4:31 PM, Jun Rao <ju...@confluent.io> wrote:
>
> Thanks everyone for the feedback and the discussion. The proposed changes
> have been checked into both 0.8.2 and trunk.
>
> Jun
>
> On Tue, Dec 16, 2014 at 10:43 PM, Joel Koshy <jj...@gmail.com> wrote:
> >
> > Jun,
> >
> > Thanks for summarizing this - it helps confirm for me that I did not
> > misunderstand anything in this thread so far; and that I disagree with
> > the premise that the steps in using the current byte-oriented API is
> > cumbersome or inflexible. It involves instantiating the K-V
> > serializers in code (as opposed to config) and a extra (but explicit
> > - i.e., making it very clear to the user) but simple call to serialize
> > before sending.
> >
> > The point about downstream queries breaking can happen just as well
> > with the implicit serializers/deserializers - since ultimately people
> > have to instantiate the specific type in their code and if they want
> > to send it they will.
> >
> > I think adoption is also equivalent since people will just instantiate
> > whatever serializer/deserializer they want in one line. Plugging in a
> > new serializer implementation does require a code change, but that can
> > also be avoided via a config driven factory.
> >
> > So I'm still +0 on the change but I'm definitely not against moving
> > forward with the changes. i.e., unless there is any strong -1 on the
> > proposal from anyone else.
> >
> > Thanks,
> >
> > Joel
> >
> > > With a byte array interface, of course there is nothing that one can't
> > do.
> > > However, the real question is that whether we want to encourage people
> to
> > > use it this way or not. Being able to flow just bytes is definitely
> > easier
> > > to get started. That's why many early adopters choose to do it that
> way.
> > > However, it's often the case that they start feeling the pain later
> when
> > > some producers change the data format. Their Hive/Pig queries start to
> > > break and it's a painful process to have the issue fixed. So, the
> purpose
> > > of this api change is really to encourage people to standardize on a
> > single
> > > serializer/deserializer that supports things like data validation and
> > > schema evolution upstream in the producer. Now, suppose there is an
> Avro
> > > serializer/deserializer implementation. How do we make it easy for
> people
> > > to adopt? If the serializer is part of the api, we can just say, wire
> in
> > > the Avro serializer for key and/or value in the config and then you can
> > > start sending Avro records to the producer. If the serializer is not
> part
> > > of the api, we have to say, first instantiate a key and/or value
> > serializer
> > > this way, send the key to the key serializer to get the key bytes, send
> > the
> > > value to the value serializer to get the value bytes, and finally send
> > the
> > > bytes to the producer. The former will be simpler and likely makes the
> > > adoption easier.
> > >
> > > Thanks,
> > >
> > > Jun
> > >
> > > On Mon, Dec 15, 2014 at 7:20 PM, Joel Koshy <jj...@gmail.com>
> wrote:
> > > >
> > > > Documentation is inevitable even if the serializer/deserializer is
> > > > part of the API - since the user has to set it up in the configs. So
> > > > again, you can only encourage people to use it through documentation.
> > > > The simpler byte-oriented API seems clearer to me because anyone who
> > > > needs to send (or receive) a specific data type will _be forced to_
> > > > (or actually, _intuitively_) select a serializer (or deserializer)
> and
> > > > will definitely pick an already available implementation if a good
> one
> > > > already exists.
> > > >
> > > > Sorry I still don't get it and this is really the only sticking point
> > > > for me, albeit a minor one (which is why I have been +0 all along on
> > > > the change). I (and I think many others) would appreciate it if
> > > > someone can help me understand this better.  So I will repeat the
> > > > question: What "usage pattern" cannot be supported by easily by the
> > > > simpler API without adding burden on the user?
> > > >
> > > > Thanks,
> > > >
> > > > Joel
> > > >
> > > > On Mon, Dec 15, 2014 at 11:59:34AM -0800, Jun Rao wrote:
> > > > > Joel,
> > > > >
> > > > > It's just that if the serializer/deserializer is not part of the
> > API, you
> > > > > can only encourage people to use it through documentation. However,
> > not
> > > > > everyone will read the documentation if it's not directly used in
> the
> > > > API.
> > > > >
> > > > > Thanks,
> > > > >
> > > > > Jun
> > > > >
> > > > > On Mon, Dec 15, 2014 at 2:11 AM, Joel Koshy <jj...@gmail.com>
> > wrote:
> > > > >
> > > > > > (sorry about the late follow-up late - I'm traveling most of this
> > > > > > month)
> > > > > >
> > > > > > I'm likely missing something obvious, but I find the following to
> > be a
> > > > > > somewhat vague point that has been mentioned more than once in
> this
> > > > > > thread without a clear explanation. i.e., why is it hard to
> share a
> > > > > > serializer/deserializer implementation and just have the clients
> > call
> > > > > > it before a send/receive? What "usage pattern" cannot be
> supported
> > by
> > > > > > the simpler API?
> > > > > >
> > > > > > > 1. Can we keep the serialization semantics outside the Producer
> > > > interface
> > > > > > > and have simple bytes in / bytes out for the interface (This is
> > what
> > > > we
> > > > > > > have today).
> > > > > > >
> > > > > > > The points for this is to keep the interface simple and usage
> > easy to
> > > > > > > understand. The points against this is that it gets hard to
> share
> > > > common
> > > > > > > usage patterns around serialization/message validations for the
> > > > future.
> > > > > >
> > > > > >
> > > > > > On Tue, Dec 09, 2014 at 03:51:08AM +0000, Sriram Subramanian
> wrote:
> > > > > > > Thank you Jay. I agree with the issue that you point w.r.t
> paired
> > > > > > > serializers. I also think having mix serialization types is
> > rare. To
> > > > get
> > > > > > > the current behavior, one can simply use a ByteArraySerializer.
> > This
> > > > is
> > > > > > > best understood by talking with many customers and you seem to
> > have
> > > > done
> > > > > > > that. I am convinced about the change.
> > > > > > >
> > > > > > > For the rest who gave -1 or 0 for this proposal, does the
> answers
> > > > for the
> > > > > > > three points(updated) below seem reasonable? Are these
> > explanations
> > > > > > > convincing?
> > > > > > >
> > > > > > >
> > > > > > > 1. Can we keep the serialization semantics outside the Producer
> > > > interface
> > > > > > > and have simple bytes in / bytes out for the interface (This is
> > what
> > > > we
> > > > > > > have today).
> > > > > > >
> > > > > > > The points for this is to keep the interface simple and usage
> > easy to
> > > > > > > understand. The points against this is that it gets hard to
> share
> > > > common
> > > > > > > usage patterns around serialization/message validations for the
> > > > future.
> > > > > > >
> > > > > > > 2. Can we create a wrapper producer that does the serialization
> > and
> > > > have
> > > > > > > different variants of it for different data formats?
> > > > > > >
> > > > > > > The points for this is again to keep the main API clean. The
> > points
> > > > > > > against this is that it duplicates the API, increases the
> surface
> > > > area
> > > > > > and
> > > > > > > creates redundancy for a minor addition.
> > > > > > >
> > > > > > > 3. Do we need to support different data types per record? The
> > current
> > > > > > > interface (bytes in/bytes out) lets you instantiate one
> producer
> > and
> > > > use
> > > > > > > it to send multiple data formats. There seems to be some valid
> > use
> > > > cases
> > > > > > > for this.
> > > > > > >
> > > > > > >
> > > > > > > Mixed serialization types are rare based on interactions with
> > > > customers.
> > > > > > > To get the current behavior, one can simply use a
> > > > ByteArraySerializer.
> > > > > > >
> > > > > > > On 12/5/14 5:00 PM, "Jay Kreps" <ja...@confluent.io> wrote:
> > > > > > >
> > > > > > > >Hey Sriram,
> > > > > > > >
> > > > > > > >Thanks! I think this is a very helpful summary.
> > > > > > > >
> > > > > > > >Let me try to address your point about passing in the serde at
> > send
> > > > > > time.
> > > > > > > >
> > > > > > > >I think the first objection is really to the paired key/value
> > > > serializer
> > > > > > > >interfaces. This leads to kind of a weird combinatorial thing
> > where
> > > > you
> > > > > > > >would have an avro/avro serializer a string/avro serializer, a
> > pb/pb
> > > > > > > >serializer, and a string/pb serializer, and so on. But your
> > proposal
> > > > > > would
> > > > > > > >work as well with separate serializers for key and value.
> > > > > > > >
> > > > > > > >I think the downside is just the one you call out--that this
> is
> > a
> > > > corner
> > > > > > > >case and you end up with two versions of all the apis to
> > support it.
> > > > > > This
> > > > > > > >also makes the serializer api more annoying to implement. I
> > think
> > > > the
> > > > > > > >alternative solution to this case and any other we can give
> > people
> > > > is
> > > > > > just
> > > > > > > >configuring ByteArraySerializer which gives you basically the
> > api
> > > > that
> > > > > > you
> > > > > > > >have now with byte arrays. If this is incredibly common then
> > this
> > > > would
> > > > > > be
> > > > > > > >a silly solution, but I guess the belief is that these cases
> are
> > > > rare
> > > > > > and
> > > > > > > >a
> > > > > > > >really well implemented avro or json serializer should be 100%
> > of
> > > > what
> > > > > > > >most
> > > > > > > >people need.
> > > > > > > >
> > > > > > > >In practice the cases that actually mix serialization types
> in a
> > > > single
> > > > > > > >stream are pretty rare I think just because the consumer then
> > has
> > > > the
> > > > > > > >problem of guessing how to deserialize, so most of these will
> > end up
> > > > > > with
> > > > > > > >at least some marker or schema id or whatever that tells you
> > how to
> > > > read
> > > > > > > >the data. Arguable this mixed serialization with marker is
> > itself a
> > > > > > > >serializer type and should have a serializer of its own...
> > > > > > > >
> > > > > > > >-Jay
> > > > > > > >
> > > > > > > >On Fri, Dec 5, 2014 at 3:48 PM, Sriram Subramanian <
> > > > > > > >srsubramanian@linkedin.com.invalid> wrote:
> > > > > > > >
> > > > > > > >> This thread has diverged multiple times now and it would be
> > worth
> > > > > > > >> summarizing them.
> > > > > > > >>
> > > > > > > >> There seems to be the following points of discussion -
> > > > > > > >>
> > > > > > > >> 1. Can we keep the serialization semantics outside the
> > Producer
> > > > > > > >>interface
> > > > > > > >> and have simple bytes in / bytes out for the interface (This
> > is
> > > > what
> > > > > > we
> > > > > > > >> have today).
> > > > > > > >>
> > > > > > > >> The points for this is to keep the interface simple and
> usage
> > > > easy to
> > > > > > > >> understand. The points against this is that it gets hard to
> > share
> > > > > > common
> > > > > > > >> usage patterns around serialization/message validations for
> > the
> > > > > > future.
> > > > > > > >>
> > > > > > > >> 2. Can we create a wrapper producer that does the
> > serialization
> > > > and
> > > > > > have
> > > > > > > >> different variants of it for different data formats?
> > > > > > > >>
> > > > > > > >> The points for this is again to keep the main API clean. The
> > > > points
> > > > > > > >> against this is that it duplicates the API, increases the
> > surface
> > > > area
> > > > > > > >>and
> > > > > > > >> creates redundancy for a minor addition.
> > > > > > > >>
> > > > > > > >> 3. Do we need to support different data types per record?
> The
> > > > current
> > > > > > > >> interface (bytes in/bytes out) lets you instantiate one
> > producer
> > > > and
> > > > > > use
> > > > > > > >> it to send multiple data formats. There seems to be some
> > valid use
> > > > > > cases
> > > > > > > >> for this.
> > > > > > > >>
> > > > > > > >> I have still not seen a strong argument against not having
> > this
> > > > > > > >> functionality. Can someone provide their views on why we
> don't
> > > > need
> > > > > > this
> > > > > > > >> support that is possible with the current API?
> > > > > > > >>
> > > > > > > >> One possible approach for the per record serialization would
> > be to
> > > > > > > >>define
> > > > > > > >>
> > > > > > > >> public interface SerDe<K,V> {
> > > > > > > >>   public byte[] serializeKey();
> > > > > > > >>
> > > > > > > >>   public K deserializeKey();
> > > > > > > >>
> > > > > > > >>   public byte[] serializeValue();
> > > > > > > >>
> > > > > > > >>   public V deserializeValue();
> > > > > > > >> }
> > > > > > > >>
> > > > > > > >> This would be used by both the Producer and the Consumer.
> > > > > > > >>
> > > > > > > >> The send APIs can then be
> > > > > > > >>
> > > > > > > >> public Future<RecordMetadata> send(ProducerRecord<K,V>
> > record);
> > > > > > > >> public Future<RecordMetadata> send(ProducerRecord<K,V>
> record,
> > > > > > Callback
> > > > > > > >> callback);
> > > > > > > >>
> > > > > > > >>
> > > > > > > >> public Future<RecordMetadata> send(ProducerRecord<K,V>
> record,
> > > > > > > >>SerDe<K,V>
> > > > > > > >> serde);
> > > > > > > >>
> > > > > > > >> public Future<RecordMetadata> send(ProducerRecord<K,V>
> record,
> > > > > > > >>SerDe<K,V>
> > > > > > > >> serde, Callback callback);
> > > > > > > >>
> > > > > > > >>
> > > > > > > >> A default SerDe can be set in the config. The producer would
> > use
> > > > the
> > > > > > > >> default from the config if the non-serde send APIs are used.
> > The
> > > > > > > >>downside
> > > > > > > >> to this approach is that we would need to have four variants
> > of
> > > > Send
> > > > > > API
> > > > > > > >> for the Producer.
> > > > > > > >>
> > > > > > > >>
> > > > > > > >>
> > > > > > > >>
> > > > > > > >>
> > > > > > > >>
> > > > > > > >> On 12/5/14 3:16 PM, "Jun Rao" <ju...@confluent.io> wrote:
> > > > > > > >>
> > > > > > > >> >Jiangjie,
> > > > > > > >> >
> > > > > > > >> >The issue with adding the serializer in ProducerRecord is
> > that
> > > > you
> > > > > > > >>need to
> > > > > > > >> >implement all combinations of serializers for key and
> value.
> > So,
> > > > > > > >>instead
> > > > > > > >> >of
> > > > > > > >> >just implementing int and string serializers, you will have
> > to
> > > > > > > >>implement
> > > > > > > >> >all 4 combinations.
> > > > > > > >> >
> > > > > > > >> >Adding a new producer constructor like Producer<K,
> > > > > > V>(KeySerializer<K>,
> > > > > > > >> >ValueSerializer<V>, Properties properties) can be useful.
> > > > > > > >> >
> > > > > > > >> >Thanks,
> > > > > > > >> >
> > > > > > > >> >Jun
> > > > > > > >> >
> > > > > > > >> >On Thu, Dec 4, 2014 at 10:33 AM, Jiangjie Qin
> > > > > > > >><jq...@linkedin.com.invalid>
> > > > > > > >> >wrote:
> > > > > > > >> >
> > > > > > > >> >>
> > > > > > > >> >> I'm just thinking instead of binding serialization with
> > > > producer,
> > > > > > > >> >>another
> > > > > > > >> >> option is to bind serializer/deserializer with
> > > > > > > >> >> ProducerRecord/ConsumerRecord (please see the detail
> > proposal
> > > > > > below.)
> > > > > > > >> >>            The arguments for this option is:
> > > > > > > >> >>         A. A single producer could send different message
> > > > types.
> > > > > > > >>There
> > > > > > > >> >>are
> > > > > > > >> >> several use cases in LinkedIn for per record serializer
> > > > > > > >> >>         - In Samza, there are some in-stream
> > order-sensitive
> > > > > > control
> > > > > > > >> >> messages
> > > > > > > >> >> having different deserializer from other messages.
> > > > > > > >> >>         - There are use cases which need support for
> > sending
> > > > both
> > > > > > > >>Avro
> > > > > > > >> >> messages
> > > > > > > >> >> and raw bytes.
> > > > > > > >> >>         - Some use cases needs to deserialize some Avro
> > > > messages
> > > > > > into
> > > > > > > >> >> generic
> > > > > > > >> >> record and some other messages into specific record.
> > > > > > > >> >>         B. In current proposal, the
> serializer/deserilizer
> > is
> > > > > > > >> >>instantiated
> > > > > > > >> >> according to config. Compared with that, binding
> serializer
> > > > with
> > > > > > > >> >> ProducerRecord and ConsumerRecord is less error prone.
> > > > > > > >> >>
> > > > > > > >> >>
> > > > > > > >> >>         This option includes the following changes:
> > > > > > > >> >>         A. Add serializer and deserializer interfaces to
> > > > replace
> > > > > > > >> >>serializer
> > > > > > > >> >> instance from config.
> > > > > > > >> >>                 Public interface Serializer <K, V> {
> > > > > > > >> >>                         public byte[] serializeKey(K
> key);
> > > > > > > >> >>                         public byte[] serializeValue(V
> > value);
> > > > > > > >> >>                 }
> > > > > > > >> >>                 Public interface deserializer <K, V> {
> > > > > > > >> >>                         Public K deserializeKey(byte[]
> > key);
> > > > > > > >> >>                         public V deserializeValue(byte[]
> > > > value);
> > > > > > > >> >>                 }
> > > > > > > >> >>
> > > > > > > >> >>         B. Make ProducerRecord and ConsumerRecord
> abstract
> > > > class
> > > > > > > >> >> implementing
> > > > > > > >> >> Serializer <K, V> and Deserializer <K, V> respectively.
> > > > > > > >> >>                 Public abstract class ProducerRecord <K,
> V>
> > > > > > > >>implements
> > > > > > > >> >> Serializer <K, V>
> > > > > > > >> >> {...}
> > > > > > > >> >>                 Public abstract class ConsumerRecord <K,
> V>
> > > > > > > >>implements
> > > > > > > >> >> Deserializer <K,
> > > > > > > >> >> V> {...}
> > > > > > > >> >>
> > > > > > > >> >>         C. Instead of instantiate the
> > serializer/Deserializer
> > > > from
> > > > > > > >> >>config,
> > > > > > > >> >> let
> > > > > > > >> >> concrete ProducerRecord/ConsumerRecord extends the
> abstract
> > > > class
> > > > > > and
> > > > > > > >> >> override the serialize/deserialize methods.
> > > > > > > >> >>
> > > > > > > >> >>                 Public class AvroProducerRecord extends
> > > > > > > >>ProducerRecord
> > > > > > > >> >> <String,
> > > > > > > >> >> GenericRecord> {
> > > > > > > >> >>                         ...
> > > > > > > >> >>                         @Override
> > > > > > > >> >>                         Public byte[] serializeKey(String
> > key)
> > > > {Š}
> > > > > > > >> >>                         @Override
> > > > > > > >> >>                         public byte[]
> > > > serializeValue(GenericRecord
> > > > > > > >> >>value);
> > > > > > > >> >>                 }
> > > > > > > >> >>
> > > > > > > >> >>                 Public class AvroConsumerRecord extends
> > > > > > > >>ConsumerRecord
> > > > > > > >> >> <String,
> > > > > > > >> >> GenericRecord> {
> > > > > > > >> >>                         ...
> > > > > > > >> >>                         @Override
> > > > > > > >> >>                         Public K deserializeKey(byte[]
> > key) {Š}
> > > > > > > >> >>                         @Override
> > > > > > > >> >>                         public V deserializeValue(byte[]
> > > > value);
> > > > > > > >> >>                 }
> > > > > > > >> >>
> > > > > > > >> >>         D. The producer API changes to
> > > > > > > >> >>                 Public class KafkaProducer {
> > > > > > > >> >>                         ...
> > > > > > > >> >>
> > > > > > > >> >>                         Future<RecordMetadata> send
> > > > (ProducerRecord
> > > > > > > >><K,
> > > > > > > >> >>V>
> > > > > > > >> >> record) {
> > > > > > > >> >>                                 ...
> > > > > > > >> >>                                 K key =
> > > > > > > >>record.serializeKey(record.key);
> > > > > > > >> >>                                 V value =
> > > > > > > >> >> record.serializedValue(record.value);
> > > > > > > >> >>                                 BytesProducerRecord
> > > > > > > >>bytesProducerRecord
> > > > > > > >> >>=
> > > > > > > >> >> new
> > > > > > > >> >> BytesProducerRecord(topic, partition, key, value);
> > > > > > > >> >>                                 ...
> > > > > > > >> >>                         }
> > > > > > > >> >>                         ...
> > > > > > > >> >>                 }
> > > > > > > >> >>
> > > > > > > >> >>
> > > > > > > >> >>
> > > > > > > >> >> We also had some brainstorm in LinkedIn and here are the
> > > > feedbacks:
> > > > > > > >> >>
> > > > > > > >> >> If the community decide to add the serialization back to
> > new
> > > > > > > >>producer,
> > > > > > > >> >> besides current proposal which changes new producer API
> to
> > be a
> > > > > > > >> >>template,
> > > > > > > >> >> there are some other options raised during our
> discussion:
> > > > > > > >> >>         1) Rather than change current new producer API,
> we
> > can
> > > > > > > >>provide a
> > > > > > > >> >> wrapper
> > > > > > > >> >> of current new producer (e.g. KafkaSerializedProducer)
> and
> > > > make it
> > > > > > > >> >> available to users. As there is value in the simplicity
> of
> > > > current
> > > > > > > >>API.
> > > > > > > >> >>
> > > > > > > >> >>         2) If we decide to go with tempalated new
> producer
> > API,
> > > > > > > >> >>according
> > > > > > > >> >> to
> > > > > > > >> >> experience in LinkedIn, it might worth considering to
> > > > instantiate
> > > > > > the
> > > > > > > >> >> serializer in code instead of from config so we can avoid
> > > > runtime
> > > > > > > >>errors
> > > > > > > >> >> due to dynamic instantiation from config, which is more
> > error
> > > > > > prone.
> > > > > > > >>If
> > > > > > > >> >> that is the case, the producer API could be changed to
> > > > something
> > > > > > > >>like:
> > > > > > > >> >>                 producer = new Producer<K,
> > V>(KeySerializer<K>,
> > > > > > > >> >> ValueSerializer<V>)
> > > > > > > >> >>
> > > > > > > >> >> --Jiangjie (Becket) Qin
> > > > > > > >> >>
> > > > > > > >> >>
> > > > > > > >> >> On 11/24/14, 5:58 PM, "Jun Rao" <ju...@gmail.com>
> 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
> > > > > > > >> >>
> > > > > > > >> >>
> > > > > > > >>
> > > > > > > >>
> > > > > > >
> > > > > >
> > > > > >
> > > >
> > > >
> >
> >
>

Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Rajiv Kurian <ra...@signalfuse.com>.
Has the mvn repo been updated too?

On Wed, Dec 17, 2014 at 4:31 PM, Jun Rao <ju...@confluent.io> wrote:
>
> Thanks everyone for the feedback and the discussion. The proposed changes
> have been checked into both 0.8.2 and trunk.
>
> Jun
>
> On Tue, Dec 16, 2014 at 10:43 PM, Joel Koshy <jj...@gmail.com> wrote:
> >
> > Jun,
> >
> > Thanks for summarizing this - it helps confirm for me that I did not
> > misunderstand anything in this thread so far; and that I disagree with
> > the premise that the steps in using the current byte-oriented API is
> > cumbersome or inflexible. It involves instantiating the K-V
> > serializers in code (as opposed to config) and a extra (but explicit
> > - i.e., making it very clear to the user) but simple call to serialize
> > before sending.
> >
> > The point about downstream queries breaking can happen just as well
> > with the implicit serializers/deserializers - since ultimately people
> > have to instantiate the specific type in their code and if they want
> > to send it they will.
> >
> > I think adoption is also equivalent since people will just instantiate
> > whatever serializer/deserializer they want in one line. Plugging in a
> > new serializer implementation does require a code change, but that can
> > also be avoided via a config driven factory.
> >
> > So I'm still +0 on the change but I'm definitely not against moving
> > forward with the changes. i.e., unless there is any strong -1 on the
> > proposal from anyone else.
> >
> > Thanks,
> >
> > Joel
> >
> > > With a byte array interface, of course there is nothing that one can't
> > do.
> > > However, the real question is that whether we want to encourage people
> to
> > > use it this way or not. Being able to flow just bytes is definitely
> > easier
> > > to get started. That's why many early adopters choose to do it that
> way.
> > > However, it's often the case that they start feeling the pain later
> when
> > > some producers change the data format. Their Hive/Pig queries start to
> > > break and it's a painful process to have the issue fixed. So, the
> purpose
> > > of this api change is really to encourage people to standardize on a
> > single
> > > serializer/deserializer that supports things like data validation and
> > > schema evolution upstream in the producer. Now, suppose there is an
> Avro
> > > serializer/deserializer implementation. How do we make it easy for
> people
> > > to adopt? If the serializer is part of the api, we can just say, wire
> in
> > > the Avro serializer for key and/or value in the config and then you can
> > > start sending Avro records to the producer. If the serializer is not
> part
> > > of the api, we have to say, first instantiate a key and/or value
> > serializer
> > > this way, send the key to the key serializer to get the key bytes, send
> > the
> > > value to the value serializer to get the value bytes, and finally send
> > the
> > > bytes to the producer. The former will be simpler and likely makes the
> > > adoption easier.
> > >
> > > Thanks,
> > >
> > > Jun
> > >
> > > On Mon, Dec 15, 2014 at 7:20 PM, Joel Koshy <jj...@gmail.com>
> wrote:
> > > >
> > > > Documentation is inevitable even if the serializer/deserializer is
> > > > part of the API - since the user has to set it up in the configs. So
> > > > again, you can only encourage people to use it through documentation.
> > > > The simpler byte-oriented API seems clearer to me because anyone who
> > > > needs to send (or receive) a specific data type will _be forced to_
> > > > (or actually, _intuitively_) select a serializer (or deserializer)
> and
> > > > will definitely pick an already available implementation if a good
> one
> > > > already exists.
> > > >
> > > > Sorry I still don't get it and this is really the only sticking point
> > > > for me, albeit a minor one (which is why I have been +0 all along on
> > > > the change). I (and I think many others) would appreciate it if
> > > > someone can help me understand this better.  So I will repeat the
> > > > question: What "usage pattern" cannot be supported by easily by the
> > > > simpler API without adding burden on the user?
> > > >
> > > > Thanks,
> > > >
> > > > Joel
> > > >
> > > > On Mon, Dec 15, 2014 at 11:59:34AM -0800, Jun Rao wrote:
> > > > > Joel,
> > > > >
> > > > > It's just that if the serializer/deserializer is not part of the
> > API, you
> > > > > can only encourage people to use it through documentation. However,
> > not
> > > > > everyone will read the documentation if it's not directly used in
> the
> > > > API.
> > > > >
> > > > > Thanks,
> > > > >
> > > > > Jun
> > > > >
> > > > > On Mon, Dec 15, 2014 at 2:11 AM, Joel Koshy <jj...@gmail.com>
> > wrote:
> > > > >
> > > > > > (sorry about the late follow-up late - I'm traveling most of this
> > > > > > month)
> > > > > >
> > > > > > I'm likely missing something obvious, but I find the following to
> > be a
> > > > > > somewhat vague point that has been mentioned more than once in
> this
> > > > > > thread without a clear explanation. i.e., why is it hard to
> share a
> > > > > > serializer/deserializer implementation and just have the clients
> > call
> > > > > > it before a send/receive? What "usage pattern" cannot be
> supported
> > by
> > > > > > the simpler API?
> > > > > >
> > > > > > > 1. Can we keep the serialization semantics outside the Producer
> > > > interface
> > > > > > > and have simple bytes in / bytes out for the interface (This is
> > what
> > > > we
> > > > > > > have today).
> > > > > > >
> > > > > > > The points for this is to keep the interface simple and usage
> > easy to
> > > > > > > understand. The points against this is that it gets hard to
> share
> > > > common
> > > > > > > usage patterns around serialization/message validations for the
> > > > future.
> > > > > >
> > > > > >
> > > > > > On Tue, Dec 09, 2014 at 03:51:08AM +0000, Sriram Subramanian
> wrote:
> > > > > > > Thank you Jay. I agree with the issue that you point w.r.t
> paired
> > > > > > > serializers. I also think having mix serialization types is
> > rare. To
> > > > get
> > > > > > > the current behavior, one can simply use a ByteArraySerializer.
> > This
> > > > is
> > > > > > > best understood by talking with many customers and you seem to
> > have
> > > > done
> > > > > > > that. I am convinced about the change.
> > > > > > >
> > > > > > > For the rest who gave -1 or 0 for this proposal, does the
> answers
> > > > for the
> > > > > > > three points(updated) below seem reasonable? Are these
> > explanations
> > > > > > > convincing?
> > > > > > >
> > > > > > >
> > > > > > > 1. Can we keep the serialization semantics outside the Producer
> > > > interface
> > > > > > > and have simple bytes in / bytes out for the interface (This is
> > what
> > > > we
> > > > > > > have today).
> > > > > > >
> > > > > > > The points for this is to keep the interface simple and usage
> > easy to
> > > > > > > understand. The points against this is that it gets hard to
> share
> > > > common
> > > > > > > usage patterns around serialization/message validations for the
> > > > future.
> > > > > > >
> > > > > > > 2. Can we create a wrapper producer that does the serialization
> > and
> > > > have
> > > > > > > different variants of it for different data formats?
> > > > > > >
> > > > > > > The points for this is again to keep the main API clean. The
> > points
> > > > > > > against this is that it duplicates the API, increases the
> surface
> > > > area
> > > > > > and
> > > > > > > creates redundancy for a minor addition.
> > > > > > >
> > > > > > > 3. Do we need to support different data types per record? The
> > current
> > > > > > > interface (bytes in/bytes out) lets you instantiate one
> producer
> > and
> > > > use
> > > > > > > it to send multiple data formats. There seems to be some valid
> > use
> > > > cases
> > > > > > > for this.
> > > > > > >
> > > > > > >
> > > > > > > Mixed serialization types are rare based on interactions with
> > > > customers.
> > > > > > > To get the current behavior, one can simply use a
> > > > ByteArraySerializer.
> > > > > > >
> > > > > > > On 12/5/14 5:00 PM, "Jay Kreps" <ja...@confluent.io> wrote:
> > > > > > >
> > > > > > > >Hey Sriram,
> > > > > > > >
> > > > > > > >Thanks! I think this is a very helpful summary.
> > > > > > > >
> > > > > > > >Let me try to address your point about passing in the serde at
> > send
> > > > > > time.
> > > > > > > >
> > > > > > > >I think the first objection is really to the paired key/value
> > > > serializer
> > > > > > > >interfaces. This leads to kind of a weird combinatorial thing
> > where
> > > > you
> > > > > > > >would have an avro/avro serializer a string/avro serializer, a
> > pb/pb
> > > > > > > >serializer, and a string/pb serializer, and so on. But your
> > proposal
> > > > > > would
> > > > > > > >work as well with separate serializers for key and value.
> > > > > > > >
> > > > > > > >I think the downside is just the one you call out--that this
> is
> > a
> > > > corner
> > > > > > > >case and you end up with two versions of all the apis to
> > support it.
> > > > > > This
> > > > > > > >also makes the serializer api more annoying to implement. I
> > think
> > > > the
> > > > > > > >alternative solution to this case and any other we can give
> > people
> > > > is
> > > > > > just
> > > > > > > >configuring ByteArraySerializer which gives you basically the
> > api
> > > > that
> > > > > > you
> > > > > > > >have now with byte arrays. If this is incredibly common then
> > this
> > > > would
> > > > > > be
> > > > > > > >a silly solution, but I guess the belief is that these cases
> are
> > > > rare
> > > > > > and
> > > > > > > >a
> > > > > > > >really well implemented avro or json serializer should be 100%
> > of
> > > > what
> > > > > > > >most
> > > > > > > >people need.
> > > > > > > >
> > > > > > > >In practice the cases that actually mix serialization types
> in a
> > > > single
> > > > > > > >stream are pretty rare I think just because the consumer then
> > has
> > > > the
> > > > > > > >problem of guessing how to deserialize, so most of these will
> > end up
> > > > > > with
> > > > > > > >at least some marker or schema id or whatever that tells you
> > how to
> > > > read
> > > > > > > >the data. Arguable this mixed serialization with marker is
> > itself a
> > > > > > > >serializer type and should have a serializer of its own...
> > > > > > > >
> > > > > > > >-Jay
> > > > > > > >
> > > > > > > >On Fri, Dec 5, 2014 at 3:48 PM, Sriram Subramanian <
> > > > > > > >srsubramanian@linkedin.com.invalid> wrote:
> > > > > > > >
> > > > > > > >> This thread has diverged multiple times now and it would be
> > worth
> > > > > > > >> summarizing them.
> > > > > > > >>
> > > > > > > >> There seems to be the following points of discussion -
> > > > > > > >>
> > > > > > > >> 1. Can we keep the serialization semantics outside the
> > Producer
> > > > > > > >>interface
> > > > > > > >> and have simple bytes in / bytes out for the interface (This
> > is
> > > > what
> > > > > > we
> > > > > > > >> have today).
> > > > > > > >>
> > > > > > > >> The points for this is to keep the interface simple and
> usage
> > > > easy to
> > > > > > > >> understand. The points against this is that it gets hard to
> > share
> > > > > > common
> > > > > > > >> usage patterns around serialization/message validations for
> > the
> > > > > > future.
> > > > > > > >>
> > > > > > > >> 2. Can we create a wrapper producer that does the
> > serialization
> > > > and
> > > > > > have
> > > > > > > >> different variants of it for different data formats?
> > > > > > > >>
> > > > > > > >> The points for this is again to keep the main API clean. The
> > > > points
> > > > > > > >> against this is that it duplicates the API, increases the
> > surface
> > > > area
> > > > > > > >>and
> > > > > > > >> creates redundancy for a minor addition.
> > > > > > > >>
> > > > > > > >> 3. Do we need to support different data types per record?
> The
> > > > current
> > > > > > > >> interface (bytes in/bytes out) lets you instantiate one
> > producer
> > > > and
> > > > > > use
> > > > > > > >> it to send multiple data formats. There seems to be some
> > valid use
> > > > > > cases
> > > > > > > >> for this.
> > > > > > > >>
> > > > > > > >> I have still not seen a strong argument against not having
> > this
> > > > > > > >> functionality. Can someone provide their views on why we
> don't
> > > > need
> > > > > > this
> > > > > > > >> support that is possible with the current API?
> > > > > > > >>
> > > > > > > >> One possible approach for the per record serialization would
> > be to
> > > > > > > >>define
> > > > > > > >>
> > > > > > > >> public interface SerDe<K,V> {
> > > > > > > >>   public byte[] serializeKey();
> > > > > > > >>
> > > > > > > >>   public K deserializeKey();
> > > > > > > >>
> > > > > > > >>   public byte[] serializeValue();
> > > > > > > >>
> > > > > > > >>   public V deserializeValue();
> > > > > > > >> }
> > > > > > > >>
> > > > > > > >> This would be used by both the Producer and the Consumer.
> > > > > > > >>
> > > > > > > >> The send APIs can then be
> > > > > > > >>
> > > > > > > >> public Future<RecordMetadata> send(ProducerRecord<K,V>
> > record);
> > > > > > > >> public Future<RecordMetadata> send(ProducerRecord<K,V>
> record,
> > > > > > Callback
> > > > > > > >> callback);
> > > > > > > >>
> > > > > > > >>
> > > > > > > >> public Future<RecordMetadata> send(ProducerRecord<K,V>
> record,
> > > > > > > >>SerDe<K,V>
> > > > > > > >> serde);
> > > > > > > >>
> > > > > > > >> public Future<RecordMetadata> send(ProducerRecord<K,V>
> record,
> > > > > > > >>SerDe<K,V>
> > > > > > > >> serde, Callback callback);
> > > > > > > >>
> > > > > > > >>
> > > > > > > >> A default SerDe can be set in the config. The producer would
> > use
> > > > the
> > > > > > > >> default from the config if the non-serde send APIs are used.
> > The
> > > > > > > >>downside
> > > > > > > >> to this approach is that we would need to have four variants
> > of
> > > > Send
> > > > > > API
> > > > > > > >> for the Producer.
> > > > > > > >>
> > > > > > > >>
> > > > > > > >>
> > > > > > > >>
> > > > > > > >>
> > > > > > > >>
> > > > > > > >> On 12/5/14 3:16 PM, "Jun Rao" <ju...@confluent.io> wrote:
> > > > > > > >>
> > > > > > > >> >Jiangjie,
> > > > > > > >> >
> > > > > > > >> >The issue with adding the serializer in ProducerRecord is
> > that
> > > > you
> > > > > > > >>need to
> > > > > > > >> >implement all combinations of serializers for key and
> value.
> > So,
> > > > > > > >>instead
> > > > > > > >> >of
> > > > > > > >> >just implementing int and string serializers, you will have
> > to
> > > > > > > >>implement
> > > > > > > >> >all 4 combinations.
> > > > > > > >> >
> > > > > > > >> >Adding a new producer constructor like Producer<K,
> > > > > > V>(KeySerializer<K>,
> > > > > > > >> >ValueSerializer<V>, Properties properties) can be useful.
> > > > > > > >> >
> > > > > > > >> >Thanks,
> > > > > > > >> >
> > > > > > > >> >Jun
> > > > > > > >> >
> > > > > > > >> >On Thu, Dec 4, 2014 at 10:33 AM, Jiangjie Qin
> > > > > > > >><jq...@linkedin.com.invalid>
> > > > > > > >> >wrote:
> > > > > > > >> >
> > > > > > > >> >>
> > > > > > > >> >> I'm just thinking instead of binding serialization with
> > > > producer,
> > > > > > > >> >>another
> > > > > > > >> >> option is to bind serializer/deserializer with
> > > > > > > >> >> ProducerRecord/ConsumerRecord (please see the detail
> > proposal
> > > > > > below.)
> > > > > > > >> >>            The arguments for this option is:
> > > > > > > >> >>         A. A single producer could send different message
> > > > types.
> > > > > > > >>There
> > > > > > > >> >>are
> > > > > > > >> >> several use cases in LinkedIn for per record serializer
> > > > > > > >> >>         - In Samza, there are some in-stream
> > order-sensitive
> > > > > > control
> > > > > > > >> >> messages
> > > > > > > >> >> having different deserializer from other messages.
> > > > > > > >> >>         - There are use cases which need support for
> > sending
> > > > both
> > > > > > > >>Avro
> > > > > > > >> >> messages
> > > > > > > >> >> and raw bytes.
> > > > > > > >> >>         - Some use cases needs to deserialize some Avro
> > > > messages
> > > > > > into
> > > > > > > >> >> generic
> > > > > > > >> >> record and some other messages into specific record.
> > > > > > > >> >>         B. In current proposal, the
> serializer/deserilizer
> > is
> > > > > > > >> >>instantiated
> > > > > > > >> >> according to config. Compared with that, binding
> serializer
> > > > with
> > > > > > > >> >> ProducerRecord and ConsumerRecord is less error prone.
> > > > > > > >> >>
> > > > > > > >> >>
> > > > > > > >> >>         This option includes the following changes:
> > > > > > > >> >>         A. Add serializer and deserializer interfaces to
> > > > replace
> > > > > > > >> >>serializer
> > > > > > > >> >> instance from config.
> > > > > > > >> >>                 Public interface Serializer <K, V> {
> > > > > > > >> >>                         public byte[] serializeKey(K
> key);
> > > > > > > >> >>                         public byte[] serializeValue(V
> > value);
> > > > > > > >> >>                 }
> > > > > > > >> >>                 Public interface deserializer <K, V> {
> > > > > > > >> >>                         Public K deserializeKey(byte[]
> > key);
> > > > > > > >> >>                         public V deserializeValue(byte[]
> > > > value);
> > > > > > > >> >>                 }
> > > > > > > >> >>
> > > > > > > >> >>         B. Make ProducerRecord and ConsumerRecord
> abstract
> > > > class
> > > > > > > >> >> implementing
> > > > > > > >> >> Serializer <K, V> and Deserializer <K, V> respectively.
> > > > > > > >> >>                 Public abstract class ProducerRecord <K,
> V>
> > > > > > > >>implements
> > > > > > > >> >> Serializer <K, V>
> > > > > > > >> >> {...}
> > > > > > > >> >>                 Public abstract class ConsumerRecord <K,
> V>
> > > > > > > >>implements
> > > > > > > >> >> Deserializer <K,
> > > > > > > >> >> V> {...}
> > > > > > > >> >>
> > > > > > > >> >>         C. Instead of instantiate the
> > serializer/Deserializer
> > > > from
> > > > > > > >> >>config,
> > > > > > > >> >> let
> > > > > > > >> >> concrete ProducerRecord/ConsumerRecord extends the
> abstract
> > > > class
> > > > > > and
> > > > > > > >> >> override the serialize/deserialize methods.
> > > > > > > >> >>
> > > > > > > >> >>                 Public class AvroProducerRecord extends
> > > > > > > >>ProducerRecord
> > > > > > > >> >> <String,
> > > > > > > >> >> GenericRecord> {
> > > > > > > >> >>                         ...
> > > > > > > >> >>                         @Override
> > > > > > > >> >>                         Public byte[] serializeKey(String
> > key)
> > > > {Š}
> > > > > > > >> >>                         @Override
> > > > > > > >> >>                         public byte[]
> > > > serializeValue(GenericRecord
> > > > > > > >> >>value);
> > > > > > > >> >>                 }
> > > > > > > >> >>
> > > > > > > >> >>                 Public class AvroConsumerRecord extends
> > > > > > > >>ConsumerRecord
> > > > > > > >> >> <String,
> > > > > > > >> >> GenericRecord> {
> > > > > > > >> >>                         ...
> > > > > > > >> >>                         @Override
> > > > > > > >> >>                         Public K deserializeKey(byte[]
> > key) {Š}
> > > > > > > >> >>                         @Override
> > > > > > > >> >>                         public V deserializeValue(byte[]
> > > > value);
> > > > > > > >> >>                 }
> > > > > > > >> >>
> > > > > > > >> >>         D. The producer API changes to
> > > > > > > >> >>                 Public class KafkaProducer {
> > > > > > > >> >>                         ...
> > > > > > > >> >>
> > > > > > > >> >>                         Future<RecordMetadata> send
> > > > (ProducerRecord
> > > > > > > >><K,
> > > > > > > >> >>V>
> > > > > > > >> >> record) {
> > > > > > > >> >>                                 ...
> > > > > > > >> >>                                 K key =
> > > > > > > >>record.serializeKey(record.key);
> > > > > > > >> >>                                 V value =
> > > > > > > >> >> record.serializedValue(record.value);
> > > > > > > >> >>                                 BytesProducerRecord
> > > > > > > >>bytesProducerRecord
> > > > > > > >> >>=
> > > > > > > >> >> new
> > > > > > > >> >> BytesProducerRecord(topic, partition, key, value);
> > > > > > > >> >>                                 ...
> > > > > > > >> >>                         }
> > > > > > > >> >>                         ...
> > > > > > > >> >>                 }
> > > > > > > >> >>
> > > > > > > >> >>
> > > > > > > >> >>
> > > > > > > >> >> We also had some brainstorm in LinkedIn and here are the
> > > > feedbacks:
> > > > > > > >> >>
> > > > > > > >> >> If the community decide to add the serialization back to
> > new
> > > > > > > >>producer,
> > > > > > > >> >> besides current proposal which changes new producer API
> to
> > be a
> > > > > > > >> >>template,
> > > > > > > >> >> there are some other options raised during our
> discussion:
> > > > > > > >> >>         1) Rather than change current new producer API,
> we
> > can
> > > > > > > >>provide a
> > > > > > > >> >> wrapper
> > > > > > > >> >> of current new producer (e.g. KafkaSerializedProducer)
> and
> > > > make it
> > > > > > > >> >> available to users. As there is value in the simplicity
> of
> > > > current
> > > > > > > >>API.
> > > > > > > >> >>
> > > > > > > >> >>         2) If we decide to go with tempalated new
> producer
> > API,
> > > > > > > >> >>according
> > > > > > > >> >> to
> > > > > > > >> >> experience in LinkedIn, it might worth considering to
> > > > instantiate
> > > > > > the
> > > > > > > >> >> serializer in code instead of from config so we can avoid
> > > > runtime
> > > > > > > >>errors
> > > > > > > >> >> due to dynamic instantiation from config, which is more
> > error
> > > > > > prone.
> > > > > > > >>If
> > > > > > > >> >> that is the case, the producer API could be changed to
> > > > something
> > > > > > > >>like:
> > > > > > > >> >>                 producer = new Producer<K,
> > V>(KeySerializer<K>,
> > > > > > > >> >> ValueSerializer<V>)
> > > > > > > >> >>
> > > > > > > >> >> --Jiangjie (Becket) Qin
> > > > > > > >> >>
> > > > > > > >> >>
> > > > > > > >> >> On 11/24/14, 5:58 PM, "Jun Rao" <ju...@gmail.com>
> 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
> > > > > > > >> >>
> > > > > > > >> >>
> > > > > > > >>
> > > > > > > >>
> > > > > > >
> > > > > >
> > > > > >
> > > >
> > > >
> >
> >
>

Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Jun Rao <ju...@confluent.io>.
Thanks everyone for the feedback and the discussion. The proposed changes
have been checked into both 0.8.2 and trunk.

Jun

On Tue, Dec 16, 2014 at 10:43 PM, Joel Koshy <jj...@gmail.com> wrote:
>
> Jun,
>
> Thanks for summarizing this - it helps confirm for me that I did not
> misunderstand anything in this thread so far; and that I disagree with
> the premise that the steps in using the current byte-oriented API is
> cumbersome or inflexible. It involves instantiating the K-V
> serializers in code (as opposed to config) and a extra (but explicit
> - i.e., making it very clear to the user) but simple call to serialize
> before sending.
>
> The point about downstream queries breaking can happen just as well
> with the implicit serializers/deserializers - since ultimately people
> have to instantiate the specific type in their code and if they want
> to send it they will.
>
> I think adoption is also equivalent since people will just instantiate
> whatever serializer/deserializer they want in one line. Plugging in a
> new serializer implementation does require a code change, but that can
> also be avoided via a config driven factory.
>
> So I'm still +0 on the change but I'm definitely not against moving
> forward with the changes. i.e., unless there is any strong -1 on the
> proposal from anyone else.
>
> Thanks,
>
> Joel
>
> > With a byte array interface, of course there is nothing that one can't
> do.
> > However, the real question is that whether we want to encourage people to
> > use it this way or not. Being able to flow just bytes is definitely
> easier
> > to get started. That's why many early adopters choose to do it that way.
> > However, it's often the case that they start feeling the pain later when
> > some producers change the data format. Their Hive/Pig queries start to
> > break and it's a painful process to have the issue fixed. So, the purpose
> > of this api change is really to encourage people to standardize on a
> single
> > serializer/deserializer that supports things like data validation and
> > schema evolution upstream in the producer. Now, suppose there is an Avro
> > serializer/deserializer implementation. How do we make it easy for people
> > to adopt? If the serializer is part of the api, we can just say, wire in
> > the Avro serializer for key and/or value in the config and then you can
> > start sending Avro records to the producer. If the serializer is not part
> > of the api, we have to say, first instantiate a key and/or value
> serializer
> > this way, send the key to the key serializer to get the key bytes, send
> the
> > value to the value serializer to get the value bytes, and finally send
> the
> > bytes to the producer. The former will be simpler and likely makes the
> > adoption easier.
> >
> > Thanks,
> >
> > Jun
> >
> > On Mon, Dec 15, 2014 at 7:20 PM, Joel Koshy <jj...@gmail.com> wrote:
> > >
> > > Documentation is inevitable even if the serializer/deserializer is
> > > part of the API - since the user has to set it up in the configs. So
> > > again, you can only encourage people to use it through documentation.
> > > The simpler byte-oriented API seems clearer to me because anyone who
> > > needs to send (or receive) a specific data type will _be forced to_
> > > (or actually, _intuitively_) select a serializer (or deserializer) and
> > > will definitely pick an already available implementation if a good one
> > > already exists.
> > >
> > > Sorry I still don't get it and this is really the only sticking point
> > > for me, albeit a minor one (which is why I have been +0 all along on
> > > the change). I (and I think many others) would appreciate it if
> > > someone can help me understand this better.  So I will repeat the
> > > question: What "usage pattern" cannot be supported by easily by the
> > > simpler API without adding burden on the user?
> > >
> > > Thanks,
> > >
> > > Joel
> > >
> > > On Mon, Dec 15, 2014 at 11:59:34AM -0800, Jun Rao wrote:
> > > > Joel,
> > > >
> > > > It's just that if the serializer/deserializer is not part of the
> API, you
> > > > can only encourage people to use it through documentation. However,
> not
> > > > everyone will read the documentation if it's not directly used in the
> > > API.
> > > >
> > > > Thanks,
> > > >
> > > > Jun
> > > >
> > > > On Mon, Dec 15, 2014 at 2:11 AM, Joel Koshy <jj...@gmail.com>
> wrote:
> > > >
> > > > > (sorry about the late follow-up late - I'm traveling most of this
> > > > > month)
> > > > >
> > > > > I'm likely missing something obvious, but I find the following to
> be a
> > > > > somewhat vague point that has been mentioned more than once in this
> > > > > thread without a clear explanation. i.e., why is it hard to share a
> > > > > serializer/deserializer implementation and just have the clients
> call
> > > > > it before a send/receive? What "usage pattern" cannot be supported
> by
> > > > > the simpler API?
> > > > >
> > > > > > 1. Can we keep the serialization semantics outside the Producer
> > > interface
> > > > > > and have simple bytes in / bytes out for the interface (This is
> what
> > > we
> > > > > > have today).
> > > > > >
> > > > > > The points for this is to keep the interface simple and usage
> easy to
> > > > > > understand. The points against this is that it gets hard to share
> > > common
> > > > > > usage patterns around serialization/message validations for the
> > > future.
> > > > >
> > > > >
> > > > > On Tue, Dec 09, 2014 at 03:51:08AM +0000, Sriram Subramanian wrote:
> > > > > > Thank you Jay. I agree with the issue that you point w.r.t paired
> > > > > > serializers. I also think having mix serialization types is
> rare. To
> > > get
> > > > > > the current behavior, one can simply use a ByteArraySerializer.
> This
> > > is
> > > > > > best understood by talking with many customers and you seem to
> have
> > > done
> > > > > > that. I am convinced about the change.
> > > > > >
> > > > > > For the rest who gave -1 or 0 for this proposal, does the answers
> > > for the
> > > > > > three points(updated) below seem reasonable? Are these
> explanations
> > > > > > convincing?
> > > > > >
> > > > > >
> > > > > > 1. Can we keep the serialization semantics outside the Producer
> > > interface
> > > > > > and have simple bytes in / bytes out for the interface (This is
> what
> > > we
> > > > > > have today).
> > > > > >
> > > > > > The points for this is to keep the interface simple and usage
> easy to
> > > > > > understand. The points against this is that it gets hard to share
> > > common
> > > > > > usage patterns around serialization/message validations for the
> > > future.
> > > > > >
> > > > > > 2. Can we create a wrapper producer that does the serialization
> and
> > > have
> > > > > > different variants of it for different data formats?
> > > > > >
> > > > > > The points for this is again to keep the main API clean. The
> points
> > > > > > against this is that it duplicates the API, increases the surface
> > > area
> > > > > and
> > > > > > creates redundancy for a minor addition.
> > > > > >
> > > > > > 3. Do we need to support different data types per record? The
> current
> > > > > > interface (bytes in/bytes out) lets you instantiate one producer
> and
> > > use
> > > > > > it to send multiple data formats. There seems to be some valid
> use
> > > cases
> > > > > > for this.
> > > > > >
> > > > > >
> > > > > > Mixed serialization types are rare based on interactions with
> > > customers.
> > > > > > To get the current behavior, one can simply use a
> > > ByteArraySerializer.
> > > > > >
> > > > > > On 12/5/14 5:00 PM, "Jay Kreps" <ja...@confluent.io> wrote:
> > > > > >
> > > > > > >Hey Sriram,
> > > > > > >
> > > > > > >Thanks! I think this is a very helpful summary.
> > > > > > >
> > > > > > >Let me try to address your point about passing in the serde at
> send
> > > > > time.
> > > > > > >
> > > > > > >I think the first objection is really to the paired key/value
> > > serializer
> > > > > > >interfaces. This leads to kind of a weird combinatorial thing
> where
> > > you
> > > > > > >would have an avro/avro serializer a string/avro serializer, a
> pb/pb
> > > > > > >serializer, and a string/pb serializer, and so on. But your
> proposal
> > > > > would
> > > > > > >work as well with separate serializers for key and value.
> > > > > > >
> > > > > > >I think the downside is just the one you call out--that this is
> a
> > > corner
> > > > > > >case and you end up with two versions of all the apis to
> support it.
> > > > > This
> > > > > > >also makes the serializer api more annoying to implement. I
> think
> > > the
> > > > > > >alternative solution to this case and any other we can give
> people
> > > is
> > > > > just
> > > > > > >configuring ByteArraySerializer which gives you basically the
> api
> > > that
> > > > > you
> > > > > > >have now with byte arrays. If this is incredibly common then
> this
> > > would
> > > > > be
> > > > > > >a silly solution, but I guess the belief is that these cases are
> > > rare
> > > > > and
> > > > > > >a
> > > > > > >really well implemented avro or json serializer should be 100%
> of
> > > what
> > > > > > >most
> > > > > > >people need.
> > > > > > >
> > > > > > >In practice the cases that actually mix serialization types in a
> > > single
> > > > > > >stream are pretty rare I think just because the consumer then
> has
> > > the
> > > > > > >problem of guessing how to deserialize, so most of these will
> end up
> > > > > with
> > > > > > >at least some marker or schema id or whatever that tells you
> how to
> > > read
> > > > > > >the data. Arguable this mixed serialization with marker is
> itself a
> > > > > > >serializer type and should have a serializer of its own...
> > > > > > >
> > > > > > >-Jay
> > > > > > >
> > > > > > >On Fri, Dec 5, 2014 at 3:48 PM, Sriram Subramanian <
> > > > > > >srsubramanian@linkedin.com.invalid> wrote:
> > > > > > >
> > > > > > >> This thread has diverged multiple times now and it would be
> worth
> > > > > > >> summarizing them.
> > > > > > >>
> > > > > > >> There seems to be the following points of discussion -
> > > > > > >>
> > > > > > >> 1. Can we keep the serialization semantics outside the
> Producer
> > > > > > >>interface
> > > > > > >> and have simple bytes in / bytes out for the interface (This
> is
> > > what
> > > > > we
> > > > > > >> have today).
> > > > > > >>
> > > > > > >> The points for this is to keep the interface simple and usage
> > > easy to
> > > > > > >> understand. The points against this is that it gets hard to
> share
> > > > > common
> > > > > > >> usage patterns around serialization/message validations for
> the
> > > > > future.
> > > > > > >>
> > > > > > >> 2. Can we create a wrapper producer that does the
> serialization
> > > and
> > > > > have
> > > > > > >> different variants of it for different data formats?
> > > > > > >>
> > > > > > >> The points for this is again to keep the main API clean. The
> > > points
> > > > > > >> against this is that it duplicates the API, increases the
> surface
> > > area
> > > > > > >>and
> > > > > > >> creates redundancy for a minor addition.
> > > > > > >>
> > > > > > >> 3. Do we need to support different data types per record? The
> > > current
> > > > > > >> interface (bytes in/bytes out) lets you instantiate one
> producer
> > > and
> > > > > use
> > > > > > >> it to send multiple data formats. There seems to be some
> valid use
> > > > > cases
> > > > > > >> for this.
> > > > > > >>
> > > > > > >> I have still not seen a strong argument against not having
> this
> > > > > > >> functionality. Can someone provide their views on why we don't
> > > need
> > > > > this
> > > > > > >> support that is possible with the current API?
> > > > > > >>
> > > > > > >> One possible approach for the per record serialization would
> be to
> > > > > > >>define
> > > > > > >>
> > > > > > >> public interface SerDe<K,V> {
> > > > > > >>   public byte[] serializeKey();
> > > > > > >>
> > > > > > >>   public K deserializeKey();
> > > > > > >>
> > > > > > >>   public byte[] serializeValue();
> > > > > > >>
> > > > > > >>   public V deserializeValue();
> > > > > > >> }
> > > > > > >>
> > > > > > >> This would be used by both the Producer and the Consumer.
> > > > > > >>
> > > > > > >> The send APIs can then be
> > > > > > >>
> > > > > > >> public Future<RecordMetadata> send(ProducerRecord<K,V>
> record);
> > > > > > >> public Future<RecordMetadata> send(ProducerRecord<K,V> record,
> > > > > Callback
> > > > > > >> callback);
> > > > > > >>
> > > > > > >>
> > > > > > >> public Future<RecordMetadata> send(ProducerRecord<K,V> record,
> > > > > > >>SerDe<K,V>
> > > > > > >> serde);
> > > > > > >>
> > > > > > >> public Future<RecordMetadata> send(ProducerRecord<K,V> record,
> > > > > > >>SerDe<K,V>
> > > > > > >> serde, Callback callback);
> > > > > > >>
> > > > > > >>
> > > > > > >> A default SerDe can be set in the config. The producer would
> use
> > > the
> > > > > > >> default from the config if the non-serde send APIs are used.
> The
> > > > > > >>downside
> > > > > > >> to this approach is that we would need to have four variants
> of
> > > Send
> > > > > API
> > > > > > >> for the Producer.
> > > > > > >>
> > > > > > >>
> > > > > > >>
> > > > > > >>
> > > > > > >>
> > > > > > >>
> > > > > > >> On 12/5/14 3:16 PM, "Jun Rao" <ju...@confluent.io> wrote:
> > > > > > >>
> > > > > > >> >Jiangjie,
> > > > > > >> >
> > > > > > >> >The issue with adding the serializer in ProducerRecord is
> that
> > > you
> > > > > > >>need to
> > > > > > >> >implement all combinations of serializers for key and value.
> So,
> > > > > > >>instead
> > > > > > >> >of
> > > > > > >> >just implementing int and string serializers, you will have
> to
> > > > > > >>implement
> > > > > > >> >all 4 combinations.
> > > > > > >> >
> > > > > > >> >Adding a new producer constructor like Producer<K,
> > > > > V>(KeySerializer<K>,
> > > > > > >> >ValueSerializer<V>, Properties properties) can be useful.
> > > > > > >> >
> > > > > > >> >Thanks,
> > > > > > >> >
> > > > > > >> >Jun
> > > > > > >> >
> > > > > > >> >On Thu, Dec 4, 2014 at 10:33 AM, Jiangjie Qin
> > > > > > >><jq...@linkedin.com.invalid>
> > > > > > >> >wrote:
> > > > > > >> >
> > > > > > >> >>
> > > > > > >> >> I'm just thinking instead of binding serialization with
> > > producer,
> > > > > > >> >>another
> > > > > > >> >> option is to bind serializer/deserializer with
> > > > > > >> >> ProducerRecord/ConsumerRecord (please see the detail
> proposal
> > > > > below.)
> > > > > > >> >>            The arguments for this option is:
> > > > > > >> >>         A. A single producer could send different message
> > > types.
> > > > > > >>There
> > > > > > >> >>are
> > > > > > >> >> several use cases in LinkedIn for per record serializer
> > > > > > >> >>         - In Samza, there are some in-stream
> order-sensitive
> > > > > control
> > > > > > >> >> messages
> > > > > > >> >> having different deserializer from other messages.
> > > > > > >> >>         - There are use cases which need support for
> sending
> > > both
> > > > > > >>Avro
> > > > > > >> >> messages
> > > > > > >> >> and raw bytes.
> > > > > > >> >>         - Some use cases needs to deserialize some Avro
> > > messages
> > > > > into
> > > > > > >> >> generic
> > > > > > >> >> record and some other messages into specific record.
> > > > > > >> >>         B. In current proposal, the serializer/deserilizer
> is
> > > > > > >> >>instantiated
> > > > > > >> >> according to config. Compared with that, binding serializer
> > > with
> > > > > > >> >> ProducerRecord and ConsumerRecord is less error prone.
> > > > > > >> >>
> > > > > > >> >>
> > > > > > >> >>         This option includes the following changes:
> > > > > > >> >>         A. Add serializer and deserializer interfaces to
> > > replace
> > > > > > >> >>serializer
> > > > > > >> >> instance from config.
> > > > > > >> >>                 Public interface Serializer <K, V> {
> > > > > > >> >>                         public byte[] serializeKey(K key);
> > > > > > >> >>                         public byte[] serializeValue(V
> value);
> > > > > > >> >>                 }
> > > > > > >> >>                 Public interface deserializer <K, V> {
> > > > > > >> >>                         Public K deserializeKey(byte[]
> key);
> > > > > > >> >>                         public V deserializeValue(byte[]
> > > value);
> > > > > > >> >>                 }
> > > > > > >> >>
> > > > > > >> >>         B. Make ProducerRecord and ConsumerRecord abstract
> > > class
> > > > > > >> >> implementing
> > > > > > >> >> Serializer <K, V> and Deserializer <K, V> respectively.
> > > > > > >> >>                 Public abstract class ProducerRecord <K, V>
> > > > > > >>implements
> > > > > > >> >> Serializer <K, V>
> > > > > > >> >> {...}
> > > > > > >> >>                 Public abstract class ConsumerRecord <K, V>
> > > > > > >>implements
> > > > > > >> >> Deserializer <K,
> > > > > > >> >> V> {...}
> > > > > > >> >>
> > > > > > >> >>         C. Instead of instantiate the
> serializer/Deserializer
> > > from
> > > > > > >> >>config,
> > > > > > >> >> let
> > > > > > >> >> concrete ProducerRecord/ConsumerRecord extends the abstract
> > > class
> > > > > and
> > > > > > >> >> override the serialize/deserialize methods.
> > > > > > >> >>
> > > > > > >> >>                 Public class AvroProducerRecord extends
> > > > > > >>ProducerRecord
> > > > > > >> >> <String,
> > > > > > >> >> GenericRecord> {
> > > > > > >> >>                         ...
> > > > > > >> >>                         @Override
> > > > > > >> >>                         Public byte[] serializeKey(String
> key)
> > > {Š}
> > > > > > >> >>                         @Override
> > > > > > >> >>                         public byte[]
> > > serializeValue(GenericRecord
> > > > > > >> >>value);
> > > > > > >> >>                 }
> > > > > > >> >>
> > > > > > >> >>                 Public class AvroConsumerRecord extends
> > > > > > >>ConsumerRecord
> > > > > > >> >> <String,
> > > > > > >> >> GenericRecord> {
> > > > > > >> >>                         ...
> > > > > > >> >>                         @Override
> > > > > > >> >>                         Public K deserializeKey(byte[]
> key) {Š}
> > > > > > >> >>                         @Override
> > > > > > >> >>                         public V deserializeValue(byte[]
> > > value);
> > > > > > >> >>                 }
> > > > > > >> >>
> > > > > > >> >>         D. The producer API changes to
> > > > > > >> >>                 Public class KafkaProducer {
> > > > > > >> >>                         ...
> > > > > > >> >>
> > > > > > >> >>                         Future<RecordMetadata> send
> > > (ProducerRecord
> > > > > > >><K,
> > > > > > >> >>V>
> > > > > > >> >> record) {
> > > > > > >> >>                                 ...
> > > > > > >> >>                                 K key =
> > > > > > >>record.serializeKey(record.key);
> > > > > > >> >>                                 V value =
> > > > > > >> >> record.serializedValue(record.value);
> > > > > > >> >>                                 BytesProducerRecord
> > > > > > >>bytesProducerRecord
> > > > > > >> >>=
> > > > > > >> >> new
> > > > > > >> >> BytesProducerRecord(topic, partition, key, value);
> > > > > > >> >>                                 ...
> > > > > > >> >>                         }
> > > > > > >> >>                         ...
> > > > > > >> >>                 }
> > > > > > >> >>
> > > > > > >> >>
> > > > > > >> >>
> > > > > > >> >> We also had some brainstorm in LinkedIn and here are the
> > > feedbacks:
> > > > > > >> >>
> > > > > > >> >> If the community decide to add the serialization back to
> new
> > > > > > >>producer,
> > > > > > >> >> besides current proposal which changes new producer API to
> be a
> > > > > > >> >>template,
> > > > > > >> >> there are some other options raised during our discussion:
> > > > > > >> >>         1) Rather than change current new producer API, we
> can
> > > > > > >>provide a
> > > > > > >> >> wrapper
> > > > > > >> >> of current new producer (e.g. KafkaSerializedProducer) and
> > > make it
> > > > > > >> >> available to users. As there is value in the simplicity of
> > > current
> > > > > > >>API.
> > > > > > >> >>
> > > > > > >> >>         2) If we decide to go with tempalated new producer
> API,
> > > > > > >> >>according
> > > > > > >> >> to
> > > > > > >> >> experience in LinkedIn, it might worth considering to
> > > instantiate
> > > > > the
> > > > > > >> >> serializer in code instead of from config so we can avoid
> > > runtime
> > > > > > >>errors
> > > > > > >> >> due to dynamic instantiation from config, which is more
> error
> > > > > prone.
> > > > > > >>If
> > > > > > >> >> that is the case, the producer API could be changed to
> > > something
> > > > > > >>like:
> > > > > > >> >>                 producer = new Producer<K,
> V>(KeySerializer<K>,
> > > > > > >> >> ValueSerializer<V>)
> > > > > > >> >>
> > > > > > >> >> --Jiangjie (Becket) Qin
> > > > > > >> >>
> > > > > > >> >>
> > > > > > >> >> On 11/24/14, 5:58 PM, "Jun Rao" <ju...@gmail.com> 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
> > > > > > >> >>
> > > > > > >> >>
> > > > > > >>
> > > > > > >>
> > > > > >
> > > > >
> > > > >
> > >
> > >
>
>

Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Jun Rao <ju...@confluent.io>.
Thanks everyone for the feedback and the discussion. The proposed changes
have been checked into both 0.8.2 and trunk.

Jun

On Tue, Dec 16, 2014 at 10:43 PM, Joel Koshy <jj...@gmail.com> wrote:
>
> Jun,
>
> Thanks for summarizing this - it helps confirm for me that I did not
> misunderstand anything in this thread so far; and that I disagree with
> the premise that the steps in using the current byte-oriented API is
> cumbersome or inflexible. It involves instantiating the K-V
> serializers in code (as opposed to config) and a extra (but explicit
> - i.e., making it very clear to the user) but simple call to serialize
> before sending.
>
> The point about downstream queries breaking can happen just as well
> with the implicit serializers/deserializers - since ultimately people
> have to instantiate the specific type in their code and if they want
> to send it they will.
>
> I think adoption is also equivalent since people will just instantiate
> whatever serializer/deserializer they want in one line. Plugging in a
> new serializer implementation does require a code change, but that can
> also be avoided via a config driven factory.
>
> So I'm still +0 on the change but I'm definitely not against moving
> forward with the changes. i.e., unless there is any strong -1 on the
> proposal from anyone else.
>
> Thanks,
>
> Joel
>
> > With a byte array interface, of course there is nothing that one can't
> do.
> > However, the real question is that whether we want to encourage people to
> > use it this way or not. Being able to flow just bytes is definitely
> easier
> > to get started. That's why many early adopters choose to do it that way.
> > However, it's often the case that they start feeling the pain later when
> > some producers change the data format. Their Hive/Pig queries start to
> > break and it's a painful process to have the issue fixed. So, the purpose
> > of this api change is really to encourage people to standardize on a
> single
> > serializer/deserializer that supports things like data validation and
> > schema evolution upstream in the producer. Now, suppose there is an Avro
> > serializer/deserializer implementation. How do we make it easy for people
> > to adopt? If the serializer is part of the api, we can just say, wire in
> > the Avro serializer for key and/or value in the config and then you can
> > start sending Avro records to the producer. If the serializer is not part
> > of the api, we have to say, first instantiate a key and/or value
> serializer
> > this way, send the key to the key serializer to get the key bytes, send
> the
> > value to the value serializer to get the value bytes, and finally send
> the
> > bytes to the producer. The former will be simpler and likely makes the
> > adoption easier.
> >
> > Thanks,
> >
> > Jun
> >
> > On Mon, Dec 15, 2014 at 7:20 PM, Joel Koshy <jj...@gmail.com> wrote:
> > >
> > > Documentation is inevitable even if the serializer/deserializer is
> > > part of the API - since the user has to set it up in the configs. So
> > > again, you can only encourage people to use it through documentation.
> > > The simpler byte-oriented API seems clearer to me because anyone who
> > > needs to send (or receive) a specific data type will _be forced to_
> > > (or actually, _intuitively_) select a serializer (or deserializer) and
> > > will definitely pick an already available implementation if a good one
> > > already exists.
> > >
> > > Sorry I still don't get it and this is really the only sticking point
> > > for me, albeit a minor one (which is why I have been +0 all along on
> > > the change). I (and I think many others) would appreciate it if
> > > someone can help me understand this better.  So I will repeat the
> > > question: What "usage pattern" cannot be supported by easily by the
> > > simpler API without adding burden on the user?
> > >
> > > Thanks,
> > >
> > > Joel
> > >
> > > On Mon, Dec 15, 2014 at 11:59:34AM -0800, Jun Rao wrote:
> > > > Joel,
> > > >
> > > > It's just that if the serializer/deserializer is not part of the
> API, you
> > > > can only encourage people to use it through documentation. However,
> not
> > > > everyone will read the documentation if it's not directly used in the
> > > API.
> > > >
> > > > Thanks,
> > > >
> > > > Jun
> > > >
> > > > On Mon, Dec 15, 2014 at 2:11 AM, Joel Koshy <jj...@gmail.com>
> wrote:
> > > >
> > > > > (sorry about the late follow-up late - I'm traveling most of this
> > > > > month)
> > > > >
> > > > > I'm likely missing something obvious, but I find the following to
> be a
> > > > > somewhat vague point that has been mentioned more than once in this
> > > > > thread without a clear explanation. i.e., why is it hard to share a
> > > > > serializer/deserializer implementation and just have the clients
> call
> > > > > it before a send/receive? What "usage pattern" cannot be supported
> by
> > > > > the simpler API?
> > > > >
> > > > > > 1. Can we keep the serialization semantics outside the Producer
> > > interface
> > > > > > and have simple bytes in / bytes out for the interface (This is
> what
> > > we
> > > > > > have today).
> > > > > >
> > > > > > The points for this is to keep the interface simple and usage
> easy to
> > > > > > understand. The points against this is that it gets hard to share
> > > common
> > > > > > usage patterns around serialization/message validations for the
> > > future.
> > > > >
> > > > >
> > > > > On Tue, Dec 09, 2014 at 03:51:08AM +0000, Sriram Subramanian wrote:
> > > > > > Thank you Jay. I agree with the issue that you point w.r.t paired
> > > > > > serializers. I also think having mix serialization types is
> rare. To
> > > get
> > > > > > the current behavior, one can simply use a ByteArraySerializer.
> This
> > > is
> > > > > > best understood by talking with many customers and you seem to
> have
> > > done
> > > > > > that. I am convinced about the change.
> > > > > >
> > > > > > For the rest who gave -1 or 0 for this proposal, does the answers
> > > for the
> > > > > > three points(updated) below seem reasonable? Are these
> explanations
> > > > > > convincing?
> > > > > >
> > > > > >
> > > > > > 1. Can we keep the serialization semantics outside the Producer
> > > interface
> > > > > > and have simple bytes in / bytes out for the interface (This is
> what
> > > we
> > > > > > have today).
> > > > > >
> > > > > > The points for this is to keep the interface simple and usage
> easy to
> > > > > > understand. The points against this is that it gets hard to share
> > > common
> > > > > > usage patterns around serialization/message validations for the
> > > future.
> > > > > >
> > > > > > 2. Can we create a wrapper producer that does the serialization
> and
> > > have
> > > > > > different variants of it for different data formats?
> > > > > >
> > > > > > The points for this is again to keep the main API clean. The
> points
> > > > > > against this is that it duplicates the API, increases the surface
> > > area
> > > > > and
> > > > > > creates redundancy for a minor addition.
> > > > > >
> > > > > > 3. Do we need to support different data types per record? The
> current
> > > > > > interface (bytes in/bytes out) lets you instantiate one producer
> and
> > > use
> > > > > > it to send multiple data formats. There seems to be some valid
> use
> > > cases
> > > > > > for this.
> > > > > >
> > > > > >
> > > > > > Mixed serialization types are rare based on interactions with
> > > customers.
> > > > > > To get the current behavior, one can simply use a
> > > ByteArraySerializer.
> > > > > >
> > > > > > On 12/5/14 5:00 PM, "Jay Kreps" <ja...@confluent.io> wrote:
> > > > > >
> > > > > > >Hey Sriram,
> > > > > > >
> > > > > > >Thanks! I think this is a very helpful summary.
> > > > > > >
> > > > > > >Let me try to address your point about passing in the serde at
> send
> > > > > time.
> > > > > > >
> > > > > > >I think the first objection is really to the paired key/value
> > > serializer
> > > > > > >interfaces. This leads to kind of a weird combinatorial thing
> where
> > > you
> > > > > > >would have an avro/avro serializer a string/avro serializer, a
> pb/pb
> > > > > > >serializer, and a string/pb serializer, and so on. But your
> proposal
> > > > > would
> > > > > > >work as well with separate serializers for key and value.
> > > > > > >
> > > > > > >I think the downside is just the one you call out--that this is
> a
> > > corner
> > > > > > >case and you end up with two versions of all the apis to
> support it.
> > > > > This
> > > > > > >also makes the serializer api more annoying to implement. I
> think
> > > the
> > > > > > >alternative solution to this case and any other we can give
> people
> > > is
> > > > > just
> > > > > > >configuring ByteArraySerializer which gives you basically the
> api
> > > that
> > > > > you
> > > > > > >have now with byte arrays. If this is incredibly common then
> this
> > > would
> > > > > be
> > > > > > >a silly solution, but I guess the belief is that these cases are
> > > rare
> > > > > and
> > > > > > >a
> > > > > > >really well implemented avro or json serializer should be 100%
> of
> > > what
> > > > > > >most
> > > > > > >people need.
> > > > > > >
> > > > > > >In practice the cases that actually mix serialization types in a
> > > single
> > > > > > >stream are pretty rare I think just because the consumer then
> has
> > > the
> > > > > > >problem of guessing how to deserialize, so most of these will
> end up
> > > > > with
> > > > > > >at least some marker or schema id or whatever that tells you
> how to
> > > read
> > > > > > >the data. Arguable this mixed serialization with marker is
> itself a
> > > > > > >serializer type and should have a serializer of its own...
> > > > > > >
> > > > > > >-Jay
> > > > > > >
> > > > > > >On Fri, Dec 5, 2014 at 3:48 PM, Sriram Subramanian <
> > > > > > >srsubramanian@linkedin.com.invalid> wrote:
> > > > > > >
> > > > > > >> This thread has diverged multiple times now and it would be
> worth
> > > > > > >> summarizing them.
> > > > > > >>
> > > > > > >> There seems to be the following points of discussion -
> > > > > > >>
> > > > > > >> 1. Can we keep the serialization semantics outside the
> Producer
> > > > > > >>interface
> > > > > > >> and have simple bytes in / bytes out for the interface (This
> is
> > > what
> > > > > we
> > > > > > >> have today).
> > > > > > >>
> > > > > > >> The points for this is to keep the interface simple and usage
> > > easy to
> > > > > > >> understand. The points against this is that it gets hard to
> share
> > > > > common
> > > > > > >> usage patterns around serialization/message validations for
> the
> > > > > future.
> > > > > > >>
> > > > > > >> 2. Can we create a wrapper producer that does the
> serialization
> > > and
> > > > > have
> > > > > > >> different variants of it for different data formats?
> > > > > > >>
> > > > > > >> The points for this is again to keep the main API clean. The
> > > points
> > > > > > >> against this is that it duplicates the API, increases the
> surface
> > > area
> > > > > > >>and
> > > > > > >> creates redundancy for a minor addition.
> > > > > > >>
> > > > > > >> 3. Do we need to support different data types per record? The
> > > current
> > > > > > >> interface (bytes in/bytes out) lets you instantiate one
> producer
> > > and
> > > > > use
> > > > > > >> it to send multiple data formats. There seems to be some
> valid use
> > > > > cases
> > > > > > >> for this.
> > > > > > >>
> > > > > > >> I have still not seen a strong argument against not having
> this
> > > > > > >> functionality. Can someone provide their views on why we don't
> > > need
> > > > > this
> > > > > > >> support that is possible with the current API?
> > > > > > >>
> > > > > > >> One possible approach for the per record serialization would
> be to
> > > > > > >>define
> > > > > > >>
> > > > > > >> public interface SerDe<K,V> {
> > > > > > >>   public byte[] serializeKey();
> > > > > > >>
> > > > > > >>   public K deserializeKey();
> > > > > > >>
> > > > > > >>   public byte[] serializeValue();
> > > > > > >>
> > > > > > >>   public V deserializeValue();
> > > > > > >> }
> > > > > > >>
> > > > > > >> This would be used by both the Producer and the Consumer.
> > > > > > >>
> > > > > > >> The send APIs can then be
> > > > > > >>
> > > > > > >> public Future<RecordMetadata> send(ProducerRecord<K,V>
> record);
> > > > > > >> public Future<RecordMetadata> send(ProducerRecord<K,V> record,
> > > > > Callback
> > > > > > >> callback);
> > > > > > >>
> > > > > > >>
> > > > > > >> public Future<RecordMetadata> send(ProducerRecord<K,V> record,
> > > > > > >>SerDe<K,V>
> > > > > > >> serde);
> > > > > > >>
> > > > > > >> public Future<RecordMetadata> send(ProducerRecord<K,V> record,
> > > > > > >>SerDe<K,V>
> > > > > > >> serde, Callback callback);
> > > > > > >>
> > > > > > >>
> > > > > > >> A default SerDe can be set in the config. The producer would
> use
> > > the
> > > > > > >> default from the config if the non-serde send APIs are used.
> The
> > > > > > >>downside
> > > > > > >> to this approach is that we would need to have four variants
> of
> > > Send
> > > > > API
> > > > > > >> for the Producer.
> > > > > > >>
> > > > > > >>
> > > > > > >>
> > > > > > >>
> > > > > > >>
> > > > > > >>
> > > > > > >> On 12/5/14 3:16 PM, "Jun Rao" <ju...@confluent.io> wrote:
> > > > > > >>
> > > > > > >> >Jiangjie,
> > > > > > >> >
> > > > > > >> >The issue with adding the serializer in ProducerRecord is
> that
> > > you
> > > > > > >>need to
> > > > > > >> >implement all combinations of serializers for key and value.
> So,
> > > > > > >>instead
> > > > > > >> >of
> > > > > > >> >just implementing int and string serializers, you will have
> to
> > > > > > >>implement
> > > > > > >> >all 4 combinations.
> > > > > > >> >
> > > > > > >> >Adding a new producer constructor like Producer<K,
> > > > > V>(KeySerializer<K>,
> > > > > > >> >ValueSerializer<V>, Properties properties) can be useful.
> > > > > > >> >
> > > > > > >> >Thanks,
> > > > > > >> >
> > > > > > >> >Jun
> > > > > > >> >
> > > > > > >> >On Thu, Dec 4, 2014 at 10:33 AM, Jiangjie Qin
> > > > > > >><jq...@linkedin.com.invalid>
> > > > > > >> >wrote:
> > > > > > >> >
> > > > > > >> >>
> > > > > > >> >> I'm just thinking instead of binding serialization with
> > > producer,
> > > > > > >> >>another
> > > > > > >> >> option is to bind serializer/deserializer with
> > > > > > >> >> ProducerRecord/ConsumerRecord (please see the detail
> proposal
> > > > > below.)
> > > > > > >> >>            The arguments for this option is:
> > > > > > >> >>         A. A single producer could send different message
> > > types.
> > > > > > >>There
> > > > > > >> >>are
> > > > > > >> >> several use cases in LinkedIn for per record serializer
> > > > > > >> >>         - In Samza, there are some in-stream
> order-sensitive
> > > > > control
> > > > > > >> >> messages
> > > > > > >> >> having different deserializer from other messages.
> > > > > > >> >>         - There are use cases which need support for
> sending
> > > both
> > > > > > >>Avro
> > > > > > >> >> messages
> > > > > > >> >> and raw bytes.
> > > > > > >> >>         - Some use cases needs to deserialize some Avro
> > > messages
> > > > > into
> > > > > > >> >> generic
> > > > > > >> >> record and some other messages into specific record.
> > > > > > >> >>         B. In current proposal, the serializer/deserilizer
> is
> > > > > > >> >>instantiated
> > > > > > >> >> according to config. Compared with that, binding serializer
> > > with
> > > > > > >> >> ProducerRecord and ConsumerRecord is less error prone.
> > > > > > >> >>
> > > > > > >> >>
> > > > > > >> >>         This option includes the following changes:
> > > > > > >> >>         A. Add serializer and deserializer interfaces to
> > > replace
> > > > > > >> >>serializer
> > > > > > >> >> instance from config.
> > > > > > >> >>                 Public interface Serializer <K, V> {
> > > > > > >> >>                         public byte[] serializeKey(K key);
> > > > > > >> >>                         public byte[] serializeValue(V
> value);
> > > > > > >> >>                 }
> > > > > > >> >>                 Public interface deserializer <K, V> {
> > > > > > >> >>                         Public K deserializeKey(byte[]
> key);
> > > > > > >> >>                         public V deserializeValue(byte[]
> > > value);
> > > > > > >> >>                 }
> > > > > > >> >>
> > > > > > >> >>         B. Make ProducerRecord and ConsumerRecord abstract
> > > class
> > > > > > >> >> implementing
> > > > > > >> >> Serializer <K, V> and Deserializer <K, V> respectively.
> > > > > > >> >>                 Public abstract class ProducerRecord <K, V>
> > > > > > >>implements
> > > > > > >> >> Serializer <K, V>
> > > > > > >> >> {...}
> > > > > > >> >>                 Public abstract class ConsumerRecord <K, V>
> > > > > > >>implements
> > > > > > >> >> Deserializer <K,
> > > > > > >> >> V> {...}
> > > > > > >> >>
> > > > > > >> >>         C. Instead of instantiate the
> serializer/Deserializer
> > > from
> > > > > > >> >>config,
> > > > > > >> >> let
> > > > > > >> >> concrete ProducerRecord/ConsumerRecord extends the abstract
> > > class
> > > > > and
> > > > > > >> >> override the serialize/deserialize methods.
> > > > > > >> >>
> > > > > > >> >>                 Public class AvroProducerRecord extends
> > > > > > >>ProducerRecord
> > > > > > >> >> <String,
> > > > > > >> >> GenericRecord> {
> > > > > > >> >>                         ...
> > > > > > >> >>                         @Override
> > > > > > >> >>                         Public byte[] serializeKey(String
> key)
> > > {Š}
> > > > > > >> >>                         @Override
> > > > > > >> >>                         public byte[]
> > > serializeValue(GenericRecord
> > > > > > >> >>value);
> > > > > > >> >>                 }
> > > > > > >> >>
> > > > > > >> >>                 Public class AvroConsumerRecord extends
> > > > > > >>ConsumerRecord
> > > > > > >> >> <String,
> > > > > > >> >> GenericRecord> {
> > > > > > >> >>                         ...
> > > > > > >> >>                         @Override
> > > > > > >> >>                         Public K deserializeKey(byte[]
> key) {Š}
> > > > > > >> >>                         @Override
> > > > > > >> >>                         public V deserializeValue(byte[]
> > > value);
> > > > > > >> >>                 }
> > > > > > >> >>
> > > > > > >> >>         D. The producer API changes to
> > > > > > >> >>                 Public class KafkaProducer {
> > > > > > >> >>                         ...
> > > > > > >> >>
> > > > > > >> >>                         Future<RecordMetadata> send
> > > (ProducerRecord
> > > > > > >><K,
> > > > > > >> >>V>
> > > > > > >> >> record) {
> > > > > > >> >>                                 ...
> > > > > > >> >>                                 K key =
> > > > > > >>record.serializeKey(record.key);
> > > > > > >> >>                                 V value =
> > > > > > >> >> record.serializedValue(record.value);
> > > > > > >> >>                                 BytesProducerRecord
> > > > > > >>bytesProducerRecord
> > > > > > >> >>=
> > > > > > >> >> new
> > > > > > >> >> BytesProducerRecord(topic, partition, key, value);
> > > > > > >> >>                                 ...
> > > > > > >> >>                         }
> > > > > > >> >>                         ...
> > > > > > >> >>                 }
> > > > > > >> >>
> > > > > > >> >>
> > > > > > >> >>
> > > > > > >> >> We also had some brainstorm in LinkedIn and here are the
> > > feedbacks:
> > > > > > >> >>
> > > > > > >> >> If the community decide to add the serialization back to
> new
> > > > > > >>producer,
> > > > > > >> >> besides current proposal which changes new producer API to
> be a
> > > > > > >> >>template,
> > > > > > >> >> there are some other options raised during our discussion:
> > > > > > >> >>         1) Rather than change current new producer API, we
> can
> > > > > > >>provide a
> > > > > > >> >> wrapper
> > > > > > >> >> of current new producer (e.g. KafkaSerializedProducer) and
> > > make it
> > > > > > >> >> available to users. As there is value in the simplicity of
> > > current
> > > > > > >>API.
> > > > > > >> >>
> > > > > > >> >>         2) If we decide to go with tempalated new producer
> API,
> > > > > > >> >>according
> > > > > > >> >> to
> > > > > > >> >> experience in LinkedIn, it might worth considering to
> > > instantiate
> > > > > the
> > > > > > >> >> serializer in code instead of from config so we can avoid
> > > runtime
> > > > > > >>errors
> > > > > > >> >> due to dynamic instantiation from config, which is more
> error
> > > > > prone.
> > > > > > >>If
> > > > > > >> >> that is the case, the producer API could be changed to
> > > something
> > > > > > >>like:
> > > > > > >> >>                 producer = new Producer<K,
> V>(KeySerializer<K>,
> > > > > > >> >> ValueSerializer<V>)
> > > > > > >> >>
> > > > > > >> >> --Jiangjie (Becket) Qin
> > > > > > >> >>
> > > > > > >> >>
> > > > > > >> >> On 11/24/14, 5:58 PM, "Jun Rao" <ju...@gmail.com> 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
> > > > > > >> >>
> > > > > > >> >>
> > > > > > >>
> > > > > > >>
> > > > > >
> > > > >
> > > > >
> > >
> > >
>
>

Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Joel Koshy <jj...@gmail.com>.
Jun,

Thanks for summarizing this - it helps confirm for me that I did not
misunderstand anything in this thread so far; and that I disagree with
the premise that the steps in using the current byte-oriented API is
cumbersome or inflexible. It involves instantiating the K-V
serializers in code (as opposed to config) and a extra (but explicit
- i.e., making it very clear to the user) but simple call to serialize
before sending.

The point about downstream queries breaking can happen just as well
with the implicit serializers/deserializers - since ultimately people
have to instantiate the specific type in their code and if they want
to send it they will.

I think adoption is also equivalent since people will just instantiate
whatever serializer/deserializer they want in one line. Plugging in a
new serializer implementation does require a code change, but that can
also be avoided via a config driven factory.

So I'm still +0 on the change but I'm definitely not against moving
forward with the changes. i.e., unless there is any strong -1 on the
proposal from anyone else.

Thanks,

Joel

> With a byte array interface, of course there is nothing that one can't do.
> However, the real question is that whether we want to encourage people to
> use it this way or not. Being able to flow just bytes is definitely easier
> to get started. That's why many early adopters choose to do it that way.
> However, it's often the case that they start feeling the pain later when
> some producers change the data format. Their Hive/Pig queries start to
> break and it's a painful process to have the issue fixed. So, the purpose
> of this api change is really to encourage people to standardize on a single
> serializer/deserializer that supports things like data validation and
> schema evolution upstream in the producer. Now, suppose there is an Avro
> serializer/deserializer implementation. How do we make it easy for people
> to adopt? If the serializer is part of the api, we can just say, wire in
> the Avro serializer for key and/or value in the config and then you can
> start sending Avro records to the producer. If the serializer is not part
> of the api, we have to say, first instantiate a key and/or value serializer
> this way, send the key to the key serializer to get the key bytes, send the
> value to the value serializer to get the value bytes, and finally send the
> bytes to the producer. The former will be simpler and likely makes the
> adoption easier.
> 
> Thanks,
> 
> Jun
> 
> On Mon, Dec 15, 2014 at 7:20 PM, Joel Koshy <jj...@gmail.com> wrote:
> >
> > Documentation is inevitable even if the serializer/deserializer is
> > part of the API - since the user has to set it up in the configs. So
> > again, you can only encourage people to use it through documentation.
> > The simpler byte-oriented API seems clearer to me because anyone who
> > needs to send (or receive) a specific data type will _be forced to_
> > (or actually, _intuitively_) select a serializer (or deserializer) and
> > will definitely pick an already available implementation if a good one
> > already exists.
> >
> > Sorry I still don't get it and this is really the only sticking point
> > for me, albeit a minor one (which is why I have been +0 all along on
> > the change). I (and I think many others) would appreciate it if
> > someone can help me understand this better.  So I will repeat the
> > question: What "usage pattern" cannot be supported by easily by the
> > simpler API without adding burden on the user?
> >
> > Thanks,
> >
> > Joel
> >
> > On Mon, Dec 15, 2014 at 11:59:34AM -0800, Jun Rao wrote:
> > > Joel,
> > >
> > > It's just that if the serializer/deserializer is not part of the API, you
> > > can only encourage people to use it through documentation. However, not
> > > everyone will read the documentation if it's not directly used in the
> > API.
> > >
> > > Thanks,
> > >
> > > Jun
> > >
> > > On Mon, Dec 15, 2014 at 2:11 AM, Joel Koshy <jj...@gmail.com> wrote:
> > >
> > > > (sorry about the late follow-up late - I'm traveling most of this
> > > > month)
> > > >
> > > > I'm likely missing something obvious, but I find the following to be a
> > > > somewhat vague point that has been mentioned more than once in this
> > > > thread without a clear explanation. i.e., why is it hard to share a
> > > > serializer/deserializer implementation and just have the clients call
> > > > it before a send/receive? What "usage pattern" cannot be supported by
> > > > the simpler API?
> > > >
> > > > > 1. Can we keep the serialization semantics outside the Producer
> > interface
> > > > > and have simple bytes in / bytes out for the interface (This is what
> > we
> > > > > have today).
> > > > >
> > > > > The points for this is to keep the interface simple and usage easy to
> > > > > understand. The points against this is that it gets hard to share
> > common
> > > > > usage patterns around serialization/message validations for the
> > future.
> > > >
> > > >
> > > > On Tue, Dec 09, 2014 at 03:51:08AM +0000, Sriram Subramanian wrote:
> > > > > Thank you Jay. I agree with the issue that you point w.r.t paired
> > > > > serializers. I also think having mix serialization types is rare. To
> > get
> > > > > the current behavior, one can simply use a ByteArraySerializer. This
> > is
> > > > > best understood by talking with many customers and you seem to have
> > done
> > > > > that. I am convinced about the change.
> > > > >
> > > > > For the rest who gave -1 or 0 for this proposal, does the answers
> > for the
> > > > > three points(updated) below seem reasonable? Are these explanations
> > > > > convincing?
> > > > >
> > > > >
> > > > > 1. Can we keep the serialization semantics outside the Producer
> > interface
> > > > > and have simple bytes in / bytes out for the interface (This is what
> > we
> > > > > have today).
> > > > >
> > > > > The points for this is to keep the interface simple and usage easy to
> > > > > understand. The points against this is that it gets hard to share
> > common
> > > > > usage patterns around serialization/message validations for the
> > future.
> > > > >
> > > > > 2. Can we create a wrapper producer that does the serialization and
> > have
> > > > > different variants of it for different data formats?
> > > > >
> > > > > The points for this is again to keep the main API clean. The points
> > > > > against this is that it duplicates the API, increases the surface
> > area
> > > > and
> > > > > creates redundancy for a minor addition.
> > > > >
> > > > > 3. Do we need to support different data types per record? The current
> > > > > interface (bytes in/bytes out) lets you instantiate one producer and
> > use
> > > > > it to send multiple data formats. There seems to be some valid use
> > cases
> > > > > for this.
> > > > >
> > > > >
> > > > > Mixed serialization types are rare based on interactions with
> > customers.
> > > > > To get the current behavior, one can simply use a
> > ByteArraySerializer.
> > > > >
> > > > > On 12/5/14 5:00 PM, "Jay Kreps" <ja...@confluent.io> wrote:
> > > > >
> > > > > >Hey Sriram,
> > > > > >
> > > > > >Thanks! I think this is a very helpful summary.
> > > > > >
> > > > > >Let me try to address your point about passing in the serde at send
> > > > time.
> > > > > >
> > > > > >I think the first objection is really to the paired key/value
> > serializer
> > > > > >interfaces. This leads to kind of a weird combinatorial thing where
> > you
> > > > > >would have an avro/avro serializer a string/avro serializer, a pb/pb
> > > > > >serializer, and a string/pb serializer, and so on. But your proposal
> > > > would
> > > > > >work as well with separate serializers for key and value.
> > > > > >
> > > > > >I think the downside is just the one you call out--that this is a
> > corner
> > > > > >case and you end up with two versions of all the apis to support it.
> > > > This
> > > > > >also makes the serializer api more annoying to implement. I think
> > the
> > > > > >alternative solution to this case and any other we can give people
> > is
> > > > just
> > > > > >configuring ByteArraySerializer which gives you basically the api
> > that
> > > > you
> > > > > >have now with byte arrays. If this is incredibly common then this
> > would
> > > > be
> > > > > >a silly solution, but I guess the belief is that these cases are
> > rare
> > > > and
> > > > > >a
> > > > > >really well implemented avro or json serializer should be 100% of
> > what
> > > > > >most
> > > > > >people need.
> > > > > >
> > > > > >In practice the cases that actually mix serialization types in a
> > single
> > > > > >stream are pretty rare I think just because the consumer then has
> > the
> > > > > >problem of guessing how to deserialize, so most of these will end up
> > > > with
> > > > > >at least some marker or schema id or whatever that tells you how to
> > read
> > > > > >the data. Arguable this mixed serialization with marker is itself a
> > > > > >serializer type and should have a serializer of its own...
> > > > > >
> > > > > >-Jay
> > > > > >
> > > > > >On Fri, Dec 5, 2014 at 3:48 PM, Sriram Subramanian <
> > > > > >srsubramanian@linkedin.com.invalid> wrote:
> > > > > >
> > > > > >> This thread has diverged multiple times now and it would be worth
> > > > > >> summarizing them.
> > > > > >>
> > > > > >> There seems to be the following points of discussion -
> > > > > >>
> > > > > >> 1. Can we keep the serialization semantics outside the Producer
> > > > > >>interface
> > > > > >> and have simple bytes in / bytes out for the interface (This is
> > what
> > > > we
> > > > > >> have today).
> > > > > >>
> > > > > >> The points for this is to keep the interface simple and usage
> > easy to
> > > > > >> understand. The points against this is that it gets hard to share
> > > > common
> > > > > >> usage patterns around serialization/message validations for the
> > > > future.
> > > > > >>
> > > > > >> 2. Can we create a wrapper producer that does the serialization
> > and
> > > > have
> > > > > >> different variants of it for different data formats?
> > > > > >>
> > > > > >> The points for this is again to keep the main API clean. The
> > points
> > > > > >> against this is that it duplicates the API, increases the surface
> > area
> > > > > >>and
> > > > > >> creates redundancy for a minor addition.
> > > > > >>
> > > > > >> 3. Do we need to support different data types per record? The
> > current
> > > > > >> interface (bytes in/bytes out) lets you instantiate one producer
> > and
> > > > use
> > > > > >> it to send multiple data formats. There seems to be some valid use
> > > > cases
> > > > > >> for this.
> > > > > >>
> > > > > >> I have still not seen a strong argument against not having this
> > > > > >> functionality. Can someone provide their views on why we don't
> > need
> > > > this
> > > > > >> support that is possible with the current API?
> > > > > >>
> > > > > >> One possible approach for the per record serialization would be to
> > > > > >>define
> > > > > >>
> > > > > >> public interface SerDe<K,V> {
> > > > > >>   public byte[] serializeKey();
> > > > > >>
> > > > > >>   public K deserializeKey();
> > > > > >>
> > > > > >>   public byte[] serializeValue();
> > > > > >>
> > > > > >>   public V deserializeValue();
> > > > > >> }
> > > > > >>
> > > > > >> This would be used by both the Producer and the Consumer.
> > > > > >>
> > > > > >> The send APIs can then be
> > > > > >>
> > > > > >> public Future<RecordMetadata> send(ProducerRecord<K,V> record);
> > > > > >> public Future<RecordMetadata> send(ProducerRecord<K,V> record,
> > > > Callback
> > > > > >> callback);
> > > > > >>
> > > > > >>
> > > > > >> public Future<RecordMetadata> send(ProducerRecord<K,V> record,
> > > > > >>SerDe<K,V>
> > > > > >> serde);
> > > > > >>
> > > > > >> public Future<RecordMetadata> send(ProducerRecord<K,V> record,
> > > > > >>SerDe<K,V>
> > > > > >> serde, Callback callback);
> > > > > >>
> > > > > >>
> > > > > >> A default SerDe can be set in the config. The producer would use
> > the
> > > > > >> default from the config if the non-serde send APIs are used. The
> > > > > >>downside
> > > > > >> to this approach is that we would need to have four variants of
> > Send
> > > > API
> > > > > >> for the Producer.
> > > > > >>
> > > > > >>
> > > > > >>
> > > > > >>
> > > > > >>
> > > > > >>
> > > > > >> On 12/5/14 3:16 PM, "Jun Rao" <ju...@confluent.io> wrote:
> > > > > >>
> > > > > >> >Jiangjie,
> > > > > >> >
> > > > > >> >The issue with adding the serializer in ProducerRecord is that
> > you
> > > > > >>need to
> > > > > >> >implement all combinations of serializers for key and value. So,
> > > > > >>instead
> > > > > >> >of
> > > > > >> >just implementing int and string serializers, you will have to
> > > > > >>implement
> > > > > >> >all 4 combinations.
> > > > > >> >
> > > > > >> >Adding a new producer constructor like Producer<K,
> > > > V>(KeySerializer<K>,
> > > > > >> >ValueSerializer<V>, Properties properties) can be useful.
> > > > > >> >
> > > > > >> >Thanks,
> > > > > >> >
> > > > > >> >Jun
> > > > > >> >
> > > > > >> >On Thu, Dec 4, 2014 at 10:33 AM, Jiangjie Qin
> > > > > >><jq...@linkedin.com.invalid>
> > > > > >> >wrote:
> > > > > >> >
> > > > > >> >>
> > > > > >> >> I'm just thinking instead of binding serialization with
> > producer,
> > > > > >> >>another
> > > > > >> >> option is to bind serializer/deserializer with
> > > > > >> >> ProducerRecord/ConsumerRecord (please see the detail proposal
> > > > below.)
> > > > > >> >>            The arguments for this option is:
> > > > > >> >>         A. A single producer could send different message
> > types.
> > > > > >>There
> > > > > >> >>are
> > > > > >> >> several use cases in LinkedIn for per record serializer
> > > > > >> >>         - In Samza, there are some in-stream order-sensitive
> > > > control
> > > > > >> >> messages
> > > > > >> >> having different deserializer from other messages.
> > > > > >> >>         - There are use cases which need support for sending
> > both
> > > > > >>Avro
> > > > > >> >> messages
> > > > > >> >> and raw bytes.
> > > > > >> >>         - Some use cases needs to deserialize some Avro
> > messages
> > > > into
> > > > > >> >> generic
> > > > > >> >> record and some other messages into specific record.
> > > > > >> >>         B. In current proposal, the serializer/deserilizer is
> > > > > >> >>instantiated
> > > > > >> >> according to config. Compared with that, binding serializer
> > with
> > > > > >> >> ProducerRecord and ConsumerRecord is less error prone.
> > > > > >> >>
> > > > > >> >>
> > > > > >> >>         This option includes the following changes:
> > > > > >> >>         A. Add serializer and deserializer interfaces to
> > replace
> > > > > >> >>serializer
> > > > > >> >> instance from config.
> > > > > >> >>                 Public interface Serializer <K, V> {
> > > > > >> >>                         public byte[] serializeKey(K key);
> > > > > >> >>                         public byte[] serializeValue(V value);
> > > > > >> >>                 }
> > > > > >> >>                 Public interface deserializer <K, V> {
> > > > > >> >>                         Public K deserializeKey(byte[] key);
> > > > > >> >>                         public V deserializeValue(byte[]
> > value);
> > > > > >> >>                 }
> > > > > >> >>
> > > > > >> >>         B. Make ProducerRecord and ConsumerRecord abstract
> > class
> > > > > >> >> implementing
> > > > > >> >> Serializer <K, V> and Deserializer <K, V> respectively.
> > > > > >> >>                 Public abstract class ProducerRecord <K, V>
> > > > > >>implements
> > > > > >> >> Serializer <K, V>
> > > > > >> >> {...}
> > > > > >> >>                 Public abstract class ConsumerRecord <K, V>
> > > > > >>implements
> > > > > >> >> Deserializer <K,
> > > > > >> >> V> {...}
> > > > > >> >>
> > > > > >> >>         C. Instead of instantiate the serializer/Deserializer
> > from
> > > > > >> >>config,
> > > > > >> >> let
> > > > > >> >> concrete ProducerRecord/ConsumerRecord extends the abstract
> > class
> > > > and
> > > > > >> >> override the serialize/deserialize methods.
> > > > > >> >>
> > > > > >> >>                 Public class AvroProducerRecord extends
> > > > > >>ProducerRecord
> > > > > >> >> <String,
> > > > > >> >> GenericRecord> {
> > > > > >> >>                         ...
> > > > > >> >>                         @Override
> > > > > >> >>                         Public byte[] serializeKey(String key)
> > {Š}
> > > > > >> >>                         @Override
> > > > > >> >>                         public byte[]
> > serializeValue(GenericRecord
> > > > > >> >>value);
> > > > > >> >>                 }
> > > > > >> >>
> > > > > >> >>                 Public class AvroConsumerRecord extends
> > > > > >>ConsumerRecord
> > > > > >> >> <String,
> > > > > >> >> GenericRecord> {
> > > > > >> >>                         ...
> > > > > >> >>                         @Override
> > > > > >> >>                         Public K deserializeKey(byte[] key) {Š}
> > > > > >> >>                         @Override
> > > > > >> >>                         public V deserializeValue(byte[]
> > value);
> > > > > >> >>                 }
> > > > > >> >>
> > > > > >> >>         D. The producer API changes to
> > > > > >> >>                 Public class KafkaProducer {
> > > > > >> >>                         ...
> > > > > >> >>
> > > > > >> >>                         Future<RecordMetadata> send
> > (ProducerRecord
> > > > > >><K,
> > > > > >> >>V>
> > > > > >> >> record) {
> > > > > >> >>                                 ...
> > > > > >> >>                                 K key =
> > > > > >>record.serializeKey(record.key);
> > > > > >> >>                                 V value =
> > > > > >> >> record.serializedValue(record.value);
> > > > > >> >>                                 BytesProducerRecord
> > > > > >>bytesProducerRecord
> > > > > >> >>=
> > > > > >> >> new
> > > > > >> >> BytesProducerRecord(topic, partition, key, value);
> > > > > >> >>                                 ...
> > > > > >> >>                         }
> > > > > >> >>                         ...
> > > > > >> >>                 }
> > > > > >> >>
> > > > > >> >>
> > > > > >> >>
> > > > > >> >> We also had some brainstorm in LinkedIn and here are the
> > feedbacks:
> > > > > >> >>
> > > > > >> >> If the community decide to add the serialization back to new
> > > > > >>producer,
> > > > > >> >> besides current proposal which changes new producer API to be a
> > > > > >> >>template,
> > > > > >> >> there are some other options raised during our discussion:
> > > > > >> >>         1) Rather than change current new producer API, we can
> > > > > >>provide a
> > > > > >> >> wrapper
> > > > > >> >> of current new producer (e.g. KafkaSerializedProducer) and
> > make it
> > > > > >> >> available to users. As there is value in the simplicity of
> > current
> > > > > >>API.
> > > > > >> >>
> > > > > >> >>         2) If we decide to go with tempalated new producer API,
> > > > > >> >>according
> > > > > >> >> to
> > > > > >> >> experience in LinkedIn, it might worth considering to
> > instantiate
> > > > the
> > > > > >> >> serializer in code instead of from config so we can avoid
> > runtime
> > > > > >>errors
> > > > > >> >> due to dynamic instantiation from config, which is more error
> > > > prone.
> > > > > >>If
> > > > > >> >> that is the case, the producer API could be changed to
> > something
> > > > > >>like:
> > > > > >> >>                 producer = new Producer<K, V>(KeySerializer<K>,
> > > > > >> >> ValueSerializer<V>)
> > > > > >> >>
> > > > > >> >> --Jiangjie (Becket) Qin
> > > > > >> >>
> > > > > >> >>
> > > > > >> >> On 11/24/14, 5:58 PM, "Jun Rao" <ju...@gmail.com> 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
> > > > > >> >>
> > > > > >> >>
> > > > > >>
> > > > > >>
> > > > >
> > > >
> > > >
> >
> >


Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Joel Koshy <jj...@gmail.com>.
Jun,

Thanks for summarizing this - it helps confirm for me that I did not
misunderstand anything in this thread so far; and that I disagree with
the premise that the steps in using the current byte-oriented API is
cumbersome or inflexible. It involves instantiating the K-V
serializers in code (as opposed to config) and a extra (but explicit
- i.e., making it very clear to the user) but simple call to serialize
before sending.

The point about downstream queries breaking can happen just as well
with the implicit serializers/deserializers - since ultimately people
have to instantiate the specific type in their code and if they want
to send it they will.

I think adoption is also equivalent since people will just instantiate
whatever serializer/deserializer they want in one line. Plugging in a
new serializer implementation does require a code change, but that can
also be avoided via a config driven factory.

So I'm still +0 on the change but I'm definitely not against moving
forward with the changes. i.e., unless there is any strong -1 on the
proposal from anyone else.

Thanks,

Joel

> With a byte array interface, of course there is nothing that one can't do.
> However, the real question is that whether we want to encourage people to
> use it this way or not. Being able to flow just bytes is definitely easier
> to get started. That's why many early adopters choose to do it that way.
> However, it's often the case that they start feeling the pain later when
> some producers change the data format. Their Hive/Pig queries start to
> break and it's a painful process to have the issue fixed. So, the purpose
> of this api change is really to encourage people to standardize on a single
> serializer/deserializer that supports things like data validation and
> schema evolution upstream in the producer. Now, suppose there is an Avro
> serializer/deserializer implementation. How do we make it easy for people
> to adopt? If the serializer is part of the api, we can just say, wire in
> the Avro serializer for key and/or value in the config and then you can
> start sending Avro records to the producer. If the serializer is not part
> of the api, we have to say, first instantiate a key and/or value serializer
> this way, send the key to the key serializer to get the key bytes, send the
> value to the value serializer to get the value bytes, and finally send the
> bytes to the producer. The former will be simpler and likely makes the
> adoption easier.
> 
> Thanks,
> 
> Jun
> 
> On Mon, Dec 15, 2014 at 7:20 PM, Joel Koshy <jj...@gmail.com> wrote:
> >
> > Documentation is inevitable even if the serializer/deserializer is
> > part of the API - since the user has to set it up in the configs. So
> > again, you can only encourage people to use it through documentation.
> > The simpler byte-oriented API seems clearer to me because anyone who
> > needs to send (or receive) a specific data type will _be forced to_
> > (or actually, _intuitively_) select a serializer (or deserializer) and
> > will definitely pick an already available implementation if a good one
> > already exists.
> >
> > Sorry I still don't get it and this is really the only sticking point
> > for me, albeit a minor one (which is why I have been +0 all along on
> > the change). I (and I think many others) would appreciate it if
> > someone can help me understand this better.  So I will repeat the
> > question: What "usage pattern" cannot be supported by easily by the
> > simpler API without adding burden on the user?
> >
> > Thanks,
> >
> > Joel
> >
> > On Mon, Dec 15, 2014 at 11:59:34AM -0800, Jun Rao wrote:
> > > Joel,
> > >
> > > It's just that if the serializer/deserializer is not part of the API, you
> > > can only encourage people to use it through documentation. However, not
> > > everyone will read the documentation if it's not directly used in the
> > API.
> > >
> > > Thanks,
> > >
> > > Jun
> > >
> > > On Mon, Dec 15, 2014 at 2:11 AM, Joel Koshy <jj...@gmail.com> wrote:
> > >
> > > > (sorry about the late follow-up late - I'm traveling most of this
> > > > month)
> > > >
> > > > I'm likely missing something obvious, but I find the following to be a
> > > > somewhat vague point that has been mentioned more than once in this
> > > > thread without a clear explanation. i.e., why is it hard to share a
> > > > serializer/deserializer implementation and just have the clients call
> > > > it before a send/receive? What "usage pattern" cannot be supported by
> > > > the simpler API?
> > > >
> > > > > 1. Can we keep the serialization semantics outside the Producer
> > interface
> > > > > and have simple bytes in / bytes out for the interface (This is what
> > we
> > > > > have today).
> > > > >
> > > > > The points for this is to keep the interface simple and usage easy to
> > > > > understand. The points against this is that it gets hard to share
> > common
> > > > > usage patterns around serialization/message validations for the
> > future.
> > > >
> > > >
> > > > On Tue, Dec 09, 2014 at 03:51:08AM +0000, Sriram Subramanian wrote:
> > > > > Thank you Jay. I agree with the issue that you point w.r.t paired
> > > > > serializers. I also think having mix serialization types is rare. To
> > get
> > > > > the current behavior, one can simply use a ByteArraySerializer. This
> > is
> > > > > best understood by talking with many customers and you seem to have
> > done
> > > > > that. I am convinced about the change.
> > > > >
> > > > > For the rest who gave -1 or 0 for this proposal, does the answers
> > for the
> > > > > three points(updated) below seem reasonable? Are these explanations
> > > > > convincing?
> > > > >
> > > > >
> > > > > 1. Can we keep the serialization semantics outside the Producer
> > interface
> > > > > and have simple bytes in / bytes out for the interface (This is what
> > we
> > > > > have today).
> > > > >
> > > > > The points for this is to keep the interface simple and usage easy to
> > > > > understand. The points against this is that it gets hard to share
> > common
> > > > > usage patterns around serialization/message validations for the
> > future.
> > > > >
> > > > > 2. Can we create a wrapper producer that does the serialization and
> > have
> > > > > different variants of it for different data formats?
> > > > >
> > > > > The points for this is again to keep the main API clean. The points
> > > > > against this is that it duplicates the API, increases the surface
> > area
> > > > and
> > > > > creates redundancy for a minor addition.
> > > > >
> > > > > 3. Do we need to support different data types per record? The current
> > > > > interface (bytes in/bytes out) lets you instantiate one producer and
> > use
> > > > > it to send multiple data formats. There seems to be some valid use
> > cases
> > > > > for this.
> > > > >
> > > > >
> > > > > Mixed serialization types are rare based on interactions with
> > customers.
> > > > > To get the current behavior, one can simply use a
> > ByteArraySerializer.
> > > > >
> > > > > On 12/5/14 5:00 PM, "Jay Kreps" <ja...@confluent.io> wrote:
> > > > >
> > > > > >Hey Sriram,
> > > > > >
> > > > > >Thanks! I think this is a very helpful summary.
> > > > > >
> > > > > >Let me try to address your point about passing in the serde at send
> > > > time.
> > > > > >
> > > > > >I think the first objection is really to the paired key/value
> > serializer
> > > > > >interfaces. This leads to kind of a weird combinatorial thing where
> > you
> > > > > >would have an avro/avro serializer a string/avro serializer, a pb/pb
> > > > > >serializer, and a string/pb serializer, and so on. But your proposal
> > > > would
> > > > > >work as well with separate serializers for key and value.
> > > > > >
> > > > > >I think the downside is just the one you call out--that this is a
> > corner
> > > > > >case and you end up with two versions of all the apis to support it.
> > > > This
> > > > > >also makes the serializer api more annoying to implement. I think
> > the
> > > > > >alternative solution to this case and any other we can give people
> > is
> > > > just
> > > > > >configuring ByteArraySerializer which gives you basically the api
> > that
> > > > you
> > > > > >have now with byte arrays. If this is incredibly common then this
> > would
> > > > be
> > > > > >a silly solution, but I guess the belief is that these cases are
> > rare
> > > > and
> > > > > >a
> > > > > >really well implemented avro or json serializer should be 100% of
> > what
> > > > > >most
> > > > > >people need.
> > > > > >
> > > > > >In practice the cases that actually mix serialization types in a
> > single
> > > > > >stream are pretty rare I think just because the consumer then has
> > the
> > > > > >problem of guessing how to deserialize, so most of these will end up
> > > > with
> > > > > >at least some marker or schema id or whatever that tells you how to
> > read
> > > > > >the data. Arguable this mixed serialization with marker is itself a
> > > > > >serializer type and should have a serializer of its own...
> > > > > >
> > > > > >-Jay
> > > > > >
> > > > > >On Fri, Dec 5, 2014 at 3:48 PM, Sriram Subramanian <
> > > > > >srsubramanian@linkedin.com.invalid> wrote:
> > > > > >
> > > > > >> This thread has diverged multiple times now and it would be worth
> > > > > >> summarizing them.
> > > > > >>
> > > > > >> There seems to be the following points of discussion -
> > > > > >>
> > > > > >> 1. Can we keep the serialization semantics outside the Producer
> > > > > >>interface
> > > > > >> and have simple bytes in / bytes out for the interface (This is
> > what
> > > > we
> > > > > >> have today).
> > > > > >>
> > > > > >> The points for this is to keep the interface simple and usage
> > easy to
> > > > > >> understand. The points against this is that it gets hard to share
> > > > common
> > > > > >> usage patterns around serialization/message validations for the
> > > > future.
> > > > > >>
> > > > > >> 2. Can we create a wrapper producer that does the serialization
> > and
> > > > have
> > > > > >> different variants of it for different data formats?
> > > > > >>
> > > > > >> The points for this is again to keep the main API clean. The
> > points
> > > > > >> against this is that it duplicates the API, increases the surface
> > area
> > > > > >>and
> > > > > >> creates redundancy for a minor addition.
> > > > > >>
> > > > > >> 3. Do we need to support different data types per record? The
> > current
> > > > > >> interface (bytes in/bytes out) lets you instantiate one producer
> > and
> > > > use
> > > > > >> it to send multiple data formats. There seems to be some valid use
> > > > cases
> > > > > >> for this.
> > > > > >>
> > > > > >> I have still not seen a strong argument against not having this
> > > > > >> functionality. Can someone provide their views on why we don't
> > need
> > > > this
> > > > > >> support that is possible with the current API?
> > > > > >>
> > > > > >> One possible approach for the per record serialization would be to
> > > > > >>define
> > > > > >>
> > > > > >> public interface SerDe<K,V> {
> > > > > >>   public byte[] serializeKey();
> > > > > >>
> > > > > >>   public K deserializeKey();
> > > > > >>
> > > > > >>   public byte[] serializeValue();
> > > > > >>
> > > > > >>   public V deserializeValue();
> > > > > >> }
> > > > > >>
> > > > > >> This would be used by both the Producer and the Consumer.
> > > > > >>
> > > > > >> The send APIs can then be
> > > > > >>
> > > > > >> public Future<RecordMetadata> send(ProducerRecord<K,V> record);
> > > > > >> public Future<RecordMetadata> send(ProducerRecord<K,V> record,
> > > > Callback
> > > > > >> callback);
> > > > > >>
> > > > > >>
> > > > > >> public Future<RecordMetadata> send(ProducerRecord<K,V> record,
> > > > > >>SerDe<K,V>
> > > > > >> serde);
> > > > > >>
> > > > > >> public Future<RecordMetadata> send(ProducerRecord<K,V> record,
> > > > > >>SerDe<K,V>
> > > > > >> serde, Callback callback);
> > > > > >>
> > > > > >>
> > > > > >> A default SerDe can be set in the config. The producer would use
> > the
> > > > > >> default from the config if the non-serde send APIs are used. The
> > > > > >>downside
> > > > > >> to this approach is that we would need to have four variants of
> > Send
> > > > API
> > > > > >> for the Producer.
> > > > > >>
> > > > > >>
> > > > > >>
> > > > > >>
> > > > > >>
> > > > > >>
> > > > > >> On 12/5/14 3:16 PM, "Jun Rao" <ju...@confluent.io> wrote:
> > > > > >>
> > > > > >> >Jiangjie,
> > > > > >> >
> > > > > >> >The issue with adding the serializer in ProducerRecord is that
> > you
> > > > > >>need to
> > > > > >> >implement all combinations of serializers for key and value. So,
> > > > > >>instead
> > > > > >> >of
> > > > > >> >just implementing int and string serializers, you will have to
> > > > > >>implement
> > > > > >> >all 4 combinations.
> > > > > >> >
> > > > > >> >Adding a new producer constructor like Producer<K,
> > > > V>(KeySerializer<K>,
> > > > > >> >ValueSerializer<V>, Properties properties) can be useful.
> > > > > >> >
> > > > > >> >Thanks,
> > > > > >> >
> > > > > >> >Jun
> > > > > >> >
> > > > > >> >On Thu, Dec 4, 2014 at 10:33 AM, Jiangjie Qin
> > > > > >><jq...@linkedin.com.invalid>
> > > > > >> >wrote:
> > > > > >> >
> > > > > >> >>
> > > > > >> >> I'm just thinking instead of binding serialization with
> > producer,
> > > > > >> >>another
> > > > > >> >> option is to bind serializer/deserializer with
> > > > > >> >> ProducerRecord/ConsumerRecord (please see the detail proposal
> > > > below.)
> > > > > >> >>            The arguments for this option is:
> > > > > >> >>         A. A single producer could send different message
> > types.
> > > > > >>There
> > > > > >> >>are
> > > > > >> >> several use cases in LinkedIn for per record serializer
> > > > > >> >>         - In Samza, there are some in-stream order-sensitive
> > > > control
> > > > > >> >> messages
> > > > > >> >> having different deserializer from other messages.
> > > > > >> >>         - There are use cases which need support for sending
> > both
> > > > > >>Avro
> > > > > >> >> messages
> > > > > >> >> and raw bytes.
> > > > > >> >>         - Some use cases needs to deserialize some Avro
> > messages
> > > > into
> > > > > >> >> generic
> > > > > >> >> record and some other messages into specific record.
> > > > > >> >>         B. In current proposal, the serializer/deserilizer is
> > > > > >> >>instantiated
> > > > > >> >> according to config. Compared with that, binding serializer
> > with
> > > > > >> >> ProducerRecord and ConsumerRecord is less error prone.
> > > > > >> >>
> > > > > >> >>
> > > > > >> >>         This option includes the following changes:
> > > > > >> >>         A. Add serializer and deserializer interfaces to
> > replace
> > > > > >> >>serializer
> > > > > >> >> instance from config.
> > > > > >> >>                 Public interface Serializer <K, V> {
> > > > > >> >>                         public byte[] serializeKey(K key);
> > > > > >> >>                         public byte[] serializeValue(V value);
> > > > > >> >>                 }
> > > > > >> >>                 Public interface deserializer <K, V> {
> > > > > >> >>                         Public K deserializeKey(byte[] key);
> > > > > >> >>                         public V deserializeValue(byte[]
> > value);
> > > > > >> >>                 }
> > > > > >> >>
> > > > > >> >>         B. Make ProducerRecord and ConsumerRecord abstract
> > class
> > > > > >> >> implementing
> > > > > >> >> Serializer <K, V> and Deserializer <K, V> respectively.
> > > > > >> >>                 Public abstract class ProducerRecord <K, V>
> > > > > >>implements
> > > > > >> >> Serializer <K, V>
> > > > > >> >> {...}
> > > > > >> >>                 Public abstract class ConsumerRecord <K, V>
> > > > > >>implements
> > > > > >> >> Deserializer <K,
> > > > > >> >> V> {...}
> > > > > >> >>
> > > > > >> >>         C. Instead of instantiate the serializer/Deserializer
> > from
> > > > > >> >>config,
> > > > > >> >> let
> > > > > >> >> concrete ProducerRecord/ConsumerRecord extends the abstract
> > class
> > > > and
> > > > > >> >> override the serialize/deserialize methods.
> > > > > >> >>
> > > > > >> >>                 Public class AvroProducerRecord extends
> > > > > >>ProducerRecord
> > > > > >> >> <String,
> > > > > >> >> GenericRecord> {
> > > > > >> >>                         ...
> > > > > >> >>                         @Override
> > > > > >> >>                         Public byte[] serializeKey(String key)
> > {Š}
> > > > > >> >>                         @Override
> > > > > >> >>                         public byte[]
> > serializeValue(GenericRecord
> > > > > >> >>value);
> > > > > >> >>                 }
> > > > > >> >>
> > > > > >> >>                 Public class AvroConsumerRecord extends
> > > > > >>ConsumerRecord
> > > > > >> >> <String,
> > > > > >> >> GenericRecord> {
> > > > > >> >>                         ...
> > > > > >> >>                         @Override
> > > > > >> >>                         Public K deserializeKey(byte[] key) {Š}
> > > > > >> >>                         @Override
> > > > > >> >>                         public V deserializeValue(byte[]
> > value);
> > > > > >> >>                 }
> > > > > >> >>
> > > > > >> >>         D. The producer API changes to
> > > > > >> >>                 Public class KafkaProducer {
> > > > > >> >>                         ...
> > > > > >> >>
> > > > > >> >>                         Future<RecordMetadata> send
> > (ProducerRecord
> > > > > >><K,
> > > > > >> >>V>
> > > > > >> >> record) {
> > > > > >> >>                                 ...
> > > > > >> >>                                 K key =
> > > > > >>record.serializeKey(record.key);
> > > > > >> >>                                 V value =
> > > > > >> >> record.serializedValue(record.value);
> > > > > >> >>                                 BytesProducerRecord
> > > > > >>bytesProducerRecord
> > > > > >> >>=
> > > > > >> >> new
> > > > > >> >> BytesProducerRecord(topic, partition, key, value);
> > > > > >> >>                                 ...
> > > > > >> >>                         }
> > > > > >> >>                         ...
> > > > > >> >>                 }
> > > > > >> >>
> > > > > >> >>
> > > > > >> >>
> > > > > >> >> We also had some brainstorm in LinkedIn and here are the
> > feedbacks:
> > > > > >> >>
> > > > > >> >> If the community decide to add the serialization back to new
> > > > > >>producer,
> > > > > >> >> besides current proposal which changes new producer API to be a
> > > > > >> >>template,
> > > > > >> >> there are some other options raised during our discussion:
> > > > > >> >>         1) Rather than change current new producer API, we can
> > > > > >>provide a
> > > > > >> >> wrapper
> > > > > >> >> of current new producer (e.g. KafkaSerializedProducer) and
> > make it
> > > > > >> >> available to users. As there is value in the simplicity of
> > current
> > > > > >>API.
> > > > > >> >>
> > > > > >> >>         2) If we decide to go with tempalated new producer API,
> > > > > >> >>according
> > > > > >> >> to
> > > > > >> >> experience in LinkedIn, it might worth considering to
> > instantiate
> > > > the
> > > > > >> >> serializer in code instead of from config so we can avoid
> > runtime
> > > > > >>errors
> > > > > >> >> due to dynamic instantiation from config, which is more error
> > > > prone.
> > > > > >>If
> > > > > >> >> that is the case, the producer API could be changed to
> > something
> > > > > >>like:
> > > > > >> >>                 producer = new Producer<K, V>(KeySerializer<K>,
> > > > > >> >> ValueSerializer<V>)
> > > > > >> >>
> > > > > >> >> --Jiangjie (Becket) Qin
> > > > > >> >>
> > > > > >> >>
> > > > > >> >> On 11/24/14, 5:58 PM, "Jun Rao" <ju...@gmail.com> 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
> > > > > >> >>
> > > > > >> >>
> > > > > >>
> > > > > >>
> > > > >
> > > >
> > > >
> >
> >


Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Jun Rao <ju...@confluent.io>.
Joel,

With a byte array interface, of course there is nothing that one can't do.
However, the real question is that whether we want to encourage people to
use it this way or not. Being able to flow just bytes is definitely easier
to get started. That's why many early adopters choose to do it that way.
However, it's often the case that they start feeling the pain later when
some producers change the data format. Their Hive/Pig queries start to
break and it's a painful process to have the issue fixed. So, the purpose
of this api change is really to encourage people to standardize on a single
serializer/deserializer that supports things like data validation and
schema evolution upstream in the producer. Now, suppose there is an Avro
serializer/deserializer implementation. How do we make it easy for people
to adopt? If the serializer is part of the api, we can just say, wire in
the Avro serializer for key and/or value in the config and then you can
start sending Avro records to the producer. If the serializer is not part
of the api, we have to say, first instantiate a key and/or value serializer
this way, send the key to the key serializer to get the key bytes, send the
value to the value serializer to get the value bytes, and finally send the
bytes to the producer. The former will be simpler and likely makes the
adoption easier.

Thanks,

Jun

On Mon, Dec 15, 2014 at 7:20 PM, Joel Koshy <jj...@gmail.com> wrote:
>
> Documentation is inevitable even if the serializer/deserializer is
> part of the API - since the user has to set it up in the configs. So
> again, you can only encourage people to use it through documentation.
> The simpler byte-oriented API seems clearer to me because anyone who
> needs to send (or receive) a specific data type will _be forced to_
> (or actually, _intuitively_) select a serializer (or deserializer) and
> will definitely pick an already available implementation if a good one
> already exists.
>
> Sorry I still don't get it and this is really the only sticking point
> for me, albeit a minor one (which is why I have been +0 all along on
> the change). I (and I think many others) would appreciate it if
> someone can help me understand this better.  So I will repeat the
> question: What "usage pattern" cannot be supported by easily by the
> simpler API without adding burden on the user?
>
> Thanks,
>
> Joel
>
> On Mon, Dec 15, 2014 at 11:59:34AM -0800, Jun Rao wrote:
> > Joel,
> >
> > It's just that if the serializer/deserializer is not part of the API, you
> > can only encourage people to use it through documentation. However, not
> > everyone will read the documentation if it's not directly used in the
> API.
> >
> > Thanks,
> >
> > Jun
> >
> > On Mon, Dec 15, 2014 at 2:11 AM, Joel Koshy <jj...@gmail.com> wrote:
> >
> > > (sorry about the late follow-up late - I'm traveling most of this
> > > month)
> > >
> > > I'm likely missing something obvious, but I find the following to be a
> > > somewhat vague point that has been mentioned more than once in this
> > > thread without a clear explanation. i.e., why is it hard to share a
> > > serializer/deserializer implementation and just have the clients call
> > > it before a send/receive? What "usage pattern" cannot be supported by
> > > the simpler API?
> > >
> > > > 1. Can we keep the serialization semantics outside the Producer
> interface
> > > > and have simple bytes in / bytes out for the interface (This is what
> we
> > > > have today).
> > > >
> > > > The points for this is to keep the interface simple and usage easy to
> > > > understand. The points against this is that it gets hard to share
> common
> > > > usage patterns around serialization/message validations for the
> future.
> > >
> > >
> > > On Tue, Dec 09, 2014 at 03:51:08AM +0000, Sriram Subramanian wrote:
> > > > Thank you Jay. I agree with the issue that you point w.r.t paired
> > > > serializers. I also think having mix serialization types is rare. To
> get
> > > > the current behavior, one can simply use a ByteArraySerializer. This
> is
> > > > best understood by talking with many customers and you seem to have
> done
> > > > that. I am convinced about the change.
> > > >
> > > > For the rest who gave -1 or 0 for this proposal, does the answers
> for the
> > > > three points(updated) below seem reasonable? Are these explanations
> > > > convincing?
> > > >
> > > >
> > > > 1. Can we keep the serialization semantics outside the Producer
> interface
> > > > and have simple bytes in / bytes out for the interface (This is what
> we
> > > > have today).
> > > >
> > > > The points for this is to keep the interface simple and usage easy to
> > > > understand. The points against this is that it gets hard to share
> common
> > > > usage patterns around serialization/message validations for the
> future.
> > > >
> > > > 2. Can we create a wrapper producer that does the serialization and
> have
> > > > different variants of it for different data formats?
> > > >
> > > > The points for this is again to keep the main API clean. The points
> > > > against this is that it duplicates the API, increases the surface
> area
> > > and
> > > > creates redundancy for a minor addition.
> > > >
> > > > 3. Do we need to support different data types per record? The current
> > > > interface (bytes in/bytes out) lets you instantiate one producer and
> use
> > > > it to send multiple data formats. There seems to be some valid use
> cases
> > > > for this.
> > > >
> > > >
> > > > Mixed serialization types are rare based on interactions with
> customers.
> > > > To get the current behavior, one can simply use a
> ByteArraySerializer.
> > > >
> > > > On 12/5/14 5:00 PM, "Jay Kreps" <ja...@confluent.io> wrote:
> > > >
> > > > >Hey Sriram,
> > > > >
> > > > >Thanks! I think this is a very helpful summary.
> > > > >
> > > > >Let me try to address your point about passing in the serde at send
> > > time.
> > > > >
> > > > >I think the first objection is really to the paired key/value
> serializer
> > > > >interfaces. This leads to kind of a weird combinatorial thing where
> you
> > > > >would have an avro/avro serializer a string/avro serializer, a pb/pb
> > > > >serializer, and a string/pb serializer, and so on. But your proposal
> > > would
> > > > >work as well with separate serializers for key and value.
> > > > >
> > > > >I think the downside is just the one you call out--that this is a
> corner
> > > > >case and you end up with two versions of all the apis to support it.
> > > This
> > > > >also makes the serializer api more annoying to implement. I think
> the
> > > > >alternative solution to this case and any other we can give people
> is
> > > just
> > > > >configuring ByteArraySerializer which gives you basically the api
> that
> > > you
> > > > >have now with byte arrays. If this is incredibly common then this
> would
> > > be
> > > > >a silly solution, but I guess the belief is that these cases are
> rare
> > > and
> > > > >a
> > > > >really well implemented avro or json serializer should be 100% of
> what
> > > > >most
> > > > >people need.
> > > > >
> > > > >In practice the cases that actually mix serialization types in a
> single
> > > > >stream are pretty rare I think just because the consumer then has
> the
> > > > >problem of guessing how to deserialize, so most of these will end up
> > > with
> > > > >at least some marker or schema id or whatever that tells you how to
> read
> > > > >the data. Arguable this mixed serialization with marker is itself a
> > > > >serializer type and should have a serializer of its own...
> > > > >
> > > > >-Jay
> > > > >
> > > > >On Fri, Dec 5, 2014 at 3:48 PM, Sriram Subramanian <
> > > > >srsubramanian@linkedin.com.invalid> wrote:
> > > > >
> > > > >> This thread has diverged multiple times now and it would be worth
> > > > >> summarizing them.
> > > > >>
> > > > >> There seems to be the following points of discussion -
> > > > >>
> > > > >> 1. Can we keep the serialization semantics outside the Producer
> > > > >>interface
> > > > >> and have simple bytes in / bytes out for the interface (This is
> what
> > > we
> > > > >> have today).
> > > > >>
> > > > >> The points for this is to keep the interface simple and usage
> easy to
> > > > >> understand. The points against this is that it gets hard to share
> > > common
> > > > >> usage patterns around serialization/message validations for the
> > > future.
> > > > >>
> > > > >> 2. Can we create a wrapper producer that does the serialization
> and
> > > have
> > > > >> different variants of it for different data formats?
> > > > >>
> > > > >> The points for this is again to keep the main API clean. The
> points
> > > > >> against this is that it duplicates the API, increases the surface
> area
> > > > >>and
> > > > >> creates redundancy for a minor addition.
> > > > >>
> > > > >> 3. Do we need to support different data types per record? The
> current
> > > > >> interface (bytes in/bytes out) lets you instantiate one producer
> and
> > > use
> > > > >> it to send multiple data formats. There seems to be some valid use
> > > cases
> > > > >> for this.
> > > > >>
> > > > >> I have still not seen a strong argument against not having this
> > > > >> functionality. Can someone provide their views on why we don't
> need
> > > this
> > > > >> support that is possible with the current API?
> > > > >>
> > > > >> One possible approach for the per record serialization would be to
> > > > >>define
> > > > >>
> > > > >> public interface SerDe<K,V> {
> > > > >>   public byte[] serializeKey();
> > > > >>
> > > > >>   public K deserializeKey();
> > > > >>
> > > > >>   public byte[] serializeValue();
> > > > >>
> > > > >>   public V deserializeValue();
> > > > >> }
> > > > >>
> > > > >> This would be used by both the Producer and the Consumer.
> > > > >>
> > > > >> The send APIs can then be
> > > > >>
> > > > >> public Future<RecordMetadata> send(ProducerRecord<K,V> record);
> > > > >> public Future<RecordMetadata> send(ProducerRecord<K,V> record,
> > > Callback
> > > > >> callback);
> > > > >>
> > > > >>
> > > > >> public Future<RecordMetadata> send(ProducerRecord<K,V> record,
> > > > >>SerDe<K,V>
> > > > >> serde);
> > > > >>
> > > > >> public Future<RecordMetadata> send(ProducerRecord<K,V> record,
> > > > >>SerDe<K,V>
> > > > >> serde, Callback callback);
> > > > >>
> > > > >>
> > > > >> A default SerDe can be set in the config. The producer would use
> the
> > > > >> default from the config if the non-serde send APIs are used. The
> > > > >>downside
> > > > >> to this approach is that we would need to have four variants of
> Send
> > > API
> > > > >> for the Producer.
> > > > >>
> > > > >>
> > > > >>
> > > > >>
> > > > >>
> > > > >>
> > > > >> On 12/5/14 3:16 PM, "Jun Rao" <ju...@confluent.io> wrote:
> > > > >>
> > > > >> >Jiangjie,
> > > > >> >
> > > > >> >The issue with adding the serializer in ProducerRecord is that
> you
> > > > >>need to
> > > > >> >implement all combinations of serializers for key and value. So,
> > > > >>instead
> > > > >> >of
> > > > >> >just implementing int and string serializers, you will have to
> > > > >>implement
> > > > >> >all 4 combinations.
> > > > >> >
> > > > >> >Adding a new producer constructor like Producer<K,
> > > V>(KeySerializer<K>,
> > > > >> >ValueSerializer<V>, Properties properties) can be useful.
> > > > >> >
> > > > >> >Thanks,
> > > > >> >
> > > > >> >Jun
> > > > >> >
> > > > >> >On Thu, Dec 4, 2014 at 10:33 AM, Jiangjie Qin
> > > > >><jq...@linkedin.com.invalid>
> > > > >> >wrote:
> > > > >> >
> > > > >> >>
> > > > >> >> I'm just thinking instead of binding serialization with
> producer,
> > > > >> >>another
> > > > >> >> option is to bind serializer/deserializer with
> > > > >> >> ProducerRecord/ConsumerRecord (please see the detail proposal
> > > below.)
> > > > >> >>            The arguments for this option is:
> > > > >> >>         A. A single producer could send different message
> types.
> > > > >>There
> > > > >> >>are
> > > > >> >> several use cases in LinkedIn for per record serializer
> > > > >> >>         - In Samza, there are some in-stream order-sensitive
> > > control
> > > > >> >> messages
> > > > >> >> having different deserializer from other messages.
> > > > >> >>         - There are use cases which need support for sending
> both
> > > > >>Avro
> > > > >> >> messages
> > > > >> >> and raw bytes.
> > > > >> >>         - Some use cases needs to deserialize some Avro
> messages
> > > into
> > > > >> >> generic
> > > > >> >> record and some other messages into specific record.
> > > > >> >>         B. In current proposal, the serializer/deserilizer is
> > > > >> >>instantiated
> > > > >> >> according to config. Compared with that, binding serializer
> with
> > > > >> >> ProducerRecord and ConsumerRecord is less error prone.
> > > > >> >>
> > > > >> >>
> > > > >> >>         This option includes the following changes:
> > > > >> >>         A. Add serializer and deserializer interfaces to
> replace
> > > > >> >>serializer
> > > > >> >> instance from config.
> > > > >> >>                 Public interface Serializer <K, V> {
> > > > >> >>                         public byte[] serializeKey(K key);
> > > > >> >>                         public byte[] serializeValue(V value);
> > > > >> >>                 }
> > > > >> >>                 Public interface deserializer <K, V> {
> > > > >> >>                         Public K deserializeKey(byte[] key);
> > > > >> >>                         public V deserializeValue(byte[]
> value);
> > > > >> >>                 }
> > > > >> >>
> > > > >> >>         B. Make ProducerRecord and ConsumerRecord abstract
> class
> > > > >> >> implementing
> > > > >> >> Serializer <K, V> and Deserializer <K, V> respectively.
> > > > >> >>                 Public abstract class ProducerRecord <K, V>
> > > > >>implements
> > > > >> >> Serializer <K, V>
> > > > >> >> {...}
> > > > >> >>                 Public abstract class ConsumerRecord <K, V>
> > > > >>implements
> > > > >> >> Deserializer <K,
> > > > >> >> V> {...}
> > > > >> >>
> > > > >> >>         C. Instead of instantiate the serializer/Deserializer
> from
> > > > >> >>config,
> > > > >> >> let
> > > > >> >> concrete ProducerRecord/ConsumerRecord extends the abstract
> class
> > > and
> > > > >> >> override the serialize/deserialize methods.
> > > > >> >>
> > > > >> >>                 Public class AvroProducerRecord extends
> > > > >>ProducerRecord
> > > > >> >> <String,
> > > > >> >> GenericRecord> {
> > > > >> >>                         ...
> > > > >> >>                         @Override
> > > > >> >>                         Public byte[] serializeKey(String key)
> {Š}
> > > > >> >>                         @Override
> > > > >> >>                         public byte[]
> serializeValue(GenericRecord
> > > > >> >>value);
> > > > >> >>                 }
> > > > >> >>
> > > > >> >>                 Public class AvroConsumerRecord extends
> > > > >>ConsumerRecord
> > > > >> >> <String,
> > > > >> >> GenericRecord> {
> > > > >> >>                         ...
> > > > >> >>                         @Override
> > > > >> >>                         Public K deserializeKey(byte[] key) {Š}
> > > > >> >>                         @Override
> > > > >> >>                         public V deserializeValue(byte[]
> value);
> > > > >> >>                 }
> > > > >> >>
> > > > >> >>         D. The producer API changes to
> > > > >> >>                 Public class KafkaProducer {
> > > > >> >>                         ...
> > > > >> >>
> > > > >> >>                         Future<RecordMetadata> send
> (ProducerRecord
> > > > >><K,
> > > > >> >>V>
> > > > >> >> record) {
> > > > >> >>                                 ...
> > > > >> >>                                 K key =
> > > > >>record.serializeKey(record.key);
> > > > >> >>                                 V value =
> > > > >> >> record.serializedValue(record.value);
> > > > >> >>                                 BytesProducerRecord
> > > > >>bytesProducerRecord
> > > > >> >>=
> > > > >> >> new
> > > > >> >> BytesProducerRecord(topic, partition, key, value);
> > > > >> >>                                 ...
> > > > >> >>                         }
> > > > >> >>                         ...
> > > > >> >>                 }
> > > > >> >>
> > > > >> >>
> > > > >> >>
> > > > >> >> We also had some brainstorm in LinkedIn and here are the
> feedbacks:
> > > > >> >>
> > > > >> >> If the community decide to add the serialization back to new
> > > > >>producer,
> > > > >> >> besides current proposal which changes new producer API to be a
> > > > >> >>template,
> > > > >> >> there are some other options raised during our discussion:
> > > > >> >>         1) Rather than change current new producer API, we can
> > > > >>provide a
> > > > >> >> wrapper
> > > > >> >> of current new producer (e.g. KafkaSerializedProducer) and
> make it
> > > > >> >> available to users. As there is value in the simplicity of
> current
> > > > >>API.
> > > > >> >>
> > > > >> >>         2) If we decide to go with tempalated new producer API,
> > > > >> >>according
> > > > >> >> to
> > > > >> >> experience in LinkedIn, it might worth considering to
> instantiate
> > > the
> > > > >> >> serializer in code instead of from config so we can avoid
> runtime
> > > > >>errors
> > > > >> >> due to dynamic instantiation from config, which is more error
> > > prone.
> > > > >>If
> > > > >> >> that is the case, the producer API could be changed to
> something
> > > > >>like:
> > > > >> >>                 producer = new Producer<K, V>(KeySerializer<K>,
> > > > >> >> ValueSerializer<V>)
> > > > >> >>
> > > > >> >> --Jiangjie (Becket) Qin
> > > > >> >>
> > > > >> >>
> > > > >> >> On 11/24/14, 5:58 PM, "Jun Rao" <ju...@gmail.com> 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
> > > > >> >>
> > > > >> >>
> > > > >>
> > > > >>
> > > >
> > >
> > >
>
>

Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Jun Rao <ju...@confluent.io>.
Joel,

With a byte array interface, of course there is nothing that one can't do.
However, the real question is that whether we want to encourage people to
use it this way or not. Being able to flow just bytes is definitely easier
to get started. That's why many early adopters choose to do it that way.
However, it's often the case that they start feeling the pain later when
some producers change the data format. Their Hive/Pig queries start to
break and it's a painful process to have the issue fixed. So, the purpose
of this api change is really to encourage people to standardize on a single
serializer/deserializer that supports things like data validation and
schema evolution upstream in the producer. Now, suppose there is an Avro
serializer/deserializer implementation. How do we make it easy for people
to adopt? If the serializer is part of the api, we can just say, wire in
the Avro serializer for key and/or value in the config and then you can
start sending Avro records to the producer. If the serializer is not part
of the api, we have to say, first instantiate a key and/or value serializer
this way, send the key to the key serializer to get the key bytes, send the
value to the value serializer to get the value bytes, and finally send the
bytes to the producer. The former will be simpler and likely makes the
adoption easier.

Thanks,

Jun

On Mon, Dec 15, 2014 at 7:20 PM, Joel Koshy <jj...@gmail.com> wrote:
>
> Documentation is inevitable even if the serializer/deserializer is
> part of the API - since the user has to set it up in the configs. So
> again, you can only encourage people to use it through documentation.
> The simpler byte-oriented API seems clearer to me because anyone who
> needs to send (or receive) a specific data type will _be forced to_
> (or actually, _intuitively_) select a serializer (or deserializer) and
> will definitely pick an already available implementation if a good one
> already exists.
>
> Sorry I still don't get it and this is really the only sticking point
> for me, albeit a minor one (which is why I have been +0 all along on
> the change). I (and I think many others) would appreciate it if
> someone can help me understand this better.  So I will repeat the
> question: What "usage pattern" cannot be supported by easily by the
> simpler API without adding burden on the user?
>
> Thanks,
>
> Joel
>
> On Mon, Dec 15, 2014 at 11:59:34AM -0800, Jun Rao wrote:
> > Joel,
> >
> > It's just that if the serializer/deserializer is not part of the API, you
> > can only encourage people to use it through documentation. However, not
> > everyone will read the documentation if it's not directly used in the
> API.
> >
> > Thanks,
> >
> > Jun
> >
> > On Mon, Dec 15, 2014 at 2:11 AM, Joel Koshy <jj...@gmail.com> wrote:
> >
> > > (sorry about the late follow-up late - I'm traveling most of this
> > > month)
> > >
> > > I'm likely missing something obvious, but I find the following to be a
> > > somewhat vague point that has been mentioned more than once in this
> > > thread without a clear explanation. i.e., why is it hard to share a
> > > serializer/deserializer implementation and just have the clients call
> > > it before a send/receive? What "usage pattern" cannot be supported by
> > > the simpler API?
> > >
> > > > 1. Can we keep the serialization semantics outside the Producer
> interface
> > > > and have simple bytes in / bytes out for the interface (This is what
> we
> > > > have today).
> > > >
> > > > The points for this is to keep the interface simple and usage easy to
> > > > understand. The points against this is that it gets hard to share
> common
> > > > usage patterns around serialization/message validations for the
> future.
> > >
> > >
> > > On Tue, Dec 09, 2014 at 03:51:08AM +0000, Sriram Subramanian wrote:
> > > > Thank you Jay. I agree with the issue that you point w.r.t paired
> > > > serializers. I also think having mix serialization types is rare. To
> get
> > > > the current behavior, one can simply use a ByteArraySerializer. This
> is
> > > > best understood by talking with many customers and you seem to have
> done
> > > > that. I am convinced about the change.
> > > >
> > > > For the rest who gave -1 or 0 for this proposal, does the answers
> for the
> > > > three points(updated) below seem reasonable? Are these explanations
> > > > convincing?
> > > >
> > > >
> > > > 1. Can we keep the serialization semantics outside the Producer
> interface
> > > > and have simple bytes in / bytes out for the interface (This is what
> we
> > > > have today).
> > > >
> > > > The points for this is to keep the interface simple and usage easy to
> > > > understand. The points against this is that it gets hard to share
> common
> > > > usage patterns around serialization/message validations for the
> future.
> > > >
> > > > 2. Can we create a wrapper producer that does the serialization and
> have
> > > > different variants of it for different data formats?
> > > >
> > > > The points for this is again to keep the main API clean. The points
> > > > against this is that it duplicates the API, increases the surface
> area
> > > and
> > > > creates redundancy for a minor addition.
> > > >
> > > > 3. Do we need to support different data types per record? The current
> > > > interface (bytes in/bytes out) lets you instantiate one producer and
> use
> > > > it to send multiple data formats. There seems to be some valid use
> cases
> > > > for this.
> > > >
> > > >
> > > > Mixed serialization types are rare based on interactions with
> customers.
> > > > To get the current behavior, one can simply use a
> ByteArraySerializer.
> > > >
> > > > On 12/5/14 5:00 PM, "Jay Kreps" <ja...@confluent.io> wrote:
> > > >
> > > > >Hey Sriram,
> > > > >
> > > > >Thanks! I think this is a very helpful summary.
> > > > >
> > > > >Let me try to address your point about passing in the serde at send
> > > time.
> > > > >
> > > > >I think the first objection is really to the paired key/value
> serializer
> > > > >interfaces. This leads to kind of a weird combinatorial thing where
> you
> > > > >would have an avro/avro serializer a string/avro serializer, a pb/pb
> > > > >serializer, and a string/pb serializer, and so on. But your proposal
> > > would
> > > > >work as well with separate serializers for key and value.
> > > > >
> > > > >I think the downside is just the one you call out--that this is a
> corner
> > > > >case and you end up with two versions of all the apis to support it.
> > > This
> > > > >also makes the serializer api more annoying to implement. I think
> the
> > > > >alternative solution to this case and any other we can give people
> is
> > > just
> > > > >configuring ByteArraySerializer which gives you basically the api
> that
> > > you
> > > > >have now with byte arrays. If this is incredibly common then this
> would
> > > be
> > > > >a silly solution, but I guess the belief is that these cases are
> rare
> > > and
> > > > >a
> > > > >really well implemented avro or json serializer should be 100% of
> what
> > > > >most
> > > > >people need.
> > > > >
> > > > >In practice the cases that actually mix serialization types in a
> single
> > > > >stream are pretty rare I think just because the consumer then has
> the
> > > > >problem of guessing how to deserialize, so most of these will end up
> > > with
> > > > >at least some marker or schema id or whatever that tells you how to
> read
> > > > >the data. Arguable this mixed serialization with marker is itself a
> > > > >serializer type and should have a serializer of its own...
> > > > >
> > > > >-Jay
> > > > >
> > > > >On Fri, Dec 5, 2014 at 3:48 PM, Sriram Subramanian <
> > > > >srsubramanian@linkedin.com.invalid> wrote:
> > > > >
> > > > >> This thread has diverged multiple times now and it would be worth
> > > > >> summarizing them.
> > > > >>
> > > > >> There seems to be the following points of discussion -
> > > > >>
> > > > >> 1. Can we keep the serialization semantics outside the Producer
> > > > >>interface
> > > > >> and have simple bytes in / bytes out for the interface (This is
> what
> > > we
> > > > >> have today).
> > > > >>
> > > > >> The points for this is to keep the interface simple and usage
> easy to
> > > > >> understand. The points against this is that it gets hard to share
> > > common
> > > > >> usage patterns around serialization/message validations for the
> > > future.
> > > > >>
> > > > >> 2. Can we create a wrapper producer that does the serialization
> and
> > > have
> > > > >> different variants of it for different data formats?
> > > > >>
> > > > >> The points for this is again to keep the main API clean. The
> points
> > > > >> against this is that it duplicates the API, increases the surface
> area
> > > > >>and
> > > > >> creates redundancy for a minor addition.
> > > > >>
> > > > >> 3. Do we need to support different data types per record? The
> current
> > > > >> interface (bytes in/bytes out) lets you instantiate one producer
> and
> > > use
> > > > >> it to send multiple data formats. There seems to be some valid use
> > > cases
> > > > >> for this.
> > > > >>
> > > > >> I have still not seen a strong argument against not having this
> > > > >> functionality. Can someone provide their views on why we don't
> need
> > > this
> > > > >> support that is possible with the current API?
> > > > >>
> > > > >> One possible approach for the per record serialization would be to
> > > > >>define
> > > > >>
> > > > >> public interface SerDe<K,V> {
> > > > >>   public byte[] serializeKey();
> > > > >>
> > > > >>   public K deserializeKey();
> > > > >>
> > > > >>   public byte[] serializeValue();
> > > > >>
> > > > >>   public V deserializeValue();
> > > > >> }
> > > > >>
> > > > >> This would be used by both the Producer and the Consumer.
> > > > >>
> > > > >> The send APIs can then be
> > > > >>
> > > > >> public Future<RecordMetadata> send(ProducerRecord<K,V> record);
> > > > >> public Future<RecordMetadata> send(ProducerRecord<K,V> record,
> > > Callback
> > > > >> callback);
> > > > >>
> > > > >>
> > > > >> public Future<RecordMetadata> send(ProducerRecord<K,V> record,
> > > > >>SerDe<K,V>
> > > > >> serde);
> > > > >>
> > > > >> public Future<RecordMetadata> send(ProducerRecord<K,V> record,
> > > > >>SerDe<K,V>
> > > > >> serde, Callback callback);
> > > > >>
> > > > >>
> > > > >> A default SerDe can be set in the config. The producer would use
> the
> > > > >> default from the config if the non-serde send APIs are used. The
> > > > >>downside
> > > > >> to this approach is that we would need to have four variants of
> Send
> > > API
> > > > >> for the Producer.
> > > > >>
> > > > >>
> > > > >>
> > > > >>
> > > > >>
> > > > >>
> > > > >> On 12/5/14 3:16 PM, "Jun Rao" <ju...@confluent.io> wrote:
> > > > >>
> > > > >> >Jiangjie,
> > > > >> >
> > > > >> >The issue with adding the serializer in ProducerRecord is that
> you
> > > > >>need to
> > > > >> >implement all combinations of serializers for key and value. So,
> > > > >>instead
> > > > >> >of
> > > > >> >just implementing int and string serializers, you will have to
> > > > >>implement
> > > > >> >all 4 combinations.
> > > > >> >
> > > > >> >Adding a new producer constructor like Producer<K,
> > > V>(KeySerializer<K>,
> > > > >> >ValueSerializer<V>, Properties properties) can be useful.
> > > > >> >
> > > > >> >Thanks,
> > > > >> >
> > > > >> >Jun
> > > > >> >
> > > > >> >On Thu, Dec 4, 2014 at 10:33 AM, Jiangjie Qin
> > > > >><jq...@linkedin.com.invalid>
> > > > >> >wrote:
> > > > >> >
> > > > >> >>
> > > > >> >> I'm just thinking instead of binding serialization with
> producer,
> > > > >> >>another
> > > > >> >> option is to bind serializer/deserializer with
> > > > >> >> ProducerRecord/ConsumerRecord (please see the detail proposal
> > > below.)
> > > > >> >>            The arguments for this option is:
> > > > >> >>         A. A single producer could send different message
> types.
> > > > >>There
> > > > >> >>are
> > > > >> >> several use cases in LinkedIn for per record serializer
> > > > >> >>         - In Samza, there are some in-stream order-sensitive
> > > control
> > > > >> >> messages
> > > > >> >> having different deserializer from other messages.
> > > > >> >>         - There are use cases which need support for sending
> both
> > > > >>Avro
> > > > >> >> messages
> > > > >> >> and raw bytes.
> > > > >> >>         - Some use cases needs to deserialize some Avro
> messages
> > > into
> > > > >> >> generic
> > > > >> >> record and some other messages into specific record.
> > > > >> >>         B. In current proposal, the serializer/deserilizer is
> > > > >> >>instantiated
> > > > >> >> according to config. Compared with that, binding serializer
> with
> > > > >> >> ProducerRecord and ConsumerRecord is less error prone.
> > > > >> >>
> > > > >> >>
> > > > >> >>         This option includes the following changes:
> > > > >> >>         A. Add serializer and deserializer interfaces to
> replace
> > > > >> >>serializer
> > > > >> >> instance from config.
> > > > >> >>                 Public interface Serializer <K, V> {
> > > > >> >>                         public byte[] serializeKey(K key);
> > > > >> >>                         public byte[] serializeValue(V value);
> > > > >> >>                 }
> > > > >> >>                 Public interface deserializer <K, V> {
> > > > >> >>                         Public K deserializeKey(byte[] key);
> > > > >> >>                         public V deserializeValue(byte[]
> value);
> > > > >> >>                 }
> > > > >> >>
> > > > >> >>         B. Make ProducerRecord and ConsumerRecord abstract
> class
> > > > >> >> implementing
> > > > >> >> Serializer <K, V> and Deserializer <K, V> respectively.
> > > > >> >>                 Public abstract class ProducerRecord <K, V>
> > > > >>implements
> > > > >> >> Serializer <K, V>
> > > > >> >> {...}
> > > > >> >>                 Public abstract class ConsumerRecord <K, V>
> > > > >>implements
> > > > >> >> Deserializer <K,
> > > > >> >> V> {...}
> > > > >> >>
> > > > >> >>         C. Instead of instantiate the serializer/Deserializer
> from
> > > > >> >>config,
> > > > >> >> let
> > > > >> >> concrete ProducerRecord/ConsumerRecord extends the abstract
> class
> > > and
> > > > >> >> override the serialize/deserialize methods.
> > > > >> >>
> > > > >> >>                 Public class AvroProducerRecord extends
> > > > >>ProducerRecord
> > > > >> >> <String,
> > > > >> >> GenericRecord> {
> > > > >> >>                         ...
> > > > >> >>                         @Override
> > > > >> >>                         Public byte[] serializeKey(String key)
> {Š}
> > > > >> >>                         @Override
> > > > >> >>                         public byte[]
> serializeValue(GenericRecord
> > > > >> >>value);
> > > > >> >>                 }
> > > > >> >>
> > > > >> >>                 Public class AvroConsumerRecord extends
> > > > >>ConsumerRecord
> > > > >> >> <String,
> > > > >> >> GenericRecord> {
> > > > >> >>                         ...
> > > > >> >>                         @Override
> > > > >> >>                         Public K deserializeKey(byte[] key) {Š}
> > > > >> >>                         @Override
> > > > >> >>                         public V deserializeValue(byte[]
> value);
> > > > >> >>                 }
> > > > >> >>
> > > > >> >>         D. The producer API changes to
> > > > >> >>                 Public class KafkaProducer {
> > > > >> >>                         ...
> > > > >> >>
> > > > >> >>                         Future<RecordMetadata> send
> (ProducerRecord
> > > > >><K,
> > > > >> >>V>
> > > > >> >> record) {
> > > > >> >>                                 ...
> > > > >> >>                                 K key =
> > > > >>record.serializeKey(record.key);
> > > > >> >>                                 V value =
> > > > >> >> record.serializedValue(record.value);
> > > > >> >>                                 BytesProducerRecord
> > > > >>bytesProducerRecord
> > > > >> >>=
> > > > >> >> new
> > > > >> >> BytesProducerRecord(topic, partition, key, value);
> > > > >> >>                                 ...
> > > > >> >>                         }
> > > > >> >>                         ...
> > > > >> >>                 }
> > > > >> >>
> > > > >> >>
> > > > >> >>
> > > > >> >> We also had some brainstorm in LinkedIn and here are the
> feedbacks:
> > > > >> >>
> > > > >> >> If the community decide to add the serialization back to new
> > > > >>producer,
> > > > >> >> besides current proposal which changes new producer API to be a
> > > > >> >>template,
> > > > >> >> there are some other options raised during our discussion:
> > > > >> >>         1) Rather than change current new producer API, we can
> > > > >>provide a
> > > > >> >> wrapper
> > > > >> >> of current new producer (e.g. KafkaSerializedProducer) and
> make it
> > > > >> >> available to users. As there is value in the simplicity of
> current
> > > > >>API.
> > > > >> >>
> > > > >> >>         2) If we decide to go with tempalated new producer API,
> > > > >> >>according
> > > > >> >> to
> > > > >> >> experience in LinkedIn, it might worth considering to
> instantiate
> > > the
> > > > >> >> serializer in code instead of from config so we can avoid
> runtime
> > > > >>errors
> > > > >> >> due to dynamic instantiation from config, which is more error
> > > prone.
> > > > >>If
> > > > >> >> that is the case, the producer API could be changed to
> something
> > > > >>like:
> > > > >> >>                 producer = new Producer<K, V>(KeySerializer<K>,
> > > > >> >> ValueSerializer<V>)
> > > > >> >>
> > > > >> >> --Jiangjie (Becket) Qin
> > > > >> >>
> > > > >> >>
> > > > >> >> On 11/24/14, 5:58 PM, "Jun Rao" <ju...@gmail.com> 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
> > > > >> >>
> > > > >> >>
> > > > >>
> > > > >>
> > > >
> > >
> > >
>
>

Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Joel Koshy <jj...@gmail.com>.
Documentation is inevitable even if the serializer/deserializer is
part of the API - since the user has to set it up in the configs. So
again, you can only encourage people to use it through documentation.
The simpler byte-oriented API seems clearer to me because anyone who
needs to send (or receive) a specific data type will _be forced to_
(or actually, _intuitively_) select a serializer (or deserializer) and
will definitely pick an already available implementation if a good one
already exists.

Sorry I still don't get it and this is really the only sticking point
for me, albeit a minor one (which is why I have been +0 all along on
the change). I (and I think many others) would appreciate it if
someone can help me understand this better.  So I will repeat the
question: What "usage pattern" cannot be supported by easily by the
simpler API without adding burden on the user?

Thanks,

Joel

On Mon, Dec 15, 2014 at 11:59:34AM -0800, Jun Rao wrote:
> Joel,
> 
> It's just that if the serializer/deserializer is not part of the API, you
> can only encourage people to use it through documentation. However, not
> everyone will read the documentation if it's not directly used in the API.
> 
> Thanks,
> 
> Jun
> 
> On Mon, Dec 15, 2014 at 2:11 AM, Joel Koshy <jj...@gmail.com> wrote:
> 
> > (sorry about the late follow-up late - I'm traveling most of this
> > month)
> >
> > I'm likely missing something obvious, but I find the following to be a
> > somewhat vague point that has been mentioned more than once in this
> > thread without a clear explanation. i.e., why is it hard to share a
> > serializer/deserializer implementation and just have the clients call
> > it before a send/receive? What "usage pattern" cannot be supported by
> > the simpler API?
> >
> > > 1. Can we keep the serialization semantics outside the Producer interface
> > > and have simple bytes in / bytes out for the interface (This is what we
> > > have today).
> > >
> > > The points for this is to keep the interface simple and usage easy to
> > > understand. The points against this is that it gets hard to share common
> > > usage patterns around serialization/message validations for the future.
> >
> >
> > On Tue, Dec 09, 2014 at 03:51:08AM +0000, Sriram Subramanian wrote:
> > > Thank you Jay. I agree with the issue that you point w.r.t paired
> > > serializers. I also think having mix serialization types is rare. To get
> > > the current behavior, one can simply use a ByteArraySerializer. This is
> > > best understood by talking with many customers and you seem to have done
> > > that. I am convinced about the change.
> > >
> > > For the rest who gave -1 or 0 for this proposal, does the answers for the
> > > three points(updated) below seem reasonable? Are these explanations
> > > convincing?
> > >
> > >
> > > 1. Can we keep the serialization semantics outside the Producer interface
> > > and have simple bytes in / bytes out for the interface (This is what we
> > > have today).
> > >
> > > The points for this is to keep the interface simple and usage easy to
> > > understand. The points against this is that it gets hard to share common
> > > usage patterns around serialization/message validations for the future.
> > >
> > > 2. Can we create a wrapper producer that does the serialization and have
> > > different variants of it for different data formats?
> > >
> > > The points for this is again to keep the main API clean. The points
> > > against this is that it duplicates the API, increases the surface area
> > and
> > > creates redundancy for a minor addition.
> > >
> > > 3. Do we need to support different data types per record? The current
> > > interface (bytes in/bytes out) lets you instantiate one producer and use
> > > it to send multiple data formats. There seems to be some valid use cases
> > > for this.
> > >
> > >
> > > Mixed serialization types are rare based on interactions with customers.
> > > To get the current behavior, one can simply use a ByteArraySerializer.
> > >
> > > On 12/5/14 5:00 PM, "Jay Kreps" <ja...@confluent.io> wrote:
> > >
> > > >Hey Sriram,
> > > >
> > > >Thanks! I think this is a very helpful summary.
> > > >
> > > >Let me try to address your point about passing in the serde at send
> > time.
> > > >
> > > >I think the first objection is really to the paired key/value serializer
> > > >interfaces. This leads to kind of a weird combinatorial thing where you
> > > >would have an avro/avro serializer a string/avro serializer, a pb/pb
> > > >serializer, and a string/pb serializer, and so on. But your proposal
> > would
> > > >work as well with separate serializers for key and value.
> > > >
> > > >I think the downside is just the one you call out--that this is a corner
> > > >case and you end up with two versions of all the apis to support it.
> > This
> > > >also makes the serializer api more annoying to implement. I think the
> > > >alternative solution to this case and any other we can give people is
> > just
> > > >configuring ByteArraySerializer which gives you basically the api that
> > you
> > > >have now with byte arrays. If this is incredibly common then this would
> > be
> > > >a silly solution, but I guess the belief is that these cases are rare
> > and
> > > >a
> > > >really well implemented avro or json serializer should be 100% of what
> > > >most
> > > >people need.
> > > >
> > > >In practice the cases that actually mix serialization types in a single
> > > >stream are pretty rare I think just because the consumer then has the
> > > >problem of guessing how to deserialize, so most of these will end up
> > with
> > > >at least some marker or schema id or whatever that tells you how to read
> > > >the data. Arguable this mixed serialization with marker is itself a
> > > >serializer type and should have a serializer of its own...
> > > >
> > > >-Jay
> > > >
> > > >On Fri, Dec 5, 2014 at 3:48 PM, Sriram Subramanian <
> > > >srsubramanian@linkedin.com.invalid> wrote:
> > > >
> > > >> This thread has diverged multiple times now and it would be worth
> > > >> summarizing them.
> > > >>
> > > >> There seems to be the following points of discussion -
> > > >>
> > > >> 1. Can we keep the serialization semantics outside the Producer
> > > >>interface
> > > >> and have simple bytes in / bytes out for the interface (This is what
> > we
> > > >> have today).
> > > >>
> > > >> The points for this is to keep the interface simple and usage easy to
> > > >> understand. The points against this is that it gets hard to share
> > common
> > > >> usage patterns around serialization/message validations for the
> > future.
> > > >>
> > > >> 2. Can we create a wrapper producer that does the serialization and
> > have
> > > >> different variants of it for different data formats?
> > > >>
> > > >> The points for this is again to keep the main API clean. The points
> > > >> against this is that it duplicates the API, increases the surface area
> > > >>and
> > > >> creates redundancy for a minor addition.
> > > >>
> > > >> 3. Do we need to support different data types per record? The current
> > > >> interface (bytes in/bytes out) lets you instantiate one producer and
> > use
> > > >> it to send multiple data formats. There seems to be some valid use
> > cases
> > > >> for this.
> > > >>
> > > >> I have still not seen a strong argument against not having this
> > > >> functionality. Can someone provide their views on why we don't need
> > this
> > > >> support that is possible with the current API?
> > > >>
> > > >> One possible approach for the per record serialization would be to
> > > >>define
> > > >>
> > > >> public interface SerDe<K,V> {
> > > >>   public byte[] serializeKey();
> > > >>
> > > >>   public K deserializeKey();
> > > >>
> > > >>   public byte[] serializeValue();
> > > >>
> > > >>   public V deserializeValue();
> > > >> }
> > > >>
> > > >> This would be used by both the Producer and the Consumer.
> > > >>
> > > >> The send APIs can then be
> > > >>
> > > >> public Future<RecordMetadata> send(ProducerRecord<K,V> record);
> > > >> public Future<RecordMetadata> send(ProducerRecord<K,V> record,
> > Callback
> > > >> callback);
> > > >>
> > > >>
> > > >> public Future<RecordMetadata> send(ProducerRecord<K,V> record,
> > > >>SerDe<K,V>
> > > >> serde);
> > > >>
> > > >> public Future<RecordMetadata> send(ProducerRecord<K,V> record,
> > > >>SerDe<K,V>
> > > >> serde, Callback callback);
> > > >>
> > > >>
> > > >> A default SerDe can be set in the config. The producer would use the
> > > >> default from the config if the non-serde send APIs are used. The
> > > >>downside
> > > >> to this approach is that we would need to have four variants of Send
> > API
> > > >> for the Producer.
> > > >>
> > > >>
> > > >>
> > > >>
> > > >>
> > > >>
> > > >> On 12/5/14 3:16 PM, "Jun Rao" <ju...@confluent.io> wrote:
> > > >>
> > > >> >Jiangjie,
> > > >> >
> > > >> >The issue with adding the serializer in ProducerRecord is that you
> > > >>need to
> > > >> >implement all combinations of serializers for key and value. So,
> > > >>instead
> > > >> >of
> > > >> >just implementing int and string serializers, you will have to
> > > >>implement
> > > >> >all 4 combinations.
> > > >> >
> > > >> >Adding a new producer constructor like Producer<K,
> > V>(KeySerializer<K>,
> > > >> >ValueSerializer<V>, Properties properties) can be useful.
> > > >> >
> > > >> >Thanks,
> > > >> >
> > > >> >Jun
> > > >> >
> > > >> >On Thu, Dec 4, 2014 at 10:33 AM, Jiangjie Qin
> > > >><jq...@linkedin.com.invalid>
> > > >> >wrote:
> > > >> >
> > > >> >>
> > > >> >> I'm just thinking instead of binding serialization with producer,
> > > >> >>another
> > > >> >> option is to bind serializer/deserializer with
> > > >> >> ProducerRecord/ConsumerRecord (please see the detail proposal
> > below.)
> > > >> >>            The arguments for this option is:
> > > >> >>         A. A single producer could send different message types.
> > > >>There
> > > >> >>are
> > > >> >> several use cases in LinkedIn for per record serializer
> > > >> >>         - In Samza, there are some in-stream order-sensitive
> > control
> > > >> >> messages
> > > >> >> having different deserializer from other messages.
> > > >> >>         - There are use cases which need support for sending both
> > > >>Avro
> > > >> >> messages
> > > >> >> and raw bytes.
> > > >> >>         - Some use cases needs to deserialize some Avro messages
> > into
> > > >> >> generic
> > > >> >> record and some other messages into specific record.
> > > >> >>         B. In current proposal, the serializer/deserilizer is
> > > >> >>instantiated
> > > >> >> according to config. Compared with that, binding serializer with
> > > >> >> ProducerRecord and ConsumerRecord is less error prone.
> > > >> >>
> > > >> >>
> > > >> >>         This option includes the following changes:
> > > >> >>         A. Add serializer and deserializer interfaces to replace
> > > >> >>serializer
> > > >> >> instance from config.
> > > >> >>                 Public interface Serializer <K, V> {
> > > >> >>                         public byte[] serializeKey(K key);
> > > >> >>                         public byte[] serializeValue(V value);
> > > >> >>                 }
> > > >> >>                 Public interface deserializer <K, V> {
> > > >> >>                         Public K deserializeKey(byte[] key);
> > > >> >>                         public V deserializeValue(byte[] value);
> > > >> >>                 }
> > > >> >>
> > > >> >>         B. Make ProducerRecord and ConsumerRecord abstract class
> > > >> >> implementing
> > > >> >> Serializer <K, V> and Deserializer <K, V> respectively.
> > > >> >>                 Public abstract class ProducerRecord <K, V>
> > > >>implements
> > > >> >> Serializer <K, V>
> > > >> >> {...}
> > > >> >>                 Public abstract class ConsumerRecord <K, V>
> > > >>implements
> > > >> >> Deserializer <K,
> > > >> >> V> {...}
> > > >> >>
> > > >> >>         C. Instead of instantiate the serializer/Deserializer from
> > > >> >>config,
> > > >> >> let
> > > >> >> concrete ProducerRecord/ConsumerRecord extends the abstract class
> > and
> > > >> >> override the serialize/deserialize methods.
> > > >> >>
> > > >> >>                 Public class AvroProducerRecord extends
> > > >>ProducerRecord
> > > >> >> <String,
> > > >> >> GenericRecord> {
> > > >> >>                         ...
> > > >> >>                         @Override
> > > >> >>                         Public byte[] serializeKey(String key) {Š}
> > > >> >>                         @Override
> > > >> >>                         public byte[] serializeValue(GenericRecord
> > > >> >>value);
> > > >> >>                 }
> > > >> >>
> > > >> >>                 Public class AvroConsumerRecord extends
> > > >>ConsumerRecord
> > > >> >> <String,
> > > >> >> GenericRecord> {
> > > >> >>                         ...
> > > >> >>                         @Override
> > > >> >>                         Public K deserializeKey(byte[] key) {Š}
> > > >> >>                         @Override
> > > >> >>                         public V deserializeValue(byte[] value);
> > > >> >>                 }
> > > >> >>
> > > >> >>         D. The producer API changes to
> > > >> >>                 Public class KafkaProducer {
> > > >> >>                         ...
> > > >> >>
> > > >> >>                         Future<RecordMetadata> send (ProducerRecord
> > > >><K,
> > > >> >>V>
> > > >> >> record) {
> > > >> >>                                 ...
> > > >> >>                                 K key =
> > > >>record.serializeKey(record.key);
> > > >> >>                                 V value =
> > > >> >> record.serializedValue(record.value);
> > > >> >>                                 BytesProducerRecord
> > > >>bytesProducerRecord
> > > >> >>=
> > > >> >> new
> > > >> >> BytesProducerRecord(topic, partition, key, value);
> > > >> >>                                 ...
> > > >> >>                         }
> > > >> >>                         ...
> > > >> >>                 }
> > > >> >>
> > > >> >>
> > > >> >>
> > > >> >> We also had some brainstorm in LinkedIn and here are the feedbacks:
> > > >> >>
> > > >> >> If the community decide to add the serialization back to new
> > > >>producer,
> > > >> >> besides current proposal which changes new producer API to be a
> > > >> >>template,
> > > >> >> there are some other options raised during our discussion:
> > > >> >>         1) Rather than change current new producer API, we can
> > > >>provide a
> > > >> >> wrapper
> > > >> >> of current new producer (e.g. KafkaSerializedProducer) and make it
> > > >> >> available to users. As there is value in the simplicity of current
> > > >>API.
> > > >> >>
> > > >> >>         2) If we decide to go with tempalated new producer API,
> > > >> >>according
> > > >> >> to
> > > >> >> experience in LinkedIn, it might worth considering to instantiate
> > the
> > > >> >> serializer in code instead of from config so we can avoid runtime
> > > >>errors
> > > >> >> due to dynamic instantiation from config, which is more error
> > prone.
> > > >>If
> > > >> >> that is the case, the producer API could be changed to something
> > > >>like:
> > > >> >>                 producer = new Producer<K, V>(KeySerializer<K>,
> > > >> >> ValueSerializer<V>)
> > > >> >>
> > > >> >> --Jiangjie (Becket) Qin
> > > >> >>
> > > >> >>
> > > >> >> On 11/24/14, 5:58 PM, "Jun Rao" <ju...@gmail.com> 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
> > > >> >>
> > > >> >>
> > > >>
> > > >>
> > >
> >
> >


Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Joel Koshy <jj...@gmail.com>.
Documentation is inevitable even if the serializer/deserializer is
part of the API - since the user has to set it up in the configs. So
again, you can only encourage people to use it through documentation.
The simpler byte-oriented API seems clearer to me because anyone who
needs to send (or receive) a specific data type will _be forced to_
(or actually, _intuitively_) select a serializer (or deserializer) and
will definitely pick an already available implementation if a good one
already exists.

Sorry I still don't get it and this is really the only sticking point
for me, albeit a minor one (which is why I have been +0 all along on
the change). I (and I think many others) would appreciate it if
someone can help me understand this better.  So I will repeat the
question: What "usage pattern" cannot be supported by easily by the
simpler API without adding burden on the user?

Thanks,

Joel

On Mon, Dec 15, 2014 at 11:59:34AM -0800, Jun Rao wrote:
> Joel,
> 
> It's just that if the serializer/deserializer is not part of the API, you
> can only encourage people to use it through documentation. However, not
> everyone will read the documentation if it's not directly used in the API.
> 
> Thanks,
> 
> Jun
> 
> On Mon, Dec 15, 2014 at 2:11 AM, Joel Koshy <jj...@gmail.com> wrote:
> 
> > (sorry about the late follow-up late - I'm traveling most of this
> > month)
> >
> > I'm likely missing something obvious, but I find the following to be a
> > somewhat vague point that has been mentioned more than once in this
> > thread without a clear explanation. i.e., why is it hard to share a
> > serializer/deserializer implementation and just have the clients call
> > it before a send/receive? What "usage pattern" cannot be supported by
> > the simpler API?
> >
> > > 1. Can we keep the serialization semantics outside the Producer interface
> > > and have simple bytes in / bytes out for the interface (This is what we
> > > have today).
> > >
> > > The points for this is to keep the interface simple and usage easy to
> > > understand. The points against this is that it gets hard to share common
> > > usage patterns around serialization/message validations for the future.
> >
> >
> > On Tue, Dec 09, 2014 at 03:51:08AM +0000, Sriram Subramanian wrote:
> > > Thank you Jay. I agree with the issue that you point w.r.t paired
> > > serializers. I also think having mix serialization types is rare. To get
> > > the current behavior, one can simply use a ByteArraySerializer. This is
> > > best understood by talking with many customers and you seem to have done
> > > that. I am convinced about the change.
> > >
> > > For the rest who gave -1 or 0 for this proposal, does the answers for the
> > > three points(updated) below seem reasonable? Are these explanations
> > > convincing?
> > >
> > >
> > > 1. Can we keep the serialization semantics outside the Producer interface
> > > and have simple bytes in / bytes out for the interface (This is what we
> > > have today).
> > >
> > > The points for this is to keep the interface simple and usage easy to
> > > understand. The points against this is that it gets hard to share common
> > > usage patterns around serialization/message validations for the future.
> > >
> > > 2. Can we create a wrapper producer that does the serialization and have
> > > different variants of it for different data formats?
> > >
> > > The points for this is again to keep the main API clean. The points
> > > against this is that it duplicates the API, increases the surface area
> > and
> > > creates redundancy for a minor addition.
> > >
> > > 3. Do we need to support different data types per record? The current
> > > interface (bytes in/bytes out) lets you instantiate one producer and use
> > > it to send multiple data formats. There seems to be some valid use cases
> > > for this.
> > >
> > >
> > > Mixed serialization types are rare based on interactions with customers.
> > > To get the current behavior, one can simply use a ByteArraySerializer.
> > >
> > > On 12/5/14 5:00 PM, "Jay Kreps" <ja...@confluent.io> wrote:
> > >
> > > >Hey Sriram,
> > > >
> > > >Thanks! I think this is a very helpful summary.
> > > >
> > > >Let me try to address your point about passing in the serde at send
> > time.
> > > >
> > > >I think the first objection is really to the paired key/value serializer
> > > >interfaces. This leads to kind of a weird combinatorial thing where you
> > > >would have an avro/avro serializer a string/avro serializer, a pb/pb
> > > >serializer, and a string/pb serializer, and so on. But your proposal
> > would
> > > >work as well with separate serializers for key and value.
> > > >
> > > >I think the downside is just the one you call out--that this is a corner
> > > >case and you end up with two versions of all the apis to support it.
> > This
> > > >also makes the serializer api more annoying to implement. I think the
> > > >alternative solution to this case and any other we can give people is
> > just
> > > >configuring ByteArraySerializer which gives you basically the api that
> > you
> > > >have now with byte arrays. If this is incredibly common then this would
> > be
> > > >a silly solution, but I guess the belief is that these cases are rare
> > and
> > > >a
> > > >really well implemented avro or json serializer should be 100% of what
> > > >most
> > > >people need.
> > > >
> > > >In practice the cases that actually mix serialization types in a single
> > > >stream are pretty rare I think just because the consumer then has the
> > > >problem of guessing how to deserialize, so most of these will end up
> > with
> > > >at least some marker or schema id or whatever that tells you how to read
> > > >the data. Arguable this mixed serialization with marker is itself a
> > > >serializer type and should have a serializer of its own...
> > > >
> > > >-Jay
> > > >
> > > >On Fri, Dec 5, 2014 at 3:48 PM, Sriram Subramanian <
> > > >srsubramanian@linkedin.com.invalid> wrote:
> > > >
> > > >> This thread has diverged multiple times now and it would be worth
> > > >> summarizing them.
> > > >>
> > > >> There seems to be the following points of discussion -
> > > >>
> > > >> 1. Can we keep the serialization semantics outside the Producer
> > > >>interface
> > > >> and have simple bytes in / bytes out for the interface (This is what
> > we
> > > >> have today).
> > > >>
> > > >> The points for this is to keep the interface simple and usage easy to
> > > >> understand. The points against this is that it gets hard to share
> > common
> > > >> usage patterns around serialization/message validations for the
> > future.
> > > >>
> > > >> 2. Can we create a wrapper producer that does the serialization and
> > have
> > > >> different variants of it for different data formats?
> > > >>
> > > >> The points for this is again to keep the main API clean. The points
> > > >> against this is that it duplicates the API, increases the surface area
> > > >>and
> > > >> creates redundancy for a minor addition.
> > > >>
> > > >> 3. Do we need to support different data types per record? The current
> > > >> interface (bytes in/bytes out) lets you instantiate one producer and
> > use
> > > >> it to send multiple data formats. There seems to be some valid use
> > cases
> > > >> for this.
> > > >>
> > > >> I have still not seen a strong argument against not having this
> > > >> functionality. Can someone provide their views on why we don't need
> > this
> > > >> support that is possible with the current API?
> > > >>
> > > >> One possible approach for the per record serialization would be to
> > > >>define
> > > >>
> > > >> public interface SerDe<K,V> {
> > > >>   public byte[] serializeKey();
> > > >>
> > > >>   public K deserializeKey();
> > > >>
> > > >>   public byte[] serializeValue();
> > > >>
> > > >>   public V deserializeValue();
> > > >> }
> > > >>
> > > >> This would be used by both the Producer and the Consumer.
> > > >>
> > > >> The send APIs can then be
> > > >>
> > > >> public Future<RecordMetadata> send(ProducerRecord<K,V> record);
> > > >> public Future<RecordMetadata> send(ProducerRecord<K,V> record,
> > Callback
> > > >> callback);
> > > >>
> > > >>
> > > >> public Future<RecordMetadata> send(ProducerRecord<K,V> record,
> > > >>SerDe<K,V>
> > > >> serde);
> > > >>
> > > >> public Future<RecordMetadata> send(ProducerRecord<K,V> record,
> > > >>SerDe<K,V>
> > > >> serde, Callback callback);
> > > >>
> > > >>
> > > >> A default SerDe can be set in the config. The producer would use the
> > > >> default from the config if the non-serde send APIs are used. The
> > > >>downside
> > > >> to this approach is that we would need to have four variants of Send
> > API
> > > >> for the Producer.
> > > >>
> > > >>
> > > >>
> > > >>
> > > >>
> > > >>
> > > >> On 12/5/14 3:16 PM, "Jun Rao" <ju...@confluent.io> wrote:
> > > >>
> > > >> >Jiangjie,
> > > >> >
> > > >> >The issue with adding the serializer in ProducerRecord is that you
> > > >>need to
> > > >> >implement all combinations of serializers for key and value. So,
> > > >>instead
> > > >> >of
> > > >> >just implementing int and string serializers, you will have to
> > > >>implement
> > > >> >all 4 combinations.
> > > >> >
> > > >> >Adding a new producer constructor like Producer<K,
> > V>(KeySerializer<K>,
> > > >> >ValueSerializer<V>, Properties properties) can be useful.
> > > >> >
> > > >> >Thanks,
> > > >> >
> > > >> >Jun
> > > >> >
> > > >> >On Thu, Dec 4, 2014 at 10:33 AM, Jiangjie Qin
> > > >><jq...@linkedin.com.invalid>
> > > >> >wrote:
> > > >> >
> > > >> >>
> > > >> >> I'm just thinking instead of binding serialization with producer,
> > > >> >>another
> > > >> >> option is to bind serializer/deserializer with
> > > >> >> ProducerRecord/ConsumerRecord (please see the detail proposal
> > below.)
> > > >> >>            The arguments for this option is:
> > > >> >>         A. A single producer could send different message types.
> > > >>There
> > > >> >>are
> > > >> >> several use cases in LinkedIn for per record serializer
> > > >> >>         - In Samza, there are some in-stream order-sensitive
> > control
> > > >> >> messages
> > > >> >> having different deserializer from other messages.
> > > >> >>         - There are use cases which need support for sending both
> > > >>Avro
> > > >> >> messages
> > > >> >> and raw bytes.
> > > >> >>         - Some use cases needs to deserialize some Avro messages
> > into
> > > >> >> generic
> > > >> >> record and some other messages into specific record.
> > > >> >>         B. In current proposal, the serializer/deserilizer is
> > > >> >>instantiated
> > > >> >> according to config. Compared with that, binding serializer with
> > > >> >> ProducerRecord and ConsumerRecord is less error prone.
> > > >> >>
> > > >> >>
> > > >> >>         This option includes the following changes:
> > > >> >>         A. Add serializer and deserializer interfaces to replace
> > > >> >>serializer
> > > >> >> instance from config.
> > > >> >>                 Public interface Serializer <K, V> {
> > > >> >>                         public byte[] serializeKey(K key);
> > > >> >>                         public byte[] serializeValue(V value);
> > > >> >>                 }
> > > >> >>                 Public interface deserializer <K, V> {
> > > >> >>                         Public K deserializeKey(byte[] key);
> > > >> >>                         public V deserializeValue(byte[] value);
> > > >> >>                 }
> > > >> >>
> > > >> >>         B. Make ProducerRecord and ConsumerRecord abstract class
> > > >> >> implementing
> > > >> >> Serializer <K, V> and Deserializer <K, V> respectively.
> > > >> >>                 Public abstract class ProducerRecord <K, V>
> > > >>implements
> > > >> >> Serializer <K, V>
> > > >> >> {...}
> > > >> >>                 Public abstract class ConsumerRecord <K, V>
> > > >>implements
> > > >> >> Deserializer <K,
> > > >> >> V> {...}
> > > >> >>
> > > >> >>         C. Instead of instantiate the serializer/Deserializer from
> > > >> >>config,
> > > >> >> let
> > > >> >> concrete ProducerRecord/ConsumerRecord extends the abstract class
> > and
> > > >> >> override the serialize/deserialize methods.
> > > >> >>
> > > >> >>                 Public class AvroProducerRecord extends
> > > >>ProducerRecord
> > > >> >> <String,
> > > >> >> GenericRecord> {
> > > >> >>                         ...
> > > >> >>                         @Override
> > > >> >>                         Public byte[] serializeKey(String key) {Š}
> > > >> >>                         @Override
> > > >> >>                         public byte[] serializeValue(GenericRecord
> > > >> >>value);
> > > >> >>                 }
> > > >> >>
> > > >> >>                 Public class AvroConsumerRecord extends
> > > >>ConsumerRecord
> > > >> >> <String,
> > > >> >> GenericRecord> {
> > > >> >>                         ...
> > > >> >>                         @Override
> > > >> >>                         Public K deserializeKey(byte[] key) {Š}
> > > >> >>                         @Override
> > > >> >>                         public V deserializeValue(byte[] value);
> > > >> >>                 }
> > > >> >>
> > > >> >>         D. The producer API changes to
> > > >> >>                 Public class KafkaProducer {
> > > >> >>                         ...
> > > >> >>
> > > >> >>                         Future<RecordMetadata> send (ProducerRecord
> > > >><K,
> > > >> >>V>
> > > >> >> record) {
> > > >> >>                                 ...
> > > >> >>                                 K key =
> > > >>record.serializeKey(record.key);
> > > >> >>                                 V value =
> > > >> >> record.serializedValue(record.value);
> > > >> >>                                 BytesProducerRecord
> > > >>bytesProducerRecord
> > > >> >>=
> > > >> >> new
> > > >> >> BytesProducerRecord(topic, partition, key, value);
> > > >> >>                                 ...
> > > >> >>                         }
> > > >> >>                         ...
> > > >> >>                 }
> > > >> >>
> > > >> >>
> > > >> >>
> > > >> >> We also had some brainstorm in LinkedIn and here are the feedbacks:
> > > >> >>
> > > >> >> If the community decide to add the serialization back to new
> > > >>producer,
> > > >> >> besides current proposal which changes new producer API to be a
> > > >> >>template,
> > > >> >> there are some other options raised during our discussion:
> > > >> >>         1) Rather than change current new producer API, we can
> > > >>provide a
> > > >> >> wrapper
> > > >> >> of current new producer (e.g. KafkaSerializedProducer) and make it
> > > >> >> available to users. As there is value in the simplicity of current
> > > >>API.
> > > >> >>
> > > >> >>         2) If we decide to go with tempalated new producer API,
> > > >> >>according
> > > >> >> to
> > > >> >> experience in LinkedIn, it might worth considering to instantiate
> > the
> > > >> >> serializer in code instead of from config so we can avoid runtime
> > > >>errors
> > > >> >> due to dynamic instantiation from config, which is more error
> > prone.
> > > >>If
> > > >> >> that is the case, the producer API could be changed to something
> > > >>like:
> > > >> >>                 producer = new Producer<K, V>(KeySerializer<K>,
> > > >> >> ValueSerializer<V>)
> > > >> >>
> > > >> >> --Jiangjie (Becket) Qin
> > > >> >>
> > > >> >>
> > > >> >> On 11/24/14, 5:58 PM, "Jun Rao" <ju...@gmail.com> 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
> > > >> >>
> > > >> >>
> > > >>
> > > >>
> > >
> >
> >


Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Jun Rao <ju...@confluent.io>.
Joel,

It's just that if the serializer/deserializer is not part of the API, you
can only encourage people to use it through documentation. However, not
everyone will read the documentation if it's not directly used in the API.

Thanks,

Jun

On Mon, Dec 15, 2014 at 2:11 AM, Joel Koshy <jj...@gmail.com> wrote:

> (sorry about the late follow-up late - I'm traveling most of this
> month)
>
> I'm likely missing something obvious, but I find the following to be a
> somewhat vague point that has been mentioned more than once in this
> thread without a clear explanation. i.e., why is it hard to share a
> serializer/deserializer implementation and just have the clients call
> it before a send/receive? What "usage pattern" cannot be supported by
> the simpler API?
>
> > 1. Can we keep the serialization semantics outside the Producer interface
> > and have simple bytes in / bytes out for the interface (This is what we
> > have today).
> >
> > The points for this is to keep the interface simple and usage easy to
> > understand. The points against this is that it gets hard to share common
> > usage patterns around serialization/message validations for the future.
>
>
> On Tue, Dec 09, 2014 at 03:51:08AM +0000, Sriram Subramanian wrote:
> > Thank you Jay. I agree with the issue that you point w.r.t paired
> > serializers. I also think having mix serialization types is rare. To get
> > the current behavior, one can simply use a ByteArraySerializer. This is
> > best understood by talking with many customers and you seem to have done
> > that. I am convinced about the change.
> >
> > For the rest who gave -1 or 0 for this proposal, does the answers for the
> > three points(updated) below seem reasonable? Are these explanations
> > convincing?
> >
> >
> > 1. Can we keep the serialization semantics outside the Producer interface
> > and have simple bytes in / bytes out for the interface (This is what we
> > have today).
> >
> > The points for this is to keep the interface simple and usage easy to
> > understand. The points against this is that it gets hard to share common
> > usage patterns around serialization/message validations for the future.
> >
> > 2. Can we create a wrapper producer that does the serialization and have
> > different variants of it for different data formats?
> >
> > The points for this is again to keep the main API clean. The points
> > against this is that it duplicates the API, increases the surface area
> and
> > creates redundancy for a minor addition.
> >
> > 3. Do we need to support different data types per record? The current
> > interface (bytes in/bytes out) lets you instantiate one producer and use
> > it to send multiple data formats. There seems to be some valid use cases
> > for this.
> >
> >
> > Mixed serialization types are rare based on interactions with customers.
> > To get the current behavior, one can simply use a ByteArraySerializer.
> >
> > On 12/5/14 5:00 PM, "Jay Kreps" <ja...@confluent.io> wrote:
> >
> > >Hey Sriram,
> > >
> > >Thanks! I think this is a very helpful summary.
> > >
> > >Let me try to address your point about passing in the serde at send
> time.
> > >
> > >I think the first objection is really to the paired key/value serializer
> > >interfaces. This leads to kind of a weird combinatorial thing where you
> > >would have an avro/avro serializer a string/avro serializer, a pb/pb
> > >serializer, and a string/pb serializer, and so on. But your proposal
> would
> > >work as well with separate serializers for key and value.
> > >
> > >I think the downside is just the one you call out--that this is a corner
> > >case and you end up with two versions of all the apis to support it.
> This
> > >also makes the serializer api more annoying to implement. I think the
> > >alternative solution to this case and any other we can give people is
> just
> > >configuring ByteArraySerializer which gives you basically the api that
> you
> > >have now with byte arrays. If this is incredibly common then this would
> be
> > >a silly solution, but I guess the belief is that these cases are rare
> and
> > >a
> > >really well implemented avro or json serializer should be 100% of what
> > >most
> > >people need.
> > >
> > >In practice the cases that actually mix serialization types in a single
> > >stream are pretty rare I think just because the consumer then has the
> > >problem of guessing how to deserialize, so most of these will end up
> with
> > >at least some marker or schema id or whatever that tells you how to read
> > >the data. Arguable this mixed serialization with marker is itself a
> > >serializer type and should have a serializer of its own...
> > >
> > >-Jay
> > >
> > >On Fri, Dec 5, 2014 at 3:48 PM, Sriram Subramanian <
> > >srsubramanian@linkedin.com.invalid> wrote:
> > >
> > >> This thread has diverged multiple times now and it would be worth
> > >> summarizing them.
> > >>
> > >> There seems to be the following points of discussion -
> > >>
> > >> 1. Can we keep the serialization semantics outside the Producer
> > >>interface
> > >> and have simple bytes in / bytes out for the interface (This is what
> we
> > >> have today).
> > >>
> > >> The points for this is to keep the interface simple and usage easy to
> > >> understand. The points against this is that it gets hard to share
> common
> > >> usage patterns around serialization/message validations for the
> future.
> > >>
> > >> 2. Can we create a wrapper producer that does the serialization and
> have
> > >> different variants of it for different data formats?
> > >>
> > >> The points for this is again to keep the main API clean. The points
> > >> against this is that it duplicates the API, increases the surface area
> > >>and
> > >> creates redundancy for a minor addition.
> > >>
> > >> 3. Do we need to support different data types per record? The current
> > >> interface (bytes in/bytes out) lets you instantiate one producer and
> use
> > >> it to send multiple data formats. There seems to be some valid use
> cases
> > >> for this.
> > >>
> > >> I have still not seen a strong argument against not having this
> > >> functionality. Can someone provide their views on why we don't need
> this
> > >> support that is possible with the current API?
> > >>
> > >> One possible approach for the per record serialization would be to
> > >>define
> > >>
> > >> public interface SerDe<K,V> {
> > >>   public byte[] serializeKey();
> > >>
> > >>   public K deserializeKey();
> > >>
> > >>   public byte[] serializeValue();
> > >>
> > >>   public V deserializeValue();
> > >> }
> > >>
> > >> This would be used by both the Producer and the Consumer.
> > >>
> > >> The send APIs can then be
> > >>
> > >> public Future<RecordMetadata> send(ProducerRecord<K,V> record);
> > >> public Future<RecordMetadata> send(ProducerRecord<K,V> record,
> Callback
> > >> callback);
> > >>
> > >>
> > >> public Future<RecordMetadata> send(ProducerRecord<K,V> record,
> > >>SerDe<K,V>
> > >> serde);
> > >>
> > >> public Future<RecordMetadata> send(ProducerRecord<K,V> record,
> > >>SerDe<K,V>
> > >> serde, Callback callback);
> > >>
> > >>
> > >> A default SerDe can be set in the config. The producer would use the
> > >> default from the config if the non-serde send APIs are used. The
> > >>downside
> > >> to this approach is that we would need to have four variants of Send
> API
> > >> for the Producer.
> > >>
> > >>
> > >>
> > >>
> > >>
> > >>
> > >> On 12/5/14 3:16 PM, "Jun Rao" <ju...@confluent.io> wrote:
> > >>
> > >> >Jiangjie,
> > >> >
> > >> >The issue with adding the serializer in ProducerRecord is that you
> > >>need to
> > >> >implement all combinations of serializers for key and value. So,
> > >>instead
> > >> >of
> > >> >just implementing int and string serializers, you will have to
> > >>implement
> > >> >all 4 combinations.
> > >> >
> > >> >Adding a new producer constructor like Producer<K,
> V>(KeySerializer<K>,
> > >> >ValueSerializer<V>, Properties properties) can be useful.
> > >> >
> > >> >Thanks,
> > >> >
> > >> >Jun
> > >> >
> > >> >On Thu, Dec 4, 2014 at 10:33 AM, Jiangjie Qin
> > >><jq...@linkedin.com.invalid>
> > >> >wrote:
> > >> >
> > >> >>
> > >> >> I'm just thinking instead of binding serialization with producer,
> > >> >>another
> > >> >> option is to bind serializer/deserializer with
> > >> >> ProducerRecord/ConsumerRecord (please see the detail proposal
> below.)
> > >> >>            The arguments for this option is:
> > >> >>         A. A single producer could send different message types.
> > >>There
> > >> >>are
> > >> >> several use cases in LinkedIn for per record serializer
> > >> >>         - In Samza, there are some in-stream order-sensitive
> control
> > >> >> messages
> > >> >> having different deserializer from other messages.
> > >> >>         - There are use cases which need support for sending both
> > >>Avro
> > >> >> messages
> > >> >> and raw bytes.
> > >> >>         - Some use cases needs to deserialize some Avro messages
> into
> > >> >> generic
> > >> >> record and some other messages into specific record.
> > >> >>         B. In current proposal, the serializer/deserilizer is
> > >> >>instantiated
> > >> >> according to config. Compared with that, binding serializer with
> > >> >> ProducerRecord and ConsumerRecord is less error prone.
> > >> >>
> > >> >>
> > >> >>         This option includes the following changes:
> > >> >>         A. Add serializer and deserializer interfaces to replace
> > >> >>serializer
> > >> >> instance from config.
> > >> >>                 Public interface Serializer <K, V> {
> > >> >>                         public byte[] serializeKey(K key);
> > >> >>                         public byte[] serializeValue(V value);
> > >> >>                 }
> > >> >>                 Public interface deserializer <K, V> {
> > >> >>                         Public K deserializeKey(byte[] key);
> > >> >>                         public V deserializeValue(byte[] value);
> > >> >>                 }
> > >> >>
> > >> >>         B. Make ProducerRecord and ConsumerRecord abstract class
> > >> >> implementing
> > >> >> Serializer <K, V> and Deserializer <K, V> respectively.
> > >> >>                 Public abstract class ProducerRecord <K, V>
> > >>implements
> > >> >> Serializer <K, V>
> > >> >> {...}
> > >> >>                 Public abstract class ConsumerRecord <K, V>
> > >>implements
> > >> >> Deserializer <K,
> > >> >> V> {...}
> > >> >>
> > >> >>         C. Instead of instantiate the serializer/Deserializer from
> > >> >>config,
> > >> >> let
> > >> >> concrete ProducerRecord/ConsumerRecord extends the abstract class
> and
> > >> >> override the serialize/deserialize methods.
> > >> >>
> > >> >>                 Public class AvroProducerRecord extends
> > >>ProducerRecord
> > >> >> <String,
> > >> >> GenericRecord> {
> > >> >>                         ...
> > >> >>                         @Override
> > >> >>                         Public byte[] serializeKey(String key) {Š}
> > >> >>                         @Override
> > >> >>                         public byte[] serializeValue(GenericRecord
> > >> >>value);
> > >> >>                 }
> > >> >>
> > >> >>                 Public class AvroConsumerRecord extends
> > >>ConsumerRecord
> > >> >> <String,
> > >> >> GenericRecord> {
> > >> >>                         ...
> > >> >>                         @Override
> > >> >>                         Public K deserializeKey(byte[] key) {Š}
> > >> >>                         @Override
> > >> >>                         public V deserializeValue(byte[] value);
> > >> >>                 }
> > >> >>
> > >> >>         D. The producer API changes to
> > >> >>                 Public class KafkaProducer {
> > >> >>                         ...
> > >> >>
> > >> >>                         Future<RecordMetadata> send (ProducerRecord
> > >><K,
> > >> >>V>
> > >> >> record) {
> > >> >>                                 ...
> > >> >>                                 K key =
> > >>record.serializeKey(record.key);
> > >> >>                                 V value =
> > >> >> record.serializedValue(record.value);
> > >> >>                                 BytesProducerRecord
> > >>bytesProducerRecord
> > >> >>=
> > >> >> new
> > >> >> BytesProducerRecord(topic, partition, key, value);
> > >> >>                                 ...
> > >> >>                         }
> > >> >>                         ...
> > >> >>                 }
> > >> >>
> > >> >>
> > >> >>
> > >> >> We also had some brainstorm in LinkedIn and here are the feedbacks:
> > >> >>
> > >> >> If the community decide to add the serialization back to new
> > >>producer,
> > >> >> besides current proposal which changes new producer API to be a
> > >> >>template,
> > >> >> there are some other options raised during our discussion:
> > >> >>         1) Rather than change current new producer API, we can
> > >>provide a
> > >> >> wrapper
> > >> >> of current new producer (e.g. KafkaSerializedProducer) and make it
> > >> >> available to users. As there is value in the simplicity of current
> > >>API.
> > >> >>
> > >> >>         2) If we decide to go with tempalated new producer API,
> > >> >>according
> > >> >> to
> > >> >> experience in LinkedIn, it might worth considering to instantiate
> the
> > >> >> serializer in code instead of from config so we can avoid runtime
> > >>errors
> > >> >> due to dynamic instantiation from config, which is more error
> prone.
> > >>If
> > >> >> that is the case, the producer API could be changed to something
> > >>like:
> > >> >>                 producer = new Producer<K, V>(KeySerializer<K>,
> > >> >> ValueSerializer<V>)
> > >> >>
> > >> >> --Jiangjie (Becket) Qin
> > >> >>
> > >> >>
> > >> >> On 11/24/14, 5:58 PM, "Jun Rao" <ju...@gmail.com> 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
> > >> >>
> > >> >>
> > >>
> > >>
> >
>
>

Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Jun Rao <ju...@confluent.io>.
Joel,

It's just that if the serializer/deserializer is not part of the API, you
can only encourage people to use it through documentation. However, not
everyone will read the documentation if it's not directly used in the API.

Thanks,

Jun

On Mon, Dec 15, 2014 at 2:11 AM, Joel Koshy <jj...@gmail.com> wrote:

> (sorry about the late follow-up late - I'm traveling most of this
> month)
>
> I'm likely missing something obvious, but I find the following to be a
> somewhat vague point that has been mentioned more than once in this
> thread without a clear explanation. i.e., why is it hard to share a
> serializer/deserializer implementation and just have the clients call
> it before a send/receive? What "usage pattern" cannot be supported by
> the simpler API?
>
> > 1. Can we keep the serialization semantics outside the Producer interface
> > and have simple bytes in / bytes out for the interface (This is what we
> > have today).
> >
> > The points for this is to keep the interface simple and usage easy to
> > understand. The points against this is that it gets hard to share common
> > usage patterns around serialization/message validations for the future.
>
>
> On Tue, Dec 09, 2014 at 03:51:08AM +0000, Sriram Subramanian wrote:
> > Thank you Jay. I agree with the issue that you point w.r.t paired
> > serializers. I also think having mix serialization types is rare. To get
> > the current behavior, one can simply use a ByteArraySerializer. This is
> > best understood by talking with many customers and you seem to have done
> > that. I am convinced about the change.
> >
> > For the rest who gave -1 or 0 for this proposal, does the answers for the
> > three points(updated) below seem reasonable? Are these explanations
> > convincing?
> >
> >
> > 1. Can we keep the serialization semantics outside the Producer interface
> > and have simple bytes in / bytes out for the interface (This is what we
> > have today).
> >
> > The points for this is to keep the interface simple and usage easy to
> > understand. The points against this is that it gets hard to share common
> > usage patterns around serialization/message validations for the future.
> >
> > 2. Can we create a wrapper producer that does the serialization and have
> > different variants of it for different data formats?
> >
> > The points for this is again to keep the main API clean. The points
> > against this is that it duplicates the API, increases the surface area
> and
> > creates redundancy for a minor addition.
> >
> > 3. Do we need to support different data types per record? The current
> > interface (bytes in/bytes out) lets you instantiate one producer and use
> > it to send multiple data formats. There seems to be some valid use cases
> > for this.
> >
> >
> > Mixed serialization types are rare based on interactions with customers.
> > To get the current behavior, one can simply use a ByteArraySerializer.
> >
> > On 12/5/14 5:00 PM, "Jay Kreps" <ja...@confluent.io> wrote:
> >
> > >Hey Sriram,
> > >
> > >Thanks! I think this is a very helpful summary.
> > >
> > >Let me try to address your point about passing in the serde at send
> time.
> > >
> > >I think the first objection is really to the paired key/value serializer
> > >interfaces. This leads to kind of a weird combinatorial thing where you
> > >would have an avro/avro serializer a string/avro serializer, a pb/pb
> > >serializer, and a string/pb serializer, and so on. But your proposal
> would
> > >work as well with separate serializers for key and value.
> > >
> > >I think the downside is just the one you call out--that this is a corner
> > >case and you end up with two versions of all the apis to support it.
> This
> > >also makes the serializer api more annoying to implement. I think the
> > >alternative solution to this case and any other we can give people is
> just
> > >configuring ByteArraySerializer which gives you basically the api that
> you
> > >have now with byte arrays. If this is incredibly common then this would
> be
> > >a silly solution, but I guess the belief is that these cases are rare
> and
> > >a
> > >really well implemented avro or json serializer should be 100% of what
> > >most
> > >people need.
> > >
> > >In practice the cases that actually mix serialization types in a single
> > >stream are pretty rare I think just because the consumer then has the
> > >problem of guessing how to deserialize, so most of these will end up
> with
> > >at least some marker or schema id or whatever that tells you how to read
> > >the data. Arguable this mixed serialization with marker is itself a
> > >serializer type and should have a serializer of its own...
> > >
> > >-Jay
> > >
> > >On Fri, Dec 5, 2014 at 3:48 PM, Sriram Subramanian <
> > >srsubramanian@linkedin.com.invalid> wrote:
> > >
> > >> This thread has diverged multiple times now and it would be worth
> > >> summarizing them.
> > >>
> > >> There seems to be the following points of discussion -
> > >>
> > >> 1. Can we keep the serialization semantics outside the Producer
> > >>interface
> > >> and have simple bytes in / bytes out for the interface (This is what
> we
> > >> have today).
> > >>
> > >> The points for this is to keep the interface simple and usage easy to
> > >> understand. The points against this is that it gets hard to share
> common
> > >> usage patterns around serialization/message validations for the
> future.
> > >>
> > >> 2. Can we create a wrapper producer that does the serialization and
> have
> > >> different variants of it for different data formats?
> > >>
> > >> The points for this is again to keep the main API clean. The points
> > >> against this is that it duplicates the API, increases the surface area
> > >>and
> > >> creates redundancy for a minor addition.
> > >>
> > >> 3. Do we need to support different data types per record? The current
> > >> interface (bytes in/bytes out) lets you instantiate one producer and
> use
> > >> it to send multiple data formats. There seems to be some valid use
> cases
> > >> for this.
> > >>
> > >> I have still not seen a strong argument against not having this
> > >> functionality. Can someone provide their views on why we don't need
> this
> > >> support that is possible with the current API?
> > >>
> > >> One possible approach for the per record serialization would be to
> > >>define
> > >>
> > >> public interface SerDe<K,V> {
> > >>   public byte[] serializeKey();
> > >>
> > >>   public K deserializeKey();
> > >>
> > >>   public byte[] serializeValue();
> > >>
> > >>   public V deserializeValue();
> > >> }
> > >>
> > >> This would be used by both the Producer and the Consumer.
> > >>
> > >> The send APIs can then be
> > >>
> > >> public Future<RecordMetadata> send(ProducerRecord<K,V> record);
> > >> public Future<RecordMetadata> send(ProducerRecord<K,V> record,
> Callback
> > >> callback);
> > >>
> > >>
> > >> public Future<RecordMetadata> send(ProducerRecord<K,V> record,
> > >>SerDe<K,V>
> > >> serde);
> > >>
> > >> public Future<RecordMetadata> send(ProducerRecord<K,V> record,
> > >>SerDe<K,V>
> > >> serde, Callback callback);
> > >>
> > >>
> > >> A default SerDe can be set in the config. The producer would use the
> > >> default from the config if the non-serde send APIs are used. The
> > >>downside
> > >> to this approach is that we would need to have four variants of Send
> API
> > >> for the Producer.
> > >>
> > >>
> > >>
> > >>
> > >>
> > >>
> > >> On 12/5/14 3:16 PM, "Jun Rao" <ju...@confluent.io> wrote:
> > >>
> > >> >Jiangjie,
> > >> >
> > >> >The issue with adding the serializer in ProducerRecord is that you
> > >>need to
> > >> >implement all combinations of serializers for key and value. So,
> > >>instead
> > >> >of
> > >> >just implementing int and string serializers, you will have to
> > >>implement
> > >> >all 4 combinations.
> > >> >
> > >> >Adding a new producer constructor like Producer<K,
> V>(KeySerializer<K>,
> > >> >ValueSerializer<V>, Properties properties) can be useful.
> > >> >
> > >> >Thanks,
> > >> >
> > >> >Jun
> > >> >
> > >> >On Thu, Dec 4, 2014 at 10:33 AM, Jiangjie Qin
> > >><jq...@linkedin.com.invalid>
> > >> >wrote:
> > >> >
> > >> >>
> > >> >> I'm just thinking instead of binding serialization with producer,
> > >> >>another
> > >> >> option is to bind serializer/deserializer with
> > >> >> ProducerRecord/ConsumerRecord (please see the detail proposal
> below.)
> > >> >>            The arguments for this option is:
> > >> >>         A. A single producer could send different message types.
> > >>There
> > >> >>are
> > >> >> several use cases in LinkedIn for per record serializer
> > >> >>         - In Samza, there are some in-stream order-sensitive
> control
> > >> >> messages
> > >> >> having different deserializer from other messages.
> > >> >>         - There are use cases which need support for sending both
> > >>Avro
> > >> >> messages
> > >> >> and raw bytes.
> > >> >>         - Some use cases needs to deserialize some Avro messages
> into
> > >> >> generic
> > >> >> record and some other messages into specific record.
> > >> >>         B. In current proposal, the serializer/deserilizer is
> > >> >>instantiated
> > >> >> according to config. Compared with that, binding serializer with
> > >> >> ProducerRecord and ConsumerRecord is less error prone.
> > >> >>
> > >> >>
> > >> >>         This option includes the following changes:
> > >> >>         A. Add serializer and deserializer interfaces to replace
> > >> >>serializer
> > >> >> instance from config.
> > >> >>                 Public interface Serializer <K, V> {
> > >> >>                         public byte[] serializeKey(K key);
> > >> >>                         public byte[] serializeValue(V value);
> > >> >>                 }
> > >> >>                 Public interface deserializer <K, V> {
> > >> >>                         Public K deserializeKey(byte[] key);
> > >> >>                         public V deserializeValue(byte[] value);
> > >> >>                 }
> > >> >>
> > >> >>         B. Make ProducerRecord and ConsumerRecord abstract class
> > >> >> implementing
> > >> >> Serializer <K, V> and Deserializer <K, V> respectively.
> > >> >>                 Public abstract class ProducerRecord <K, V>
> > >>implements
> > >> >> Serializer <K, V>
> > >> >> {...}
> > >> >>                 Public abstract class ConsumerRecord <K, V>
> > >>implements
> > >> >> Deserializer <K,
> > >> >> V> {...}
> > >> >>
> > >> >>         C. Instead of instantiate the serializer/Deserializer from
> > >> >>config,
> > >> >> let
> > >> >> concrete ProducerRecord/ConsumerRecord extends the abstract class
> and
> > >> >> override the serialize/deserialize methods.
> > >> >>
> > >> >>                 Public class AvroProducerRecord extends
> > >>ProducerRecord
> > >> >> <String,
> > >> >> GenericRecord> {
> > >> >>                         ...
> > >> >>                         @Override
> > >> >>                         Public byte[] serializeKey(String key) {Š}
> > >> >>                         @Override
> > >> >>                         public byte[] serializeValue(GenericRecord
> > >> >>value);
> > >> >>                 }
> > >> >>
> > >> >>                 Public class AvroConsumerRecord extends
> > >>ConsumerRecord
> > >> >> <String,
> > >> >> GenericRecord> {
> > >> >>                         ...
> > >> >>                         @Override
> > >> >>                         Public K deserializeKey(byte[] key) {Š}
> > >> >>                         @Override
> > >> >>                         public V deserializeValue(byte[] value);
> > >> >>                 }
> > >> >>
> > >> >>         D. The producer API changes to
> > >> >>                 Public class KafkaProducer {
> > >> >>                         ...
> > >> >>
> > >> >>                         Future<RecordMetadata> send (ProducerRecord
> > >><K,
> > >> >>V>
> > >> >> record) {
> > >> >>                                 ...
> > >> >>                                 K key =
> > >>record.serializeKey(record.key);
> > >> >>                                 V value =
> > >> >> record.serializedValue(record.value);
> > >> >>                                 BytesProducerRecord
> > >>bytesProducerRecord
> > >> >>=
> > >> >> new
> > >> >> BytesProducerRecord(topic, partition, key, value);
> > >> >>                                 ...
> > >> >>                         }
> > >> >>                         ...
> > >> >>                 }
> > >> >>
> > >> >>
> > >> >>
> > >> >> We also had some brainstorm in LinkedIn and here are the feedbacks:
> > >> >>
> > >> >> If the community decide to add the serialization back to new
> > >>producer,
> > >> >> besides current proposal which changes new producer API to be a
> > >> >>template,
> > >> >> there are some other options raised during our discussion:
> > >> >>         1) Rather than change current new producer API, we can
> > >>provide a
> > >> >> wrapper
> > >> >> of current new producer (e.g. KafkaSerializedProducer) and make it
> > >> >> available to users. As there is value in the simplicity of current
> > >>API.
> > >> >>
> > >> >>         2) If we decide to go with tempalated new producer API,
> > >> >>according
> > >> >> to
> > >> >> experience in LinkedIn, it might worth considering to instantiate
> the
> > >> >> serializer in code instead of from config so we can avoid runtime
> > >>errors
> > >> >> due to dynamic instantiation from config, which is more error
> prone.
> > >>If
> > >> >> that is the case, the producer API could be changed to something
> > >>like:
> > >> >>                 producer = new Producer<K, V>(KeySerializer<K>,
> > >> >> ValueSerializer<V>)
> > >> >>
> > >> >> --Jiangjie (Becket) Qin
> > >> >>
> > >> >>
> > >> >> On 11/24/14, 5:58 PM, "Jun Rao" <ju...@gmail.com> 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
> > >> >>
> > >> >>
> > >>
> > >>
> >
>
>

Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Joel Koshy <jj...@gmail.com>.
(sorry about the late follow-up late - I'm traveling most of this
month)

I'm likely missing something obvious, but I find the following to be a
somewhat vague point that has been mentioned more than once in this
thread without a clear explanation. i.e., why is it hard to share a
serializer/deserializer implementation and just have the clients call
it before a send/receive? What "usage pattern" cannot be supported by
the simpler API?

> 1. Can we keep the serialization semantics outside the Producer interface
> and have simple bytes in / bytes out for the interface (This is what we
> have today).
> 
> The points for this is to keep the interface simple and usage easy to
> understand. The points against this is that it gets hard to share common
> usage patterns around serialization/message validations for the future.


On Tue, Dec 09, 2014 at 03:51:08AM +0000, Sriram Subramanian wrote:
> Thank you Jay. I agree with the issue that you point w.r.t paired
> serializers. I also think having mix serialization types is rare. To get
> the current behavior, one can simply use a ByteArraySerializer. This is
> best understood by talking with many customers and you seem to have done
> that. I am convinced about the change.
> 
> For the rest who gave -1 or 0 for this proposal, does the answers for the
> three points(updated) below seem reasonable? Are these explanations
> convincing? 
> 
> 
> 1. Can we keep the serialization semantics outside the Producer interface
> and have simple bytes in / bytes out for the interface (This is what we
> have today).
> 
> The points for this is to keep the interface simple and usage easy to
> understand. The points against this is that it gets hard to share common
> usage patterns around serialization/message validations for the future.
> 
> 2. Can we create a wrapper producer that does the serialization and have
> different variants of it for different data formats?
> 
> The points for this is again to keep the main API clean. The points
> against this is that it duplicates the API, increases the surface area and
> creates redundancy for a minor addition.
> 
> 3. Do we need to support different data types per record? The current
> interface (bytes in/bytes out) lets you instantiate one producer and use
> it to send multiple data formats. There seems to be some valid use cases
> for this.
> 
> 
> Mixed serialization types are rare based on interactions with customers.
> To get the current behavior, one can simply use a ByteArraySerializer.
> 
> On 12/5/14 5:00 PM, "Jay Kreps" <ja...@confluent.io> wrote:
> 
> >Hey Sriram,
> >
> >Thanks! I think this is a very helpful summary.
> >
> >Let me try to address your point about passing in the serde at send time.
> >
> >I think the first objection is really to the paired key/value serializer
> >interfaces. This leads to kind of a weird combinatorial thing where you
> >would have an avro/avro serializer a string/avro serializer, a pb/pb
> >serializer, and a string/pb serializer, and so on. But your proposal would
> >work as well with separate serializers for key and value.
> >
> >I think the downside is just the one you call out--that this is a corner
> >case and you end up with two versions of all the apis to support it. This
> >also makes the serializer api more annoying to implement. I think the
> >alternative solution to this case and any other we can give people is just
> >configuring ByteArraySerializer which gives you basically the api that you
> >have now with byte arrays. If this is incredibly common then this would be
> >a silly solution, but I guess the belief is that these cases are rare and
> >a
> >really well implemented avro or json serializer should be 100% of what
> >most
> >people need.
> >
> >In practice the cases that actually mix serialization types in a single
> >stream are pretty rare I think just because the consumer then has the
> >problem of guessing how to deserialize, so most of these will end up with
> >at least some marker or schema id or whatever that tells you how to read
> >the data. Arguable this mixed serialization with marker is itself a
> >serializer type and should have a serializer of its own...
> >
> >-Jay
> >
> >On Fri, Dec 5, 2014 at 3:48 PM, Sriram Subramanian <
> >srsubramanian@linkedin.com.invalid> wrote:
> >
> >> This thread has diverged multiple times now and it would be worth
> >> summarizing them.
> >>
> >> There seems to be the following points of discussion -
> >>
> >> 1. Can we keep the serialization semantics outside the Producer
> >>interface
> >> and have simple bytes in / bytes out for the interface (This is what we
> >> have today).
> >>
> >> The points for this is to keep the interface simple and usage easy to
> >> understand. The points against this is that it gets hard to share common
> >> usage patterns around serialization/message validations for the future.
> >>
> >> 2. Can we create a wrapper producer that does the serialization and have
> >> different variants of it for different data formats?
> >>
> >> The points for this is again to keep the main API clean. The points
> >> against this is that it duplicates the API, increases the surface area
> >>and
> >> creates redundancy for a minor addition.
> >>
> >> 3. Do we need to support different data types per record? The current
> >> interface (bytes in/bytes out) lets you instantiate one producer and use
> >> it to send multiple data formats. There seems to be some valid use cases
> >> for this.
> >>
> >> I have still not seen a strong argument against not having this
> >> functionality. Can someone provide their views on why we don't need this
> >> support that is possible with the current API?
> >>
> >> One possible approach for the per record serialization would be to
> >>define
> >>
> >> public interface SerDe<K,V> {
> >>   public byte[] serializeKey();
> >>
> >>   public K deserializeKey();
> >>
> >>   public byte[] serializeValue();
> >>
> >>   public V deserializeValue();
> >> }
> >>
> >> This would be used by both the Producer and the Consumer.
> >>
> >> The send APIs can then be
> >>
> >> public Future<RecordMetadata> send(ProducerRecord<K,V> record);
> >> public Future<RecordMetadata> send(ProducerRecord<K,V> record, Callback
> >> callback);
> >>
> >>
> >> public Future<RecordMetadata> send(ProducerRecord<K,V> record,
> >>SerDe<K,V>
> >> serde);
> >>
> >> public Future<RecordMetadata> send(ProducerRecord<K,V> record,
> >>SerDe<K,V>
> >> serde, Callback callback);
> >>
> >>
> >> A default SerDe can be set in the config. The producer would use the
> >> default from the config if the non-serde send APIs are used. The
> >>downside
> >> to this approach is that we would need to have four variants of Send API
> >> for the Producer.
> >>
> >>
> >>
> >>
> >>
> >>
> >> On 12/5/14 3:16 PM, "Jun Rao" <ju...@confluent.io> wrote:
> >>
> >> >Jiangjie,
> >> >
> >> >The issue with adding the serializer in ProducerRecord is that you
> >>need to
> >> >implement all combinations of serializers for key and value. So,
> >>instead
> >> >of
> >> >just implementing int and string serializers, you will have to
> >>implement
> >> >all 4 combinations.
> >> >
> >> >Adding a new producer constructor like Producer<K, V>(KeySerializer<K>,
> >> >ValueSerializer<V>, Properties properties) can be useful.
> >> >
> >> >Thanks,
> >> >
> >> >Jun
> >> >
> >> >On Thu, Dec 4, 2014 at 10:33 AM, Jiangjie Qin
> >><jq...@linkedin.com.invalid>
> >> >wrote:
> >> >
> >> >>
> >> >> I'm just thinking instead of binding serialization with producer,
> >> >>another
> >> >> option is to bind serializer/deserializer with
> >> >> ProducerRecord/ConsumerRecord (please see the detail proposal below.)
> >> >>            The arguments for this option is:
> >> >>         A. A single producer could send different message types.
> >>There
> >> >>are
> >> >> several use cases in LinkedIn for per record serializer
> >> >>         - In Samza, there are some in-stream order-sensitive control
> >> >> messages
> >> >> having different deserializer from other messages.
> >> >>         - There are use cases which need support for sending both
> >>Avro
> >> >> messages
> >> >> and raw bytes.
> >> >>         - Some use cases needs to deserialize some Avro messages into
> >> >> generic
> >> >> record and some other messages into specific record.
> >> >>         B. In current proposal, the serializer/deserilizer is
> >> >>instantiated
> >> >> according to config. Compared with that, binding serializer with
> >> >> ProducerRecord and ConsumerRecord is less error prone.
> >> >>
> >> >>
> >> >>         This option includes the following changes:
> >> >>         A. Add serializer and deserializer interfaces to replace
> >> >>serializer
> >> >> instance from config.
> >> >>                 Public interface Serializer <K, V> {
> >> >>                         public byte[] serializeKey(K key);
> >> >>                         public byte[] serializeValue(V value);
> >> >>                 }
> >> >>                 Public interface deserializer <K, V> {
> >> >>                         Public K deserializeKey(byte[] key);
> >> >>                         public V deserializeValue(byte[] value);
> >> >>                 }
> >> >>
> >> >>         B. Make ProducerRecord and ConsumerRecord abstract class
> >> >> implementing
> >> >> Serializer <K, V> and Deserializer <K, V> respectively.
> >> >>                 Public abstract class ProducerRecord <K, V>
> >>implements
> >> >> Serializer <K, V>
> >> >> {...}
> >> >>                 Public abstract class ConsumerRecord <K, V>
> >>implements
> >> >> Deserializer <K,
> >> >> V> {...}
> >> >>
> >> >>         C. Instead of instantiate the serializer/Deserializer from
> >> >>config,
> >> >> let
> >> >> concrete ProducerRecord/ConsumerRecord extends the abstract class and
> >> >> override the serialize/deserialize methods.
> >> >>
> >> >>                 Public class AvroProducerRecord extends
> >>ProducerRecord
> >> >> <String,
> >> >> GenericRecord> {
> >> >>                         ...
> >> >>                         @Override
> >> >>                         Public byte[] serializeKey(String key) {Š}
> >> >>                         @Override
> >> >>                         public byte[] serializeValue(GenericRecord
> >> >>value);
> >> >>                 }
> >> >>
> >> >>                 Public class AvroConsumerRecord extends
> >>ConsumerRecord
> >> >> <String,
> >> >> GenericRecord> {
> >> >>                         ...
> >> >>                         @Override
> >> >>                         Public K deserializeKey(byte[] key) {Š}
> >> >>                         @Override
> >> >>                         public V deserializeValue(byte[] value);
> >> >>                 }
> >> >>
> >> >>         D. The producer API changes to
> >> >>                 Public class KafkaProducer {
> >> >>                         ...
> >> >>
> >> >>                         Future<RecordMetadata> send (ProducerRecord
> >><K,
> >> >>V>
> >> >> record) {
> >> >>                                 ...
> >> >>                                 K key =
> >>record.serializeKey(record.key);
> >> >>                                 V value =
> >> >> record.serializedValue(record.value);
> >> >>                                 BytesProducerRecord
> >>bytesProducerRecord
> >> >>=
> >> >> new
> >> >> BytesProducerRecord(topic, partition, key, value);
> >> >>                                 ...
> >> >>                         }
> >> >>                         ...
> >> >>                 }
> >> >>
> >> >>
> >> >>
> >> >> We also had some brainstorm in LinkedIn and here are the feedbacks:
> >> >>
> >> >> If the community decide to add the serialization back to new
> >>producer,
> >> >> besides current proposal which changes new producer API to be a
> >> >>template,
> >> >> there are some other options raised during our discussion:
> >> >>         1) Rather than change current new producer API, we can
> >>provide a
> >> >> wrapper
> >> >> of current new producer (e.g. KafkaSerializedProducer) and make it
> >> >> available to users. As there is value in the simplicity of current
> >>API.
> >> >>
> >> >>         2) If we decide to go with tempalated new producer API,
> >> >>according
> >> >> to
> >> >> experience in LinkedIn, it might worth considering to instantiate the
> >> >> serializer in code instead of from config so we can avoid runtime
> >>errors
> >> >> due to dynamic instantiation from config, which is more error prone.
> >>If
> >> >> that is the case, the producer API could be changed to something
> >>like:
> >> >>                 producer = new Producer<K, V>(KeySerializer<K>,
> >> >> ValueSerializer<V>)
> >> >>
> >> >> --Jiangjie (Becket) Qin
> >> >>
> >> >>
> >> >> On 11/24/14, 5:58 PM, "Jun Rao" <ju...@gmail.com> 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
> >> >>
> >> >>
> >>
> >>
> 


Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Joel Koshy <jj...@gmail.com>.
(sorry about the late follow-up late - I'm traveling most of this
month)

I'm likely missing something obvious, but I find the following to be a
somewhat vague point that has been mentioned more than once in this
thread without a clear explanation. i.e., why is it hard to share a
serializer/deserializer implementation and just have the clients call
it before a send/receive? What "usage pattern" cannot be supported by
the simpler API?

> 1. Can we keep the serialization semantics outside the Producer interface
> and have simple bytes in / bytes out for the interface (This is what we
> have today).
> 
> The points for this is to keep the interface simple and usage easy to
> understand. The points against this is that it gets hard to share common
> usage patterns around serialization/message validations for the future.


On Tue, Dec 09, 2014 at 03:51:08AM +0000, Sriram Subramanian wrote:
> Thank you Jay. I agree with the issue that you point w.r.t paired
> serializers. I also think having mix serialization types is rare. To get
> the current behavior, one can simply use a ByteArraySerializer. This is
> best understood by talking with many customers and you seem to have done
> that. I am convinced about the change.
> 
> For the rest who gave -1 or 0 for this proposal, does the answers for the
> three points(updated) below seem reasonable? Are these explanations
> convincing? 
> 
> 
> 1. Can we keep the serialization semantics outside the Producer interface
> and have simple bytes in / bytes out for the interface (This is what we
> have today).
> 
> The points for this is to keep the interface simple and usage easy to
> understand. The points against this is that it gets hard to share common
> usage patterns around serialization/message validations for the future.
> 
> 2. Can we create a wrapper producer that does the serialization and have
> different variants of it for different data formats?
> 
> The points for this is again to keep the main API clean. The points
> against this is that it duplicates the API, increases the surface area and
> creates redundancy for a minor addition.
> 
> 3. Do we need to support different data types per record? The current
> interface (bytes in/bytes out) lets you instantiate one producer and use
> it to send multiple data formats. There seems to be some valid use cases
> for this.
> 
> 
> Mixed serialization types are rare based on interactions with customers.
> To get the current behavior, one can simply use a ByteArraySerializer.
> 
> On 12/5/14 5:00 PM, "Jay Kreps" <ja...@confluent.io> wrote:
> 
> >Hey Sriram,
> >
> >Thanks! I think this is a very helpful summary.
> >
> >Let me try to address your point about passing in the serde at send time.
> >
> >I think the first objection is really to the paired key/value serializer
> >interfaces. This leads to kind of a weird combinatorial thing where you
> >would have an avro/avro serializer a string/avro serializer, a pb/pb
> >serializer, and a string/pb serializer, and so on. But your proposal would
> >work as well with separate serializers for key and value.
> >
> >I think the downside is just the one you call out--that this is a corner
> >case and you end up with two versions of all the apis to support it. This
> >also makes the serializer api more annoying to implement. I think the
> >alternative solution to this case and any other we can give people is just
> >configuring ByteArraySerializer which gives you basically the api that you
> >have now with byte arrays. If this is incredibly common then this would be
> >a silly solution, but I guess the belief is that these cases are rare and
> >a
> >really well implemented avro or json serializer should be 100% of what
> >most
> >people need.
> >
> >In practice the cases that actually mix serialization types in a single
> >stream are pretty rare I think just because the consumer then has the
> >problem of guessing how to deserialize, so most of these will end up with
> >at least some marker or schema id or whatever that tells you how to read
> >the data. Arguable this mixed serialization with marker is itself a
> >serializer type and should have a serializer of its own...
> >
> >-Jay
> >
> >On Fri, Dec 5, 2014 at 3:48 PM, Sriram Subramanian <
> >srsubramanian@linkedin.com.invalid> wrote:
> >
> >> This thread has diverged multiple times now and it would be worth
> >> summarizing them.
> >>
> >> There seems to be the following points of discussion -
> >>
> >> 1. Can we keep the serialization semantics outside the Producer
> >>interface
> >> and have simple bytes in / bytes out for the interface (This is what we
> >> have today).
> >>
> >> The points for this is to keep the interface simple and usage easy to
> >> understand. The points against this is that it gets hard to share common
> >> usage patterns around serialization/message validations for the future.
> >>
> >> 2. Can we create a wrapper producer that does the serialization and have
> >> different variants of it for different data formats?
> >>
> >> The points for this is again to keep the main API clean. The points
> >> against this is that it duplicates the API, increases the surface area
> >>and
> >> creates redundancy for a minor addition.
> >>
> >> 3. Do we need to support different data types per record? The current
> >> interface (bytes in/bytes out) lets you instantiate one producer and use
> >> it to send multiple data formats. There seems to be some valid use cases
> >> for this.
> >>
> >> I have still not seen a strong argument against not having this
> >> functionality. Can someone provide their views on why we don't need this
> >> support that is possible with the current API?
> >>
> >> One possible approach for the per record serialization would be to
> >>define
> >>
> >> public interface SerDe<K,V> {
> >>   public byte[] serializeKey();
> >>
> >>   public K deserializeKey();
> >>
> >>   public byte[] serializeValue();
> >>
> >>   public V deserializeValue();
> >> }
> >>
> >> This would be used by both the Producer and the Consumer.
> >>
> >> The send APIs can then be
> >>
> >> public Future<RecordMetadata> send(ProducerRecord<K,V> record);
> >> public Future<RecordMetadata> send(ProducerRecord<K,V> record, Callback
> >> callback);
> >>
> >>
> >> public Future<RecordMetadata> send(ProducerRecord<K,V> record,
> >>SerDe<K,V>
> >> serde);
> >>
> >> public Future<RecordMetadata> send(ProducerRecord<K,V> record,
> >>SerDe<K,V>
> >> serde, Callback callback);
> >>
> >>
> >> A default SerDe can be set in the config. The producer would use the
> >> default from the config if the non-serde send APIs are used. The
> >>downside
> >> to this approach is that we would need to have four variants of Send API
> >> for the Producer.
> >>
> >>
> >>
> >>
> >>
> >>
> >> On 12/5/14 3:16 PM, "Jun Rao" <ju...@confluent.io> wrote:
> >>
> >> >Jiangjie,
> >> >
> >> >The issue with adding the serializer in ProducerRecord is that you
> >>need to
> >> >implement all combinations of serializers for key and value. So,
> >>instead
> >> >of
> >> >just implementing int and string serializers, you will have to
> >>implement
> >> >all 4 combinations.
> >> >
> >> >Adding a new producer constructor like Producer<K, V>(KeySerializer<K>,
> >> >ValueSerializer<V>, Properties properties) can be useful.
> >> >
> >> >Thanks,
> >> >
> >> >Jun
> >> >
> >> >On Thu, Dec 4, 2014 at 10:33 AM, Jiangjie Qin
> >><jq...@linkedin.com.invalid>
> >> >wrote:
> >> >
> >> >>
> >> >> I'm just thinking instead of binding serialization with producer,
> >> >>another
> >> >> option is to bind serializer/deserializer with
> >> >> ProducerRecord/ConsumerRecord (please see the detail proposal below.)
> >> >>            The arguments for this option is:
> >> >>         A. A single producer could send different message types.
> >>There
> >> >>are
> >> >> several use cases in LinkedIn for per record serializer
> >> >>         - In Samza, there are some in-stream order-sensitive control
> >> >> messages
> >> >> having different deserializer from other messages.
> >> >>         - There are use cases which need support for sending both
> >>Avro
> >> >> messages
> >> >> and raw bytes.
> >> >>         - Some use cases needs to deserialize some Avro messages into
> >> >> generic
> >> >> record and some other messages into specific record.
> >> >>         B. In current proposal, the serializer/deserilizer is
> >> >>instantiated
> >> >> according to config. Compared with that, binding serializer with
> >> >> ProducerRecord and ConsumerRecord is less error prone.
> >> >>
> >> >>
> >> >>         This option includes the following changes:
> >> >>         A. Add serializer and deserializer interfaces to replace
> >> >>serializer
> >> >> instance from config.
> >> >>                 Public interface Serializer <K, V> {
> >> >>                         public byte[] serializeKey(K key);
> >> >>                         public byte[] serializeValue(V value);
> >> >>                 }
> >> >>                 Public interface deserializer <K, V> {
> >> >>                         Public K deserializeKey(byte[] key);
> >> >>                         public V deserializeValue(byte[] value);
> >> >>                 }
> >> >>
> >> >>         B. Make ProducerRecord and ConsumerRecord abstract class
> >> >> implementing
> >> >> Serializer <K, V> and Deserializer <K, V> respectively.
> >> >>                 Public abstract class ProducerRecord <K, V>
> >>implements
> >> >> Serializer <K, V>
> >> >> {...}
> >> >>                 Public abstract class ConsumerRecord <K, V>
> >>implements
> >> >> Deserializer <K,
> >> >> V> {...}
> >> >>
> >> >>         C. Instead of instantiate the serializer/Deserializer from
> >> >>config,
> >> >> let
> >> >> concrete ProducerRecord/ConsumerRecord extends the abstract class and
> >> >> override the serialize/deserialize methods.
> >> >>
> >> >>                 Public class AvroProducerRecord extends
> >>ProducerRecord
> >> >> <String,
> >> >> GenericRecord> {
> >> >>                         ...
> >> >>                         @Override
> >> >>                         Public byte[] serializeKey(String key) {Š}
> >> >>                         @Override
> >> >>                         public byte[] serializeValue(GenericRecord
> >> >>value);
> >> >>                 }
> >> >>
> >> >>                 Public class AvroConsumerRecord extends
> >>ConsumerRecord
> >> >> <String,
> >> >> GenericRecord> {
> >> >>                         ...
> >> >>                         @Override
> >> >>                         Public K deserializeKey(byte[] key) {Š}
> >> >>                         @Override
> >> >>                         public V deserializeValue(byte[] value);
> >> >>                 }
> >> >>
> >> >>         D. The producer API changes to
> >> >>                 Public class KafkaProducer {
> >> >>                         ...
> >> >>
> >> >>                         Future<RecordMetadata> send (ProducerRecord
> >><K,
> >> >>V>
> >> >> record) {
> >> >>                                 ...
> >> >>                                 K key =
> >>record.serializeKey(record.key);
> >> >>                                 V value =
> >> >> record.serializedValue(record.value);
> >> >>                                 BytesProducerRecord
> >>bytesProducerRecord
> >> >>=
> >> >> new
> >> >> BytesProducerRecord(topic, partition, key, value);
> >> >>                                 ...
> >> >>                         }
> >> >>                         ...
> >> >>                 }
> >> >>
> >> >>
> >> >>
> >> >> We also had some brainstorm in LinkedIn and here are the feedbacks:
> >> >>
> >> >> If the community decide to add the serialization back to new
> >>producer,
> >> >> besides current proposal which changes new producer API to be a
> >> >>template,
> >> >> there are some other options raised during our discussion:
> >> >>         1) Rather than change current new producer API, we can
> >>provide a
> >> >> wrapper
> >> >> of current new producer (e.g. KafkaSerializedProducer) and make it
> >> >> available to users. As there is value in the simplicity of current
> >>API.
> >> >>
> >> >>         2) If we decide to go with tempalated new producer API,
> >> >>according
> >> >> to
> >> >> experience in LinkedIn, it might worth considering to instantiate the
> >> >> serializer in code instead of from config so we can avoid runtime
> >>errors
> >> >> due to dynamic instantiation from config, which is more error prone.
> >>If
> >> >> that is the case, the producer API could be changed to something
> >>like:
> >> >>                 producer = new Producer<K, V>(KeySerializer<K>,
> >> >> ValueSerializer<V>)
> >> >>
> >> >> --Jiangjie (Becket) Qin
> >> >>
> >> >>
> >> >> On 11/24/14, 5:58 PM, "Jun Rao" <ju...@gmail.com> 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
> >> >>
> >> >>
> >>
> >>
> 


Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Sriram Subramanian <sr...@linkedin.com.INVALID>.
Thank you Jay. I agree with the issue that you point w.r.t paired
serializers. I also think having mix serialization types is rare. To get
the current behavior, one can simply use a ByteArraySerializer. This is
best understood by talking with many customers and you seem to have done
that. I am convinced about the change.

For the rest who gave -1 or 0 for this proposal, does the answers for the
three points(updated) below seem reasonable? Are these explanations
convincing? 


1. Can we keep the serialization semantics outside the Producer interface
and have simple bytes in / bytes out for the interface (This is what we
have today).

The points for this is to keep the interface simple and usage easy to
understand. The points against this is that it gets hard to share common
usage patterns around serialization/message validations for the future.

2. Can we create a wrapper producer that does the serialization and have
different variants of it for different data formats?

The points for this is again to keep the main API clean. The points
against this is that it duplicates the API, increases the surface area and
creates redundancy for a minor addition.

3. Do we need to support different data types per record? The current
interface (bytes in/bytes out) lets you instantiate one producer and use
it to send multiple data formats. There seems to be some valid use cases
for this.


Mixed serialization types are rare based on interactions with customers.
To get the current behavior, one can simply use a ByteArraySerializer.

On 12/5/14 5:00 PM, "Jay Kreps" <ja...@confluent.io> wrote:

>Hey Sriram,
>
>Thanks! I think this is a very helpful summary.
>
>Let me try to address your point about passing in the serde at send time.
>
>I think the first objection is really to the paired key/value serializer
>interfaces. This leads to kind of a weird combinatorial thing where you
>would have an avro/avro serializer a string/avro serializer, a pb/pb
>serializer, and a string/pb serializer, and so on. But your proposal would
>work as well with separate serializers for key and value.
>
>I think the downside is just the one you call out--that this is a corner
>case and you end up with two versions of all the apis to support it. This
>also makes the serializer api more annoying to implement. I think the
>alternative solution to this case and any other we can give people is just
>configuring ByteArraySerializer which gives you basically the api that you
>have now with byte arrays. If this is incredibly common then this would be
>a silly solution, but I guess the belief is that these cases are rare and
>a
>really well implemented avro or json serializer should be 100% of what
>most
>people need.
>
>In practice the cases that actually mix serialization types in a single
>stream are pretty rare I think just because the consumer then has the
>problem of guessing how to deserialize, so most of these will end up with
>at least some marker or schema id or whatever that tells you how to read
>the data. Arguable this mixed serialization with marker is itself a
>serializer type and should have a serializer of its own...
>
>-Jay
>
>On Fri, Dec 5, 2014 at 3:48 PM, Sriram Subramanian <
>srsubramanian@linkedin.com.invalid> wrote:
>
>> This thread has diverged multiple times now and it would be worth
>> summarizing them.
>>
>> There seems to be the following points of discussion -
>>
>> 1. Can we keep the serialization semantics outside the Producer
>>interface
>> and have simple bytes in / bytes out for the interface (This is what we
>> have today).
>>
>> The points for this is to keep the interface simple and usage easy to
>> understand. The points against this is that it gets hard to share common
>> usage patterns around serialization/message validations for the future.
>>
>> 2. Can we create a wrapper producer that does the serialization and have
>> different variants of it for different data formats?
>>
>> The points for this is again to keep the main API clean. The points
>> against this is that it duplicates the API, increases the surface area
>>and
>> creates redundancy for a minor addition.
>>
>> 3. Do we need to support different data types per record? The current
>> interface (bytes in/bytes out) lets you instantiate one producer and use
>> it to send multiple data formats. There seems to be some valid use cases
>> for this.
>>
>> I have still not seen a strong argument against not having this
>> functionality. Can someone provide their views on why we don't need this
>> support that is possible with the current API?
>>
>> One possible approach for the per record serialization would be to
>>define
>>
>> public interface SerDe<K,V> {
>>   public byte[] serializeKey();
>>
>>   public K deserializeKey();
>>
>>   public byte[] serializeValue();
>>
>>   public V deserializeValue();
>> }
>>
>> This would be used by both the Producer and the Consumer.
>>
>> The send APIs can then be
>>
>> public Future<RecordMetadata> send(ProducerRecord<K,V> record);
>> public Future<RecordMetadata> send(ProducerRecord<K,V> record, Callback
>> callback);
>>
>>
>> public Future<RecordMetadata> send(ProducerRecord<K,V> record,
>>SerDe<K,V>
>> serde);
>>
>> public Future<RecordMetadata> send(ProducerRecord<K,V> record,
>>SerDe<K,V>
>> serde, Callback callback);
>>
>>
>> A default SerDe can be set in the config. The producer would use the
>> default from the config if the non-serde send APIs are used. The
>>downside
>> to this approach is that we would need to have four variants of Send API
>> for the Producer.
>>
>>
>>
>>
>>
>>
>> On 12/5/14 3:16 PM, "Jun Rao" <ju...@confluent.io> wrote:
>>
>> >Jiangjie,
>> >
>> >The issue with adding the serializer in ProducerRecord is that you
>>need to
>> >implement all combinations of serializers for key and value. So,
>>instead
>> >of
>> >just implementing int and string serializers, you will have to
>>implement
>> >all 4 combinations.
>> >
>> >Adding a new producer constructor like Producer<K, V>(KeySerializer<K>,
>> >ValueSerializer<V>, Properties properties) can be useful.
>> >
>> >Thanks,
>> >
>> >Jun
>> >
>> >On Thu, Dec 4, 2014 at 10:33 AM, Jiangjie Qin
>><jq...@linkedin.com.invalid>
>> >wrote:
>> >
>> >>
>> >> I'm just thinking instead of binding serialization with producer,
>> >>another
>> >> option is to bind serializer/deserializer with
>> >> ProducerRecord/ConsumerRecord (please see the detail proposal below.)
>> >>            The arguments for this option is:
>> >>         A. A single producer could send different message types.
>>There
>> >>are
>> >> several use cases in LinkedIn for per record serializer
>> >>         - In Samza, there are some in-stream order-sensitive control
>> >> messages
>> >> having different deserializer from other messages.
>> >>         - There are use cases which need support for sending both
>>Avro
>> >> messages
>> >> and raw bytes.
>> >>         - Some use cases needs to deserialize some Avro messages into
>> >> generic
>> >> record and some other messages into specific record.
>> >>         B. In current proposal, the serializer/deserilizer is
>> >>instantiated
>> >> according to config. Compared with that, binding serializer with
>> >> ProducerRecord and ConsumerRecord is less error prone.
>> >>
>> >>
>> >>         This option includes the following changes:
>> >>         A. Add serializer and deserializer interfaces to replace
>> >>serializer
>> >> instance from config.
>> >>                 Public interface Serializer <K, V> {
>> >>                         public byte[] serializeKey(K key);
>> >>                         public byte[] serializeValue(V value);
>> >>                 }
>> >>                 Public interface deserializer <K, V> {
>> >>                         Public K deserializeKey(byte[] key);
>> >>                         public V deserializeValue(byte[] value);
>> >>                 }
>> >>
>> >>         B. Make ProducerRecord and ConsumerRecord abstract class
>> >> implementing
>> >> Serializer <K, V> and Deserializer <K, V> respectively.
>> >>                 Public abstract class ProducerRecord <K, V>
>>implements
>> >> Serializer <K, V>
>> >> {...}
>> >>                 Public abstract class ConsumerRecord <K, V>
>>implements
>> >> Deserializer <K,
>> >> V> {...}
>> >>
>> >>         C. Instead of instantiate the serializer/Deserializer from
>> >>config,
>> >> let
>> >> concrete ProducerRecord/ConsumerRecord extends the abstract class and
>> >> override the serialize/deserialize methods.
>> >>
>> >>                 Public class AvroProducerRecord extends
>>ProducerRecord
>> >> <String,
>> >> GenericRecord> {
>> >>                         ...
>> >>                         @Override
>> >>                         Public byte[] serializeKey(String key) {Š}
>> >>                         @Override
>> >>                         public byte[] serializeValue(GenericRecord
>> >>value);
>> >>                 }
>> >>
>> >>                 Public class AvroConsumerRecord extends
>>ConsumerRecord
>> >> <String,
>> >> GenericRecord> {
>> >>                         ...
>> >>                         @Override
>> >>                         Public K deserializeKey(byte[] key) {Š}
>> >>                         @Override
>> >>                         public V deserializeValue(byte[] value);
>> >>                 }
>> >>
>> >>         D. The producer API changes to
>> >>                 Public class KafkaProducer {
>> >>                         ...
>> >>
>> >>                         Future<RecordMetadata> send (ProducerRecord
>><K,
>> >>V>
>> >> record) {
>> >>                                 ...
>> >>                                 K key =
>>record.serializeKey(record.key);
>> >>                                 V value =
>> >> record.serializedValue(record.value);
>> >>                                 BytesProducerRecord
>>bytesProducerRecord
>> >>=
>> >> new
>> >> BytesProducerRecord(topic, partition, key, value);
>> >>                                 ...
>> >>                         }
>> >>                         ...
>> >>                 }
>> >>
>> >>
>> >>
>> >> We also had some brainstorm in LinkedIn and here are the feedbacks:
>> >>
>> >> If the community decide to add the serialization back to new
>>producer,
>> >> besides current proposal which changes new producer API to be a
>> >>template,
>> >> there are some other options raised during our discussion:
>> >>         1) Rather than change current new producer API, we can
>>provide a
>> >> wrapper
>> >> of current new producer (e.g. KafkaSerializedProducer) and make it
>> >> available to users. As there is value in the simplicity of current
>>API.
>> >>
>> >>         2) If we decide to go with tempalated new producer API,
>> >>according
>> >> to
>> >> experience in LinkedIn, it might worth considering to instantiate the
>> >> serializer in code instead of from config so we can avoid runtime
>>errors
>> >> due to dynamic instantiation from config, which is more error prone.
>>If
>> >> that is the case, the producer API could be changed to something
>>like:
>> >>                 producer = new Producer<K, V>(KeySerializer<K>,
>> >> ValueSerializer<V>)
>> >>
>> >> --Jiangjie (Becket) Qin
>> >>
>> >>
>> >> On 11/24/14, 5:58 PM, "Jun Rao" <ju...@gmail.com> 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
>> >>
>> >>
>>
>>


Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Guozhang Wang <wa...@gmail.com>.
Thanks Jun.

I think we all understand the motivation of adding serialization API back,
but are just proposing different ways of doing such. I personally prefer to
not bind the producer instance with a fixed serialization, but that said I
am fine with the current proposal too as this can still be done via other
workarounds.

Guozhang

On Tue, Dec 9, 2014 at 3:46 PM, Bhavesh Mistry <mi...@gmail.com>
wrote:

> Hi All,
>
> This is very likely when you have large site such as Linked-in and you have
> thousand of servers producing data.  You will mixed bag of producer and
> serialization or deserialization because of incremental code deployment.
> So, it is best to keep the API as generic as possible and each org  /
> company can wrap the generic API with how ever they fit with serialization/
> de-serialization  framework (java or proto buffer or avro or base 64).
>
> Keep the API as generic as possible.
>
> Thanks,
>
> Bhavesh
>
> On Tue, Dec 9, 2014 at 3:29 PM, Steven Wu <st...@gmail.com> wrote:
>
> > > In practice the cases that actually mix serialization types in a single
> > stream are pretty rare I think just because the consumer then has the
> > problem of guessing how to deserialize, so most of these will end up with
> > at least some marker or schema id or whatever that tells you how to read
> > the data. Arguable this mixed serialization with marker is itself a
> > serializer type and should have a serializer of its own...
> >
> > agree that it is unlikely to have mixed serialization format for one
> > topic/type. But we sometimes/often create one Producer object for one
> > cluster. and there can be many topics on this cluster. different topics
> may
> > have different serialization formats. So I agree with Guozhang's point
> > regarding "data type flexibility" of using simple byte[] (instead of
> > generic <K, V>).
> >
> > On Fri, Dec 5, 2014 at 5:00 PM, Jay Kreps <ja...@confluent.io> wrote:
> >
> > > Hey Sriram,
> > >
> > > Thanks! I think this is a very helpful summary.
> > >
> > > Let me try to address your point about passing in the serde at send
> time.
> > >
> > > I think the first objection is really to the paired key/value
> serializer
> > > interfaces. This leads to kind of a weird combinatorial thing where you
> > > would have an avro/avro serializer a string/avro serializer, a pb/pb
> > > serializer, and a string/pb serializer, and so on. But your proposal
> > would
> > > work as well with separate serializers for key and value.
> > >
> > > I think the downside is just the one you call out--that this is a
> corner
> > > case and you end up with two versions of all the apis to support it.
> This
> > > also makes the serializer api more annoying to implement. I think the
> > > alternative solution to this case and any other we can give people is
> > just
> > > configuring ByteArraySerializer which gives you basically the api that
> > you
> > > have now with byte arrays. If this is incredibly common then this would
> > be
> > > a silly solution, but I guess the belief is that these cases are rare
> > and a
> > > really well implemented avro or json serializer should be 100% of what
> > most
> > > people need.
> > >
> > > In practice the cases that actually mix serialization types in a single
> > > stream are pretty rare I think just because the consumer then has the
> > > problem of guessing how to deserialize, so most of these will end up
> with
> > > at least some marker or schema id or whatever that tells you how to
> read
> > > the data. Arguable this mixed serialization with marker is itself a
> > > serializer type and should have a serializer of its own...
> > >
> > > -Jay
> > >
> > > On Fri, Dec 5, 2014 at 3:48 PM, Sriram Subramanian <
> > > srsubramanian@linkedin.com.invalid> wrote:
> > >
> > > > This thread has diverged multiple times now and it would be worth
> > > > summarizing them.
> > > >
> > > > There seems to be the following points of discussion -
> > > >
> > > > 1. Can we keep the serialization semantics outside the Producer
> > interface
> > > > and have simple bytes in / bytes out for the interface (This is what
> we
> > > > have today).
> > > >
> > > > The points for this is to keep the interface simple and usage easy to
> > > > understand. The points against this is that it gets hard to share
> > common
> > > > usage patterns around serialization/message validations for the
> future.
> > > >
> > > > 2. Can we create a wrapper producer that does the serialization and
> > have
> > > > different variants of it for different data formats?
> > > >
> > > > The points for this is again to keep the main API clean. The points
> > > > against this is that it duplicates the API, increases the surface
> area
> > > and
> > > > creates redundancy for a minor addition.
> > > >
> > > > 3. Do we need to support different data types per record? The current
> > > > interface (bytes in/bytes out) lets you instantiate one producer and
> > use
> > > > it to send multiple data formats. There seems to be some valid use
> > cases
> > > > for this.
> > > >
> > > > I have still not seen a strong argument against not having this
> > > > functionality. Can someone provide their views on why we don't need
> > this
> > > > support that is possible with the current API?
> > > >
> > > > One possible approach for the per record serialization would be to
> > define
> > > >
> > > > public interface SerDe<K,V> {
> > > >   public byte[] serializeKey();
> > > >
> > > >   public K deserializeKey();
> > > >
> > > >   public byte[] serializeValue();
> > > >
> > > >   public V deserializeValue();
> > > > }
> > > >
> > > > This would be used by both the Producer and the Consumer.
> > > >
> > > > The send APIs can then be
> > > >
> > > > public Future<RecordMetadata> send(ProducerRecord<K,V> record);
> > > > public Future<RecordMetadata> send(ProducerRecord<K,V> record,
> Callback
> > > > callback);
> > > >
> > > >
> > > > public Future<RecordMetadata> send(ProducerRecord<K,V> record,
> > SerDe<K,V>
> > > > serde);
> > > >
> > > > public Future<RecordMetadata> send(ProducerRecord<K,V> record,
> > SerDe<K,V>
> > > > serde, Callback callback);
> > > >
> > > >
> > > > A default SerDe can be set in the config. The producer would use the
> > > > default from the config if the non-serde send APIs are used. The
> > downside
> > > > to this approach is that we would need to have four variants of Send
> > API
> > > > for the Producer.
> > > >
> > > >
> > > >
> > > >
> > > >
> > > >
> > > > On 12/5/14 3:16 PM, "Jun Rao" <ju...@confluent.io> wrote:
> > > >
> > > > >Jiangjie,
> > > > >
> > > > >The issue with adding the serializer in ProducerRecord is that you
> > need
> > > to
> > > > >implement all combinations of serializers for key and value. So,
> > instead
> > > > >of
> > > > >just implementing int and string serializers, you will have to
> > implement
> > > > >all 4 combinations.
> > > > >
> > > > >Adding a new producer constructor like Producer<K,
> > V>(KeySerializer<K>,
> > > > >ValueSerializer<V>, Properties properties) can be useful.
> > > > >
> > > > >Thanks,
> > > > >
> > > > >Jun
> > > > >
> > > > >On Thu, Dec 4, 2014 at 10:33 AM, Jiangjie Qin
> > <jqin@linkedin.com.invalid
> > > >
> > > > >wrote:
> > > > >
> > > > >>
> > > > >> I'm just thinking instead of binding serialization with producer,
> > > > >>another
> > > > >> option is to bind serializer/deserializer with
> > > > >> ProducerRecord/ConsumerRecord (please see the detail proposal
> > below.)
> > > > >>            The arguments for this option is:
> > > > >>         A. A single producer could send different message types.
> > There
> > > > >>are
> > > > >> several use cases in LinkedIn for per record serializer
> > > > >>         - In Samza, there are some in-stream order-sensitive
> control
> > > > >> messages
> > > > >> having different deserializer from other messages.
> > > > >>         - There are use cases which need support for sending both
> > Avro
> > > > >> messages
> > > > >> and raw bytes.
> > > > >>         - Some use cases needs to deserialize some Avro messages
> > into
> > > > >> generic
> > > > >> record and some other messages into specific record.
> > > > >>         B. In current proposal, the serializer/deserilizer is
> > > > >>instantiated
> > > > >> according to config. Compared with that, binding serializer with
> > > > >> ProducerRecord and ConsumerRecord is less error prone.
> > > > >>
> > > > >>
> > > > >>         This option includes the following changes:
> > > > >>         A. Add serializer and deserializer interfaces to replace
> > > > >>serializer
> > > > >> instance from config.
> > > > >>                 Public interface Serializer <K, V> {
> > > > >>                         public byte[] serializeKey(K key);
> > > > >>                         public byte[] serializeValue(V value);
> > > > >>                 }
> > > > >>                 Public interface deserializer <K, V> {
> > > > >>                         Public K deserializeKey(byte[] key);
> > > > >>                         public V deserializeValue(byte[] value);
> > > > >>                 }
> > > > >>
> > > > >>         B. Make ProducerRecord and ConsumerRecord abstract class
> > > > >> implementing
> > > > >> Serializer <K, V> and Deserializer <K, V> respectively.
> > > > >>                 Public abstract class ProducerRecord <K, V>
> > implements
> > > > >> Serializer <K, V>
> > > > >> {...}
> > > > >>                 Public abstract class ConsumerRecord <K, V>
> > implements
> > > > >> Deserializer <K,
> > > > >> V> {...}
> > > > >>
> > > > >>         C. Instead of instantiate the serializer/Deserializer from
> > > > >>config,
> > > > >> let
> > > > >> concrete ProducerRecord/ConsumerRecord extends the abstract class
> > and
> > > > >> override the serialize/deserialize methods.
> > > > >>
> > > > >>                 Public class AvroProducerRecord extends
> > ProducerRecord
> > > > >> <String,
> > > > >> GenericRecord> {
> > > > >>                         ...
> > > > >>                         @Override
> > > > >>                         Public byte[] serializeKey(String key) {Š}
> > > > >>                         @Override
> > > > >>                         public byte[] serializeValue(GenericRecord
> > > > >>value);
> > > > >>                 }
> > > > >>
> > > > >>                 Public class AvroConsumerRecord extends
> > ConsumerRecord
> > > > >> <String,
> > > > >> GenericRecord> {
> > > > >>                         ...
> > > > >>                         @Override
> > > > >>                         Public K deserializeKey(byte[] key) {Š}
> > > > >>                         @Override
> > > > >>                         public V deserializeValue(byte[] value);
> > > > >>                 }
> > > > >>
> > > > >>         D. The producer API changes to
> > > > >>                 Public class KafkaProducer {
> > > > >>                         ...
> > > > >>
> > > > >>                         Future<RecordMetadata> send
> (ProducerRecord
> > > <K,
> > > > >>V>
> > > > >> record) {
> > > > >>                                 ...
> > > > >>                                 K key =
> > > record.serializeKey(record.key);
> > > > >>                                 V value =
> > > > >> record.serializedValue(record.value);
> > > > >>                                 BytesProducerRecord
> > > bytesProducerRecord
> > > > >>=
> > > > >> new
> > > > >> BytesProducerRecord(topic, partition, key, value);
> > > > >>                                 ...
> > > > >>                         }
> > > > >>                         ...
> > > > >>                 }
> > > > >>
> > > > >>
> > > > >>
> > > > >> We also had some brainstorm in LinkedIn and here are the
> feedbacks:
> > > > >>
> > > > >> If the community decide to add the serialization back to new
> > producer,
> > > > >> besides current proposal which changes new producer API to be a
> > > > >>template,
> > > > >> there are some other options raised during our discussion:
> > > > >>         1) Rather than change current new producer API, we can
> > > provide a
> > > > >> wrapper
> > > > >> of current new producer (e.g. KafkaSerializedProducer) and make it
> > > > >> available to users. As there is value in the simplicity of current
> > > API.
> > > > >>
> > > > >>         2) If we decide to go with tempalated new producer API,
> > > > >>according
> > > > >> to
> > > > >> experience in LinkedIn, it might worth considering to instantiate
> > the
> > > > >> serializer in code instead of from config so we can avoid runtime
> > > errors
> > > > >> due to dynamic instantiation from config, which is more error
> prone.
> > > If
> > > > >> that is the case, the producer API could be changed to something
> > like:
> > > > >>                 producer = new Producer<K, V>(KeySerializer<K>,
> > > > >> ValueSerializer<V>)
> > > > >>
> > > > >> --Jiangjie (Becket) Qin
> > > > >>
> > > > >>
> > > > >> On 11/24/14, 5:58 PM, "Jun Rao" <ju...@gmail.com> 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
> > > > >>
> > > > >>
> > > >
> > > >
> > >
> >
>



-- 
-- Guozhang

Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Bhavesh Mistry <mi...@gmail.com>.
Hi All,

This is very likely when you have large site such as Linked-in and you have
thousand of servers producing data.  You will mixed bag of producer and
serialization or deserialization because of incremental code deployment.
So, it is best to keep the API as generic as possible and each org  /
company can wrap the generic API with how ever they fit with serialization/
de-serialization  framework (java or proto buffer or avro or base 64).

Keep the API as generic as possible.

Thanks,

Bhavesh

On Tue, Dec 9, 2014 at 3:29 PM, Steven Wu <st...@gmail.com> wrote:

> > In practice the cases that actually mix serialization types in a single
> stream are pretty rare I think just because the consumer then has the
> problem of guessing how to deserialize, so most of these will end up with
> at least some marker or schema id or whatever that tells you how to read
> the data. Arguable this mixed serialization with marker is itself a
> serializer type and should have a serializer of its own...
>
> agree that it is unlikely to have mixed serialization format for one
> topic/type. But we sometimes/often create one Producer object for one
> cluster. and there can be many topics on this cluster. different topics may
> have different serialization formats. So I agree with Guozhang's point
> regarding "data type flexibility" of using simple byte[] (instead of
> generic <K, V>).
>
> On Fri, Dec 5, 2014 at 5:00 PM, Jay Kreps <ja...@confluent.io> wrote:
>
> > Hey Sriram,
> >
> > Thanks! I think this is a very helpful summary.
> >
> > Let me try to address your point about passing in the serde at send time.
> >
> > I think the first objection is really to the paired key/value serializer
> > interfaces. This leads to kind of a weird combinatorial thing where you
> > would have an avro/avro serializer a string/avro serializer, a pb/pb
> > serializer, and a string/pb serializer, and so on. But your proposal
> would
> > work as well with separate serializers for key and value.
> >
> > I think the downside is just the one you call out--that this is a corner
> > case and you end up with two versions of all the apis to support it. This
> > also makes the serializer api more annoying to implement. I think the
> > alternative solution to this case and any other we can give people is
> just
> > configuring ByteArraySerializer which gives you basically the api that
> you
> > have now with byte arrays. If this is incredibly common then this would
> be
> > a silly solution, but I guess the belief is that these cases are rare
> and a
> > really well implemented avro or json serializer should be 100% of what
> most
> > people need.
> >
> > In practice the cases that actually mix serialization types in a single
> > stream are pretty rare I think just because the consumer then has the
> > problem of guessing how to deserialize, so most of these will end up with
> > at least some marker or schema id or whatever that tells you how to read
> > the data. Arguable this mixed serialization with marker is itself a
> > serializer type and should have a serializer of its own...
> >
> > -Jay
> >
> > On Fri, Dec 5, 2014 at 3:48 PM, Sriram Subramanian <
> > srsubramanian@linkedin.com.invalid> wrote:
> >
> > > This thread has diverged multiple times now and it would be worth
> > > summarizing them.
> > >
> > > There seems to be the following points of discussion -
> > >
> > > 1. Can we keep the serialization semantics outside the Producer
> interface
> > > and have simple bytes in / bytes out for the interface (This is what we
> > > have today).
> > >
> > > The points for this is to keep the interface simple and usage easy to
> > > understand. The points against this is that it gets hard to share
> common
> > > usage patterns around serialization/message validations for the future.
> > >
> > > 2. Can we create a wrapper producer that does the serialization and
> have
> > > different variants of it for different data formats?
> > >
> > > The points for this is again to keep the main API clean. The points
> > > against this is that it duplicates the API, increases the surface area
> > and
> > > creates redundancy for a minor addition.
> > >
> > > 3. Do we need to support different data types per record? The current
> > > interface (bytes in/bytes out) lets you instantiate one producer and
> use
> > > it to send multiple data formats. There seems to be some valid use
> cases
> > > for this.
> > >
> > > I have still not seen a strong argument against not having this
> > > functionality. Can someone provide their views on why we don't need
> this
> > > support that is possible with the current API?
> > >
> > > One possible approach for the per record serialization would be to
> define
> > >
> > > public interface SerDe<K,V> {
> > >   public byte[] serializeKey();
> > >
> > >   public K deserializeKey();
> > >
> > >   public byte[] serializeValue();
> > >
> > >   public V deserializeValue();
> > > }
> > >
> > > This would be used by both the Producer and the Consumer.
> > >
> > > The send APIs can then be
> > >
> > > public Future<RecordMetadata> send(ProducerRecord<K,V> record);
> > > public Future<RecordMetadata> send(ProducerRecord<K,V> record, Callback
> > > callback);
> > >
> > >
> > > public Future<RecordMetadata> send(ProducerRecord<K,V> record,
> SerDe<K,V>
> > > serde);
> > >
> > > public Future<RecordMetadata> send(ProducerRecord<K,V> record,
> SerDe<K,V>
> > > serde, Callback callback);
> > >
> > >
> > > A default SerDe can be set in the config. The producer would use the
> > > default from the config if the non-serde send APIs are used. The
> downside
> > > to this approach is that we would need to have four variants of Send
> API
> > > for the Producer.
> > >
> > >
> > >
> > >
> > >
> > >
> > > On 12/5/14 3:16 PM, "Jun Rao" <ju...@confluent.io> wrote:
> > >
> > > >Jiangjie,
> > > >
> > > >The issue with adding the serializer in ProducerRecord is that you
> need
> > to
> > > >implement all combinations of serializers for key and value. So,
> instead
> > > >of
> > > >just implementing int and string serializers, you will have to
> implement
> > > >all 4 combinations.
> > > >
> > > >Adding a new producer constructor like Producer<K,
> V>(KeySerializer<K>,
> > > >ValueSerializer<V>, Properties properties) can be useful.
> > > >
> > > >Thanks,
> > > >
> > > >Jun
> > > >
> > > >On Thu, Dec 4, 2014 at 10:33 AM, Jiangjie Qin
> <jqin@linkedin.com.invalid
> > >
> > > >wrote:
> > > >
> > > >>
> > > >> I'm just thinking instead of binding serialization with producer,
> > > >>another
> > > >> option is to bind serializer/deserializer with
> > > >> ProducerRecord/ConsumerRecord (please see the detail proposal
> below.)
> > > >>            The arguments for this option is:
> > > >>         A. A single producer could send different message types.
> There
> > > >>are
> > > >> several use cases in LinkedIn for per record serializer
> > > >>         - In Samza, there are some in-stream order-sensitive control
> > > >> messages
> > > >> having different deserializer from other messages.
> > > >>         - There are use cases which need support for sending both
> Avro
> > > >> messages
> > > >> and raw bytes.
> > > >>         - Some use cases needs to deserialize some Avro messages
> into
> > > >> generic
> > > >> record and some other messages into specific record.
> > > >>         B. In current proposal, the serializer/deserilizer is
> > > >>instantiated
> > > >> according to config. Compared with that, binding serializer with
> > > >> ProducerRecord and ConsumerRecord is less error prone.
> > > >>
> > > >>
> > > >>         This option includes the following changes:
> > > >>         A. Add serializer and deserializer interfaces to replace
> > > >>serializer
> > > >> instance from config.
> > > >>                 Public interface Serializer <K, V> {
> > > >>                         public byte[] serializeKey(K key);
> > > >>                         public byte[] serializeValue(V value);
> > > >>                 }
> > > >>                 Public interface deserializer <K, V> {
> > > >>                         Public K deserializeKey(byte[] key);
> > > >>                         public V deserializeValue(byte[] value);
> > > >>                 }
> > > >>
> > > >>         B. Make ProducerRecord and ConsumerRecord abstract class
> > > >> implementing
> > > >> Serializer <K, V> and Deserializer <K, V> respectively.
> > > >>                 Public abstract class ProducerRecord <K, V>
> implements
> > > >> Serializer <K, V>
> > > >> {...}
> > > >>                 Public abstract class ConsumerRecord <K, V>
> implements
> > > >> Deserializer <K,
> > > >> V> {...}
> > > >>
> > > >>         C. Instead of instantiate the serializer/Deserializer from
> > > >>config,
> > > >> let
> > > >> concrete ProducerRecord/ConsumerRecord extends the abstract class
> and
> > > >> override the serialize/deserialize methods.
> > > >>
> > > >>                 Public class AvroProducerRecord extends
> ProducerRecord
> > > >> <String,
> > > >> GenericRecord> {
> > > >>                         ...
> > > >>                         @Override
> > > >>                         Public byte[] serializeKey(String key) {Š}
> > > >>                         @Override
> > > >>                         public byte[] serializeValue(GenericRecord
> > > >>value);
> > > >>                 }
> > > >>
> > > >>                 Public class AvroConsumerRecord extends
> ConsumerRecord
> > > >> <String,
> > > >> GenericRecord> {
> > > >>                         ...
> > > >>                         @Override
> > > >>                         Public K deserializeKey(byte[] key) {Š}
> > > >>                         @Override
> > > >>                         public V deserializeValue(byte[] value);
> > > >>                 }
> > > >>
> > > >>         D. The producer API changes to
> > > >>                 Public class KafkaProducer {
> > > >>                         ...
> > > >>
> > > >>                         Future<RecordMetadata> send (ProducerRecord
> > <K,
> > > >>V>
> > > >> record) {
> > > >>                                 ...
> > > >>                                 K key =
> > record.serializeKey(record.key);
> > > >>                                 V value =
> > > >> record.serializedValue(record.value);
> > > >>                                 BytesProducerRecord
> > bytesProducerRecord
> > > >>=
> > > >> new
> > > >> BytesProducerRecord(topic, partition, key, value);
> > > >>                                 ...
> > > >>                         }
> > > >>                         ...
> > > >>                 }
> > > >>
> > > >>
> > > >>
> > > >> We also had some brainstorm in LinkedIn and here are the feedbacks:
> > > >>
> > > >> If the community decide to add the serialization back to new
> producer,
> > > >> besides current proposal which changes new producer API to be a
> > > >>template,
> > > >> there are some other options raised during our discussion:
> > > >>         1) Rather than change current new producer API, we can
> > provide a
> > > >> wrapper
> > > >> of current new producer (e.g. KafkaSerializedProducer) and make it
> > > >> available to users. As there is value in the simplicity of current
> > API.
> > > >>
> > > >>         2) If we decide to go with tempalated new producer API,
> > > >>according
> > > >> to
> > > >> experience in LinkedIn, it might worth considering to instantiate
> the
> > > >> serializer in code instead of from config so we can avoid runtime
> > errors
> > > >> due to dynamic instantiation from config, which is more error prone.
> > If
> > > >> that is the case, the producer API could be changed to something
> like:
> > > >>                 producer = new Producer<K, V>(KeySerializer<K>,
> > > >> ValueSerializer<V>)
> > > >>
> > > >> --Jiangjie (Becket) Qin
> > > >>
> > > >>
> > > >> On 11/24/14, 5:58 PM, "Jun Rao" <ju...@gmail.com> 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
> > > >>
> > > >>
> > >
> > >
> >
>

Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Steven Wu <st...@gmail.com>.
> In practice the cases that actually mix serialization types in a single
stream are pretty rare I think just because the consumer then has the
problem of guessing how to deserialize, so most of these will end up with
at least some marker or schema id or whatever that tells you how to read
the data. Arguable this mixed serialization with marker is itself a
serializer type and should have a serializer of its own...

agree that it is unlikely to have mixed serialization format for one
topic/type. But we sometimes/often create one Producer object for one
cluster. and there can be many topics on this cluster. different topics may
have different serialization formats. So I agree with Guozhang's point
regarding "data type flexibility" of using simple byte[] (instead of
generic <K, V>).

On Fri, Dec 5, 2014 at 5:00 PM, Jay Kreps <ja...@confluent.io> wrote:

> Hey Sriram,
>
> Thanks! I think this is a very helpful summary.
>
> Let me try to address your point about passing in the serde at send time.
>
> I think the first objection is really to the paired key/value serializer
> interfaces. This leads to kind of a weird combinatorial thing where you
> would have an avro/avro serializer a string/avro serializer, a pb/pb
> serializer, and a string/pb serializer, and so on. But your proposal would
> work as well with separate serializers for key and value.
>
> I think the downside is just the one you call out--that this is a corner
> case and you end up with two versions of all the apis to support it. This
> also makes the serializer api more annoying to implement. I think the
> alternative solution to this case and any other we can give people is just
> configuring ByteArraySerializer which gives you basically the api that you
> have now with byte arrays. If this is incredibly common then this would be
> a silly solution, but I guess the belief is that these cases are rare and a
> really well implemented avro or json serializer should be 100% of what most
> people need.
>
> In practice the cases that actually mix serialization types in a single
> stream are pretty rare I think just because the consumer then has the
> problem of guessing how to deserialize, so most of these will end up with
> at least some marker or schema id or whatever that tells you how to read
> the data. Arguable this mixed serialization with marker is itself a
> serializer type and should have a serializer of its own...
>
> -Jay
>
> On Fri, Dec 5, 2014 at 3:48 PM, Sriram Subramanian <
> srsubramanian@linkedin.com.invalid> wrote:
>
> > This thread has diverged multiple times now and it would be worth
> > summarizing them.
> >
> > There seems to be the following points of discussion -
> >
> > 1. Can we keep the serialization semantics outside the Producer interface
> > and have simple bytes in / bytes out for the interface (This is what we
> > have today).
> >
> > The points for this is to keep the interface simple and usage easy to
> > understand. The points against this is that it gets hard to share common
> > usage patterns around serialization/message validations for the future.
> >
> > 2. Can we create a wrapper producer that does the serialization and have
> > different variants of it for different data formats?
> >
> > The points for this is again to keep the main API clean. The points
> > against this is that it duplicates the API, increases the surface area
> and
> > creates redundancy for a minor addition.
> >
> > 3. Do we need to support different data types per record? The current
> > interface (bytes in/bytes out) lets you instantiate one producer and use
> > it to send multiple data formats. There seems to be some valid use cases
> > for this.
> >
> > I have still not seen a strong argument against not having this
> > functionality. Can someone provide their views on why we don't need this
> > support that is possible with the current API?
> >
> > One possible approach for the per record serialization would be to define
> >
> > public interface SerDe<K,V> {
> >   public byte[] serializeKey();
> >
> >   public K deserializeKey();
> >
> >   public byte[] serializeValue();
> >
> >   public V deserializeValue();
> > }
> >
> > This would be used by both the Producer and the Consumer.
> >
> > The send APIs can then be
> >
> > public Future<RecordMetadata> send(ProducerRecord<K,V> record);
> > public Future<RecordMetadata> send(ProducerRecord<K,V> record, Callback
> > callback);
> >
> >
> > public Future<RecordMetadata> send(ProducerRecord<K,V> record, SerDe<K,V>
> > serde);
> >
> > public Future<RecordMetadata> send(ProducerRecord<K,V> record, SerDe<K,V>
> > serde, Callback callback);
> >
> >
> > A default SerDe can be set in the config. The producer would use the
> > default from the config if the non-serde send APIs are used. The downside
> > to this approach is that we would need to have four variants of Send API
> > for the Producer.
> >
> >
> >
> >
> >
> >
> > On 12/5/14 3:16 PM, "Jun Rao" <ju...@confluent.io> wrote:
> >
> > >Jiangjie,
> > >
> > >The issue with adding the serializer in ProducerRecord is that you need
> to
> > >implement all combinations of serializers for key and value. So, instead
> > >of
> > >just implementing int and string serializers, you will have to implement
> > >all 4 combinations.
> > >
> > >Adding a new producer constructor like Producer<K, V>(KeySerializer<K>,
> > >ValueSerializer<V>, Properties properties) can be useful.
> > >
> > >Thanks,
> > >
> > >Jun
> > >
> > >On Thu, Dec 4, 2014 at 10:33 AM, Jiangjie Qin <jqin@linkedin.com.invalid
> >
> > >wrote:
> > >
> > >>
> > >> I'm just thinking instead of binding serialization with producer,
> > >>another
> > >> option is to bind serializer/deserializer with
> > >> ProducerRecord/ConsumerRecord (please see the detail proposal below.)
> > >>            The arguments for this option is:
> > >>         A. A single producer could send different message types. There
> > >>are
> > >> several use cases in LinkedIn for per record serializer
> > >>         - In Samza, there are some in-stream order-sensitive control
> > >> messages
> > >> having different deserializer from other messages.
> > >>         - There are use cases which need support for sending both Avro
> > >> messages
> > >> and raw bytes.
> > >>         - Some use cases needs to deserialize some Avro messages into
> > >> generic
> > >> record and some other messages into specific record.
> > >>         B. In current proposal, the serializer/deserilizer is
> > >>instantiated
> > >> according to config. Compared with that, binding serializer with
> > >> ProducerRecord and ConsumerRecord is less error prone.
> > >>
> > >>
> > >>         This option includes the following changes:
> > >>         A. Add serializer and deserializer interfaces to replace
> > >>serializer
> > >> instance from config.
> > >>                 Public interface Serializer <K, V> {
> > >>                         public byte[] serializeKey(K key);
> > >>                         public byte[] serializeValue(V value);
> > >>                 }
> > >>                 Public interface deserializer <K, V> {
> > >>                         Public K deserializeKey(byte[] key);
> > >>                         public V deserializeValue(byte[] value);
> > >>                 }
> > >>
> > >>         B. Make ProducerRecord and ConsumerRecord abstract class
> > >> implementing
> > >> Serializer <K, V> and Deserializer <K, V> respectively.
> > >>                 Public abstract class ProducerRecord <K, V> implements
> > >> Serializer <K, V>
> > >> {...}
> > >>                 Public abstract class ConsumerRecord <K, V> implements
> > >> Deserializer <K,
> > >> V> {...}
> > >>
> > >>         C. Instead of instantiate the serializer/Deserializer from
> > >>config,
> > >> let
> > >> concrete ProducerRecord/ConsumerRecord extends the abstract class and
> > >> override the serialize/deserialize methods.
> > >>
> > >>                 Public class AvroProducerRecord extends ProducerRecord
> > >> <String,
> > >> GenericRecord> {
> > >>                         ...
> > >>                         @Override
> > >>                         Public byte[] serializeKey(String key) {Š}
> > >>                         @Override
> > >>                         public byte[] serializeValue(GenericRecord
> > >>value);
> > >>                 }
> > >>
> > >>                 Public class AvroConsumerRecord extends ConsumerRecord
> > >> <String,
> > >> GenericRecord> {
> > >>                         ...
> > >>                         @Override
> > >>                         Public K deserializeKey(byte[] key) {Š}
> > >>                         @Override
> > >>                         public V deserializeValue(byte[] value);
> > >>                 }
> > >>
> > >>         D. The producer API changes to
> > >>                 Public class KafkaProducer {
> > >>                         ...
> > >>
> > >>                         Future<RecordMetadata> send (ProducerRecord
> <K,
> > >>V>
> > >> record) {
> > >>                                 ...
> > >>                                 K key =
> record.serializeKey(record.key);
> > >>                                 V value =
> > >> record.serializedValue(record.value);
> > >>                                 BytesProducerRecord
> bytesProducerRecord
> > >>=
> > >> new
> > >> BytesProducerRecord(topic, partition, key, value);
> > >>                                 ...
> > >>                         }
> > >>                         ...
> > >>                 }
> > >>
> > >>
> > >>
> > >> We also had some brainstorm in LinkedIn and here are the feedbacks:
> > >>
> > >> If the community decide to add the serialization back to new producer,
> > >> besides current proposal which changes new producer API to be a
> > >>template,
> > >> there are some other options raised during our discussion:
> > >>         1) Rather than change current new producer API, we can
> provide a
> > >> wrapper
> > >> of current new producer (e.g. KafkaSerializedProducer) and make it
> > >> available to users. As there is value in the simplicity of current
> API.
> > >>
> > >>         2) If we decide to go with tempalated new producer API,
> > >>according
> > >> to
> > >> experience in LinkedIn, it might worth considering to instantiate the
> > >> serializer in code instead of from config so we can avoid runtime
> errors
> > >> due to dynamic instantiation from config, which is more error prone.
> If
> > >> that is the case, the producer API could be changed to something like:
> > >>                 producer = new Producer<K, V>(KeySerializer<K>,
> > >> ValueSerializer<V>)
> > >>
> > >> --Jiangjie (Becket) Qin
> > >>
> > >>
> > >> On 11/24/14, 5:58 PM, "Jun Rao" <ju...@gmail.com> 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
> > >>
> > >>
> >
> >
>

Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Sriram Subramanian <sr...@linkedin.com.INVALID>.
Thank you Jay. I agree with the issue that you point w.r.t paired
serializers. I also think having mix serialization types is rare. To get
the current behavior, one can simply use a ByteArraySerializer. This is
best understood by talking with many customers and you seem to have done
that. I am convinced about the change.

For the rest who gave -1 or 0 for this proposal, does the answers for the
three points(updated) below seem reasonable? Are these explanations
convincing? 


1. Can we keep the serialization semantics outside the Producer interface
and have simple bytes in / bytes out for the interface (This is what we
have today).

The points for this is to keep the interface simple and usage easy to
understand. The points against this is that it gets hard to share common
usage patterns around serialization/message validations for the future.

2. Can we create a wrapper producer that does the serialization and have
different variants of it for different data formats?

The points for this is again to keep the main API clean. The points
against this is that it duplicates the API, increases the surface area and
creates redundancy for a minor addition.

3. Do we need to support different data types per record? The current
interface (bytes in/bytes out) lets you instantiate one producer and use
it to send multiple data formats. There seems to be some valid use cases
for this.


Mixed serialization types are rare based on interactions with customers.
To get the current behavior, one can simply use a ByteArraySerializer.

On 12/5/14 5:00 PM, "Jay Kreps" <ja...@confluent.io> wrote:

>Hey Sriram,
>
>Thanks! I think this is a very helpful summary.
>
>Let me try to address your point about passing in the serde at send time.
>
>I think the first objection is really to the paired key/value serializer
>interfaces. This leads to kind of a weird combinatorial thing where you
>would have an avro/avro serializer a string/avro serializer, a pb/pb
>serializer, and a string/pb serializer, and so on. But your proposal would
>work as well with separate serializers for key and value.
>
>I think the downside is just the one you call out--that this is a corner
>case and you end up with two versions of all the apis to support it. This
>also makes the serializer api more annoying to implement. I think the
>alternative solution to this case and any other we can give people is just
>configuring ByteArraySerializer which gives you basically the api that you
>have now with byte arrays. If this is incredibly common then this would be
>a silly solution, but I guess the belief is that these cases are rare and
>a
>really well implemented avro or json serializer should be 100% of what
>most
>people need.
>
>In practice the cases that actually mix serialization types in a single
>stream are pretty rare I think just because the consumer then has the
>problem of guessing how to deserialize, so most of these will end up with
>at least some marker or schema id or whatever that tells you how to read
>the data. Arguable this mixed serialization with marker is itself a
>serializer type and should have a serializer of its own...
>
>-Jay
>
>On Fri, Dec 5, 2014 at 3:48 PM, Sriram Subramanian <
>srsubramanian@linkedin.com.invalid> wrote:
>
>> This thread has diverged multiple times now and it would be worth
>> summarizing them.
>>
>> There seems to be the following points of discussion -
>>
>> 1. Can we keep the serialization semantics outside the Producer
>>interface
>> and have simple bytes in / bytes out for the interface (This is what we
>> have today).
>>
>> The points for this is to keep the interface simple and usage easy to
>> understand. The points against this is that it gets hard to share common
>> usage patterns around serialization/message validations for the future.
>>
>> 2. Can we create a wrapper producer that does the serialization and have
>> different variants of it for different data formats?
>>
>> The points for this is again to keep the main API clean. The points
>> against this is that it duplicates the API, increases the surface area
>>and
>> creates redundancy for a minor addition.
>>
>> 3. Do we need to support different data types per record? The current
>> interface (bytes in/bytes out) lets you instantiate one producer and use
>> it to send multiple data formats. There seems to be some valid use cases
>> for this.
>>
>> I have still not seen a strong argument against not having this
>> functionality. Can someone provide their views on why we don't need this
>> support that is possible with the current API?
>>
>> One possible approach for the per record serialization would be to
>>define
>>
>> public interface SerDe<K,V> {
>>   public byte[] serializeKey();
>>
>>   public K deserializeKey();
>>
>>   public byte[] serializeValue();
>>
>>   public V deserializeValue();
>> }
>>
>> This would be used by both the Producer and the Consumer.
>>
>> The send APIs can then be
>>
>> public Future<RecordMetadata> send(ProducerRecord<K,V> record);
>> public Future<RecordMetadata> send(ProducerRecord<K,V> record, Callback
>> callback);
>>
>>
>> public Future<RecordMetadata> send(ProducerRecord<K,V> record,
>>SerDe<K,V>
>> serde);
>>
>> public Future<RecordMetadata> send(ProducerRecord<K,V> record,
>>SerDe<K,V>
>> serde, Callback callback);
>>
>>
>> A default SerDe can be set in the config. The producer would use the
>> default from the config if the non-serde send APIs are used. The
>>downside
>> to this approach is that we would need to have four variants of Send API
>> for the Producer.
>>
>>
>>
>>
>>
>>
>> On 12/5/14 3:16 PM, "Jun Rao" <ju...@confluent.io> wrote:
>>
>> >Jiangjie,
>> >
>> >The issue with adding the serializer in ProducerRecord is that you
>>need to
>> >implement all combinations of serializers for key and value. So,
>>instead
>> >of
>> >just implementing int and string serializers, you will have to
>>implement
>> >all 4 combinations.
>> >
>> >Adding a new producer constructor like Producer<K, V>(KeySerializer<K>,
>> >ValueSerializer<V>, Properties properties) can be useful.
>> >
>> >Thanks,
>> >
>> >Jun
>> >
>> >On Thu, Dec 4, 2014 at 10:33 AM, Jiangjie Qin
>><jq...@linkedin.com.invalid>
>> >wrote:
>> >
>> >>
>> >> I'm just thinking instead of binding serialization with producer,
>> >>another
>> >> option is to bind serializer/deserializer with
>> >> ProducerRecord/ConsumerRecord (please see the detail proposal below.)
>> >>            The arguments for this option is:
>> >>         A. A single producer could send different message types.
>>There
>> >>are
>> >> several use cases in LinkedIn for per record serializer
>> >>         - In Samza, there are some in-stream order-sensitive control
>> >> messages
>> >> having different deserializer from other messages.
>> >>         - There are use cases which need support for sending both
>>Avro
>> >> messages
>> >> and raw bytes.
>> >>         - Some use cases needs to deserialize some Avro messages into
>> >> generic
>> >> record and some other messages into specific record.
>> >>         B. In current proposal, the serializer/deserilizer is
>> >>instantiated
>> >> according to config. Compared with that, binding serializer with
>> >> ProducerRecord and ConsumerRecord is less error prone.
>> >>
>> >>
>> >>         This option includes the following changes:
>> >>         A. Add serializer and deserializer interfaces to replace
>> >>serializer
>> >> instance from config.
>> >>                 Public interface Serializer <K, V> {
>> >>                         public byte[] serializeKey(K key);
>> >>                         public byte[] serializeValue(V value);
>> >>                 }
>> >>                 Public interface deserializer <K, V> {
>> >>                         Public K deserializeKey(byte[] key);
>> >>                         public V deserializeValue(byte[] value);
>> >>                 }
>> >>
>> >>         B. Make ProducerRecord and ConsumerRecord abstract class
>> >> implementing
>> >> Serializer <K, V> and Deserializer <K, V> respectively.
>> >>                 Public abstract class ProducerRecord <K, V>
>>implements
>> >> Serializer <K, V>
>> >> {...}
>> >>                 Public abstract class ConsumerRecord <K, V>
>>implements
>> >> Deserializer <K,
>> >> V> {...}
>> >>
>> >>         C. Instead of instantiate the serializer/Deserializer from
>> >>config,
>> >> let
>> >> concrete ProducerRecord/ConsumerRecord extends the abstract class and
>> >> override the serialize/deserialize methods.
>> >>
>> >>                 Public class AvroProducerRecord extends
>>ProducerRecord
>> >> <String,
>> >> GenericRecord> {
>> >>                         ...
>> >>                         @Override
>> >>                         Public byte[] serializeKey(String key) {Š}
>> >>                         @Override
>> >>                         public byte[] serializeValue(GenericRecord
>> >>value);
>> >>                 }
>> >>
>> >>                 Public class AvroConsumerRecord extends
>>ConsumerRecord
>> >> <String,
>> >> GenericRecord> {
>> >>                         ...
>> >>                         @Override
>> >>                         Public K deserializeKey(byte[] key) {Š}
>> >>                         @Override
>> >>                         public V deserializeValue(byte[] value);
>> >>                 }
>> >>
>> >>         D. The producer API changes to
>> >>                 Public class KafkaProducer {
>> >>                         ...
>> >>
>> >>                         Future<RecordMetadata> send (ProducerRecord
>><K,
>> >>V>
>> >> record) {
>> >>                                 ...
>> >>                                 K key =
>>record.serializeKey(record.key);
>> >>                                 V value =
>> >> record.serializedValue(record.value);
>> >>                                 BytesProducerRecord
>>bytesProducerRecord
>> >>=
>> >> new
>> >> BytesProducerRecord(topic, partition, key, value);
>> >>                                 ...
>> >>                         }
>> >>                         ...
>> >>                 }
>> >>
>> >>
>> >>
>> >> We also had some brainstorm in LinkedIn and here are the feedbacks:
>> >>
>> >> If the community decide to add the serialization back to new
>>producer,
>> >> besides current proposal which changes new producer API to be a
>> >>template,
>> >> there are some other options raised during our discussion:
>> >>         1) Rather than change current new producer API, we can
>>provide a
>> >> wrapper
>> >> of current new producer (e.g. KafkaSerializedProducer) and make it
>> >> available to users. As there is value in the simplicity of current
>>API.
>> >>
>> >>         2) If we decide to go with tempalated new producer API,
>> >>according
>> >> to
>> >> experience in LinkedIn, it might worth considering to instantiate the
>> >> serializer in code instead of from config so we can avoid runtime
>>errors
>> >> due to dynamic instantiation from config, which is more error prone.
>>If
>> >> that is the case, the producer API could be changed to something
>>like:
>> >>                 producer = new Producer<K, V>(KeySerializer<K>,
>> >> ValueSerializer<V>)
>> >>
>> >> --Jiangjie (Becket) Qin
>> >>
>> >>
>> >> On 11/24/14, 5:58 PM, "Jun Rao" <ju...@gmail.com> 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
>> >>
>> >>
>>
>>


Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Jay Kreps <ja...@confluent.io>.
Hey Sriram,

Thanks! I think this is a very helpful summary.

Let me try to address your point about passing in the serde at send time.

I think the first objection is really to the paired key/value serializer
interfaces. This leads to kind of a weird combinatorial thing where you
would have an avro/avro serializer a string/avro serializer, a pb/pb
serializer, and a string/pb serializer, and so on. But your proposal would
work as well with separate serializers for key and value.

I think the downside is just the one you call out--that this is a corner
case and you end up with two versions of all the apis to support it. This
also makes the serializer api more annoying to implement. I think the
alternative solution to this case and any other we can give people is just
configuring ByteArraySerializer which gives you basically the api that you
have now with byte arrays. If this is incredibly common then this would be
a silly solution, but I guess the belief is that these cases are rare and a
really well implemented avro or json serializer should be 100% of what most
people need.

In practice the cases that actually mix serialization types in a single
stream are pretty rare I think just because the consumer then has the
problem of guessing how to deserialize, so most of these will end up with
at least some marker or schema id or whatever that tells you how to read
the data. Arguable this mixed serialization with marker is itself a
serializer type and should have a serializer of its own...

-Jay

On Fri, Dec 5, 2014 at 3:48 PM, Sriram Subramanian <
srsubramanian@linkedin.com.invalid> wrote:

> This thread has diverged multiple times now and it would be worth
> summarizing them.
>
> There seems to be the following points of discussion -
>
> 1. Can we keep the serialization semantics outside the Producer interface
> and have simple bytes in / bytes out for the interface (This is what we
> have today).
>
> The points for this is to keep the interface simple and usage easy to
> understand. The points against this is that it gets hard to share common
> usage patterns around serialization/message validations for the future.
>
> 2. Can we create a wrapper producer that does the serialization and have
> different variants of it for different data formats?
>
> The points for this is again to keep the main API clean. The points
> against this is that it duplicates the API, increases the surface area and
> creates redundancy for a minor addition.
>
> 3. Do we need to support different data types per record? The current
> interface (bytes in/bytes out) lets you instantiate one producer and use
> it to send multiple data formats. There seems to be some valid use cases
> for this.
>
> I have still not seen a strong argument against not having this
> functionality. Can someone provide their views on why we don't need this
> support that is possible with the current API?
>
> One possible approach for the per record serialization would be to define
>
> public interface SerDe<K,V> {
>   public byte[] serializeKey();
>
>   public K deserializeKey();
>
>   public byte[] serializeValue();
>
>   public V deserializeValue();
> }
>
> This would be used by both the Producer and the Consumer.
>
> The send APIs can then be
>
> public Future<RecordMetadata> send(ProducerRecord<K,V> record);
> public Future<RecordMetadata> send(ProducerRecord<K,V> record, Callback
> callback);
>
>
> public Future<RecordMetadata> send(ProducerRecord<K,V> record, SerDe<K,V>
> serde);
>
> public Future<RecordMetadata> send(ProducerRecord<K,V> record, SerDe<K,V>
> serde, Callback callback);
>
>
> A default SerDe can be set in the config. The producer would use the
> default from the config if the non-serde send APIs are used. The downside
> to this approach is that we would need to have four variants of Send API
> for the Producer.
>
>
>
>
>
>
> On 12/5/14 3:16 PM, "Jun Rao" <ju...@confluent.io> wrote:
>
> >Jiangjie,
> >
> >The issue with adding the serializer in ProducerRecord is that you need to
> >implement all combinations of serializers for key and value. So, instead
> >of
> >just implementing int and string serializers, you will have to implement
> >all 4 combinations.
> >
> >Adding a new producer constructor like Producer<K, V>(KeySerializer<K>,
> >ValueSerializer<V>, Properties properties) can be useful.
> >
> >Thanks,
> >
> >Jun
> >
> >On Thu, Dec 4, 2014 at 10:33 AM, Jiangjie Qin <jq...@linkedin.com.invalid>
> >wrote:
> >
> >>
> >> I'm just thinking instead of binding serialization with producer,
> >>another
> >> option is to bind serializer/deserializer with
> >> ProducerRecord/ConsumerRecord (please see the detail proposal below.)
> >>            The arguments for this option is:
> >>         A. A single producer could send different message types. There
> >>are
> >> several use cases in LinkedIn for per record serializer
> >>         - In Samza, there are some in-stream order-sensitive control
> >> messages
> >> having different deserializer from other messages.
> >>         - There are use cases which need support for sending both Avro
> >> messages
> >> and raw bytes.
> >>         - Some use cases needs to deserialize some Avro messages into
> >> generic
> >> record and some other messages into specific record.
> >>         B. In current proposal, the serializer/deserilizer is
> >>instantiated
> >> according to config. Compared with that, binding serializer with
> >> ProducerRecord and ConsumerRecord is less error prone.
> >>
> >>
> >>         This option includes the following changes:
> >>         A. Add serializer and deserializer interfaces to replace
> >>serializer
> >> instance from config.
> >>                 Public interface Serializer <K, V> {
> >>                         public byte[] serializeKey(K key);
> >>                         public byte[] serializeValue(V value);
> >>                 }
> >>                 Public interface deserializer <K, V> {
> >>                         Public K deserializeKey(byte[] key);
> >>                         public V deserializeValue(byte[] value);
> >>                 }
> >>
> >>         B. Make ProducerRecord and ConsumerRecord abstract class
> >> implementing
> >> Serializer <K, V> and Deserializer <K, V> respectively.
> >>                 Public abstract class ProducerRecord <K, V> implements
> >> Serializer <K, V>
> >> {...}
> >>                 Public abstract class ConsumerRecord <K, V> implements
> >> Deserializer <K,
> >> V> {...}
> >>
> >>         C. Instead of instantiate the serializer/Deserializer from
> >>config,
> >> let
> >> concrete ProducerRecord/ConsumerRecord extends the abstract class and
> >> override the serialize/deserialize methods.
> >>
> >>                 Public class AvroProducerRecord extends ProducerRecord
> >> <String,
> >> GenericRecord> {
> >>                         ...
> >>                         @Override
> >>                         Public byte[] serializeKey(String key) {Š}
> >>                         @Override
> >>                         public byte[] serializeValue(GenericRecord
> >>value);
> >>                 }
> >>
> >>                 Public class AvroConsumerRecord extends ConsumerRecord
> >> <String,
> >> GenericRecord> {
> >>                         ...
> >>                         @Override
> >>                         Public K deserializeKey(byte[] key) {Š}
> >>                         @Override
> >>                         public V deserializeValue(byte[] value);
> >>                 }
> >>
> >>         D. The producer API changes to
> >>                 Public class KafkaProducer {
> >>                         ...
> >>
> >>                         Future<RecordMetadata> send (ProducerRecord <K,
> >>V>
> >> record) {
> >>                                 ...
> >>                                 K key = record.serializeKey(record.key);
> >>                                 V value =
> >> record.serializedValue(record.value);
> >>                                 BytesProducerRecord bytesProducerRecord
> >>=
> >> new
> >> BytesProducerRecord(topic, partition, key, value);
> >>                                 ...
> >>                         }
> >>                         ...
> >>                 }
> >>
> >>
> >>
> >> We also had some brainstorm in LinkedIn and here are the feedbacks:
> >>
> >> If the community decide to add the serialization back to new producer,
> >> besides current proposal which changes new producer API to be a
> >>template,
> >> there are some other options raised during our discussion:
> >>         1) Rather than change current new producer API, we can provide a
> >> wrapper
> >> of current new producer (e.g. KafkaSerializedProducer) and make it
> >> available to users. As there is value in the simplicity of current API.
> >>
> >>         2) If we decide to go with tempalated new producer API,
> >>according
> >> to
> >> experience in LinkedIn, it might worth considering to instantiate the
> >> serializer in code instead of from config so we can avoid runtime errors
> >> due to dynamic instantiation from config, which is more error prone. If
> >> that is the case, the producer API could be changed to something like:
> >>                 producer = new Producer<K, V>(KeySerializer<K>,
> >> ValueSerializer<V>)
> >>
> >> --Jiangjie (Becket) Qin
> >>
> >>
> >> On 11/24/14, 5:58 PM, "Jun Rao" <ju...@gmail.com> 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
> >>
> >>
>
>

Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Sriram Subramanian <sr...@linkedin.com.INVALID>.
This thread has diverged multiple times now and it would be worth
summarizing them. 

There seems to be the following points of discussion -

1. Can we keep the serialization semantics outside the Producer interface
and have simple bytes in / bytes out for the interface (This is what we
have today).

The points for this is to keep the interface simple and usage easy to
understand. The points against this is that it gets hard to share common
usage patterns around serialization/message validations for the future.

2. Can we create a wrapper producer that does the serialization and have
different variants of it for different data formats?

The points for this is again to keep the main API clean. The points
against this is that it duplicates the API, increases the surface area and
creates redundancy for a minor addition.

3. Do we need to support different data types per record? The current
interface (bytes in/bytes out) lets you instantiate one producer and use
it to send multiple data formats. There seems to be some valid use cases
for this.

I have still not seen a strong argument against not having this
functionality. Can someone provide their views on why we don't need this
support that is possible with the current API?

One possible approach for the per record serialization would be to define

public interface SerDe<K,V> {
  public byte[] serializeKey();

  public K deserializeKey();

  public byte[] serializeValue();

  public V deserializeValue();
}

This would be used by both the Producer and the Consumer.

The send APIs can then be

public Future<RecordMetadata> send(ProducerRecord<K,V> record);
public Future<RecordMetadata> send(ProducerRecord<K,V> record, Callback
callback);


public Future<RecordMetadata> send(ProducerRecord<K,V> record, SerDe<K,V>
serde);

public Future<RecordMetadata> send(ProducerRecord<K,V> record, SerDe<K,V>
serde, Callback callback);


A default SerDe can be set in the config. The producer would use the
default from the config if the non-serde send APIs are used. The downside
to this approach is that we would need to have four variants of Send API
for the Producer. 






On 12/5/14 3:16 PM, "Jun Rao" <ju...@confluent.io> wrote:

>Jiangjie,
>
>The issue with adding the serializer in ProducerRecord is that you need to
>implement all combinations of serializers for key and value. So, instead
>of
>just implementing int and string serializers, you will have to implement
>all 4 combinations.
>
>Adding a new producer constructor like Producer<K, V>(KeySerializer<K>,
>ValueSerializer<V>, Properties properties) can be useful.
>
>Thanks,
>
>Jun
>
>On Thu, Dec 4, 2014 at 10:33 AM, Jiangjie Qin <jq...@linkedin.com.invalid>
>wrote:
>
>>
>> I'm just thinking instead of binding serialization with producer,
>>another
>> option is to bind serializer/deserializer with
>> ProducerRecord/ConsumerRecord (please see the detail proposal below.)
>>            The arguments for this option is:
>>         A. A single producer could send different message types. There
>>are
>> several use cases in LinkedIn for per record serializer
>>         - In Samza, there are some in-stream order-sensitive control
>> messages
>> having different deserializer from other messages.
>>         - There are use cases which need support for sending both Avro
>> messages
>> and raw bytes.
>>         - Some use cases needs to deserialize some Avro messages into
>> generic
>> record and some other messages into specific record.
>>         B. In current proposal, the serializer/deserilizer is
>>instantiated
>> according to config. Compared with that, binding serializer with
>> ProducerRecord and ConsumerRecord is less error prone.
>>
>>
>>         This option includes the following changes:
>>         A. Add serializer and deserializer interfaces to replace
>>serializer
>> instance from config.
>>                 Public interface Serializer <K, V> {
>>                         public byte[] serializeKey(K key);
>>                         public byte[] serializeValue(V value);
>>                 }
>>                 Public interface deserializer <K, V> {
>>                         Public K deserializeKey(byte[] key);
>>                         public V deserializeValue(byte[] value);
>>                 }
>>
>>         B. Make ProducerRecord and ConsumerRecord abstract class
>> implementing
>> Serializer <K, V> and Deserializer <K, V> respectively.
>>                 Public abstract class ProducerRecord <K, V> implements
>> Serializer <K, V>
>> {...}
>>                 Public abstract class ConsumerRecord <K, V> implements
>> Deserializer <K,
>> V> {...}
>>
>>         C. Instead of instantiate the serializer/Deserializer from
>>config,
>> let
>> concrete ProducerRecord/ConsumerRecord extends the abstract class and
>> override the serialize/deserialize methods.
>>
>>                 Public class AvroProducerRecord extends ProducerRecord
>> <String,
>> GenericRecord> {
>>                         ...
>>                         @Override
>>                         Public byte[] serializeKey(String key) {Š}
>>                         @Override
>>                         public byte[] serializeValue(GenericRecord
>>value);
>>                 }
>>
>>                 Public class AvroConsumerRecord extends ConsumerRecord
>> <String,
>> GenericRecord> {
>>                         ...
>>                         @Override
>>                         Public K deserializeKey(byte[] key) {Š}
>>                         @Override
>>                         public V deserializeValue(byte[] value);
>>                 }
>>
>>         D. The producer API changes to
>>                 Public class KafkaProducer {
>>                         ...
>>
>>                         Future<RecordMetadata> send (ProducerRecord <K,
>>V>
>> record) {
>>                                 ...
>>                                 K key = record.serializeKey(record.key);
>>                                 V value =
>> record.serializedValue(record.value);
>>                                 BytesProducerRecord bytesProducerRecord
>>=
>> new
>> BytesProducerRecord(topic, partition, key, value);
>>                                 ...
>>                         }
>>                         ...
>>                 }
>>
>>
>>
>> We also had some brainstorm in LinkedIn and here are the feedbacks:
>>
>> If the community decide to add the serialization back to new producer,
>> besides current proposal which changes new producer API to be a
>>template,
>> there are some other options raised during our discussion:
>>         1) Rather than change current new producer API, we can provide a
>> wrapper
>> of current new producer (e.g. KafkaSerializedProducer) and make it
>> available to users. As there is value in the simplicity of current API.
>>
>>         2) If we decide to go with tempalated new producer API,
>>according
>> to
>> experience in LinkedIn, it might worth considering to instantiate the
>> serializer in code instead of from config so we can avoid runtime errors
>> due to dynamic instantiation from config, which is more error prone. If
>> that is the case, the producer API could be changed to something like:
>>                 producer = new Producer<K, V>(KeySerializer<K>,
>> ValueSerializer<V>)
>>
>> --Jiangjie (Becket) Qin
>>
>>
>> On 11/24/14, 5:58 PM, "Jun Rao" <ju...@gmail.com> 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
>>
>>


Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Jun Rao <ju...@confluent.io>.
Jiangjie,

The issue with adding the serializer in ProducerRecord is that you need to
implement all combinations of serializers for key and value. So, instead of
just implementing int and string serializers, you will have to implement
all 4 combinations.

Adding a new producer constructor like Producer<K, V>(KeySerializer<K>,
ValueSerializer<V>, Properties properties) can be useful.

Thanks,

Jun

On Thu, Dec 4, 2014 at 10:33 AM, Jiangjie Qin <jq...@linkedin.com.invalid>
wrote:

>
> I'm just thinking instead of binding serialization with producer, another
> option is to bind serializer/deserializer with
> ProducerRecord/ConsumerRecord (please see the detail proposal below.)
>            The arguments for this option is:
>         A. A single producer could send different message types. There are
> several use cases in LinkedIn for per record serializer
>         - In Samza, there are some in-stream order-sensitive control
> messages
> having different deserializer from other messages.
>         - There are use cases which need support for sending both Avro
> messages
> and raw bytes.
>         - Some use cases needs to deserialize some Avro messages into
> generic
> record and some other messages into specific record.
>         B. In current proposal, the serializer/deserilizer is instantiated
> according to config. Compared with that, binding serializer with
> ProducerRecord and ConsumerRecord is less error prone.
>
>
>         This option includes the following changes:
>         A. Add serializer and deserializer interfaces to replace serializer
> instance from config.
>                 Public interface Serializer <K, V> {
>                         public byte[] serializeKey(K key);
>                         public byte[] serializeValue(V value);
>                 }
>                 Public interface deserializer <K, V> {
>                         Public K deserializeKey(byte[] key);
>                         public V deserializeValue(byte[] value);
>                 }
>
>         B. Make ProducerRecord and ConsumerRecord abstract class
> implementing
> Serializer <K, V> and Deserializer <K, V> respectively.
>                 Public abstract class ProducerRecord <K, V> implements
> Serializer <K, V>
> {...}
>                 Public abstract class ConsumerRecord <K, V> implements
> Deserializer <K,
> V> {...}
>
>         C. Instead of instantiate the serializer/Deserializer from config,
> let
> concrete ProducerRecord/ConsumerRecord extends the abstract class and
> override the serialize/deserialize methods.
>
>                 Public class AvroProducerRecord extends ProducerRecord
> <String,
> GenericRecord> {
>                         ...
>                         @Override
>                         Public byte[] serializeKey(String key) {Š}
>                         @Override
>                         public byte[] serializeValue(GenericRecord value);
>                 }
>
>                 Public class AvroConsumerRecord extends ConsumerRecord
> <String,
> GenericRecord> {
>                         ...
>                         @Override
>                         Public K deserializeKey(byte[] key) {Š}
>                         @Override
>                         public V deserializeValue(byte[] value);
>                 }
>
>         D. The producer API changes to
>                 Public class KafkaProducer {
>                         ...
>
>                         Future<RecordMetadata> send (ProducerRecord <K, V>
> record) {
>                                 ...
>                                 K key = record.serializeKey(record.key);
>                                 V value =
> record.serializedValue(record.value);
>                                 BytesProducerRecord bytesProducerRecord =
> new
> BytesProducerRecord(topic, partition, key, value);
>                                 ...
>                         }
>                         ...
>                 }
>
>
>
> We also had some brainstorm in LinkedIn and here are the feedbacks:
>
> If the community decide to add the serialization back to new producer,
> besides current proposal which changes new producer API to be a template,
> there are some other options raised during our discussion:
>         1) Rather than change current new producer API, we can provide a
> wrapper
> of current new producer (e.g. KafkaSerializedProducer) and make it
> available to users. As there is value in the simplicity of current API.
>
>         2) If we decide to go with tempalated new producer API, according
> to
> experience in LinkedIn, it might worth considering to instantiate the
> serializer in code instead of from config so we can avoid runtime errors
> due to dynamic instantiation from config, which is more error prone. If
> that is the case, the producer API could be changed to something like:
>                 producer = new Producer<K, V>(KeySerializer<K>,
> ValueSerializer<V>)
>
> --Jiangjie (Becket) Qin
>
>
> On 11/24/14, 5:58 PM, "Jun Rao" <ju...@gmail.com> 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
>
>

Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Jiangjie Qin <jq...@linkedin.com.INVALID>.
I'm just thinking instead of binding serialization with producer, another
option is to bind serializer/deserializer with
ProducerRecord/ConsumerRecord (please see the detail proposal below.)
	   The arguments for this option is:
	A. A single producer could send different message types. There are
several use cases in LinkedIn for per record serializer
	- In Samza, there are some in-stream order-sensitive control messages
having different deserializer from other messages.
	- There are use cases which need support for sending both Avro messages
and raw bytes.
	- Some use cases needs to deserialize some Avro messages into generic
record and some other messages into specific record.
	B. In current proposal, the serializer/deserilizer is instantiated
according to config. Compared with that, binding serializer with
ProducerRecord and ConsumerRecord is less error prone.


	This option includes the following changes:
	A. Add serializer and deserializer interfaces to replace serializer
instance from config.
		Public interface Serializer <K, V> {
			public byte[] serializeKey(K key);
			public byte[] serializeValue(V value);
		}
		Public interface deserializer <K, V> {
			Public K deserializeKey(byte[] key);
			public V deserializeValue(byte[] value);
		}

	B. Make ProducerRecord and ConsumerRecord abstract class implementing
Serializer <K, V> and Deserializer <K, V> respectively.
		Public abstract class ProducerRecord <K, V> implements Serializer <K, V>
{...}
		Public abstract class ConsumerRecord <K, V> implements Deserializer <K,
V> {...}

	C. Instead of instantiate the serializer/Deserializer from config, let
concrete ProducerRecord/ConsumerRecord extends the abstract class and
override the serialize/deserialize methods.

		Public class AvroProducerRecord extends ProducerRecord <String,
GenericRecord> {
			...
			@Override
			Public byte[] serializeKey(String key) {Š}
			@Override
			public byte[] serializeValue(GenericRecord value);
		}

		Public class AvroConsumerRecord extends ConsumerRecord <String,
GenericRecord> {
			...
			@Override
			Public K deserializeKey(byte[] key) {Š}
			@Override
			public V deserializeValue(byte[] value);
		}

	D. The producer API changes to
		Public class KafkaProducer {
			...

			Future<RecordMetadata> send (ProducerRecord <K, V> record) {
				...
				K key = record.serializeKey(record.key);
				V value = record.serializedValue(record.value);
				BytesProducerRecord bytesProducerRecord = new
BytesProducerRecord(topic, partition, key, value);
				...
			}
			...
		}



We also had some brainstorm in LinkedIn and here are the feedbacks:

If the community decide to add the serialization back to new producer,
besides current proposal which changes new producer API to be a template,
there are some other options raised during our discussion:
	1) Rather than change current new producer API, we can provide a wrapper
of current new producer (e.g. KafkaSerializedProducer) and make it
available to users. As there is value in the simplicity of current API.
	
	2) If we decide to go with tempalated new producer API, according to
experience in LinkedIn, it might worth considering to instantiate the
serializer in code instead of from config so we can avoid runtime errors
due to dynamic instantiation from config, which is more error prone. If
that is the case, the producer API could be changed to something like:
		producer = new Producer<K, V>(KeySerializer<K>, ValueSerializer<V>)

--Jiangjie (Becket) Qin


On 11/24/14, 5:58 PM, "Jun Rao" <ju...@gmail.com> 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


Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Jiangjie Qin <jq...@linkedin.com.INVALID>.
I'm just thinking instead of binding serialization with producer, another
option is to bind serializer/deserializer with
ProducerRecord/ConsumerRecord (please see the detail proposal below.)
	   The arguments for this option is:
	A. A single producer could send different message types. There are
several use cases in LinkedIn for per record serializer
	- In Samza, there are some in-stream order-sensitive control messages
having different deserializer from other messages.
	- There are use cases which need support for sending both Avro messages
and raw bytes.
	- Some use cases needs to deserialize some Avro messages into generic
record and some other messages into specific record.
	B. In current proposal, the serializer/deserilizer is instantiated
according to config. Compared with that, binding serializer with
ProducerRecord and ConsumerRecord is less error prone.


	This option includes the following changes:
	A. Add serializer and deserializer interfaces to replace serializer
instance from config.
		Public interface Serializer <K, V> {
			public byte[] serializeKey(K key);
			public byte[] serializeValue(V value);
		}
		Public interface deserializer <K, V> {
			Public K deserializeKey(byte[] key);
			public V deserializeValue(byte[] value);
		}

	B. Make ProducerRecord and ConsumerRecord abstract class implementing
Serializer <K, V> and Deserializer <K, V> respectively.
		Public abstract class ProducerRecord <K, V> implements Serializer <K, V>
{...}
		Public abstract class ConsumerRecord <K, V> implements Deserializer <K,
V> {...}

	C. Instead of instantiate the serializer/Deserializer from config, let
concrete ProducerRecord/ConsumerRecord extends the abstract class and
override the serialize/deserialize methods.

		Public class AvroProducerRecord extends ProducerRecord <String,
GenericRecord> {
			...
			@Override
			Public byte[] serializeKey(String key) {Š}
			@Override
			public byte[] serializeValue(GenericRecord value);
		}

		Public class AvroConsumerRecord extends ConsumerRecord <String,
GenericRecord> {
			...
			@Override
			Public K deserializeKey(byte[] key) {Š}
			@Override
			public V deserializeValue(byte[] value);
		}

	D. The producer API changes to
		Public class KafkaProducer {
			...

			Future<RecordMetadata> send (ProducerRecord <K, V> record) {
				...
				K key = record.serializeKey(record.key);
				V value = record.serializedValue(record.value);
				BytesProducerRecord bytesProducerRecord = new
BytesProducerRecord(topic, partition, key, value);
				...
			}
			...
		}



We also had some brainstorm in LinkedIn and here are the feedbacks:

If the community decide to add the serialization back to new producer,
besides current proposal which changes new producer API to be a template,
there are some other options raised during our discussion:
	1) Rather than change current new producer API, we can provide a wrapper
of current new producer (e.g. KafkaSerializedProducer) and make it
available to users. As there is value in the simplicity of current API.
	
	2) If we decide to go with tempalated new producer API, according to
experience in LinkedIn, it might worth considering to instantiate the
serializer in code instead of from config so we can avoid runtime errors
due to dynamic instantiation from config, which is more error prone. If
that is the case, the producer API could be changed to something like:
		producer = new Producer<K, V>(KeySerializer<K>, ValueSerializer<V>)

--Jiangjie (Becket) Qin


On 11/24/14, 5:58 PM, "Jun Rao" <ju...@gmail.com> 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


Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Jay Kreps <ja...@gmail.com>.
Hey Shlomi,

I agree that we just blew this one from a timing perspective. We ideally
should have thought this through in the original api discussion. But as we
really started to think about this area we realized that the existing api
made it really hard to provide a simple way package of serialization and
data model stuff. I have heard that there is a saying that the best time to
plant a tree is a generation ago, but the second best time is right now.
And I think this is kind of along those lines.

This was really brought home to me as for the last month or so I have been
going around and talking to a lot of people using Kafka and essentially
every one of them has had to make some kind of wrapper api. There is
nothing so terrible about these wrappers except that they make it hard to
have central documentation that explains how the system works, and they
usually strip off a lot of the functionality of the client, so you always
have to learn the in-house wrapper and can't really do everything you could
do with the main client. Since all the wrappers were trying to provide a
few things: serialization, message validation, etc. All of these depend on
having access to the original object. I think if we make this change on
serialization we can later add any additional hooks for message validation
with no compatibility problems.

-Jay

On Tue, Nov 25, 2014 at 12:12 AM, Shlomi Hazan <sh...@viber.com> wrote:

> Jun, while just a humble user, I would like to recall that it was just 6
> days ago that you told me on the user list that the producer is stable when
> I asked what producer to go with and if the new producer is production
> stable (you can still see that email down the list).
> maybe I miss something, but for me, stable includes the API.
> So it looks rather too big and too late from where I am standing to make
> this change now. this kind of change will introduce generics, add major
> mandatory interface, and make the whole producer more complicated then it
> really has to be when you consider only Kafka and not Avro.
> I can see the obvious benefits for the many other use cases, but once you
> declare something stable it is usually expected that the API will not
> change unless something really big was discovered.
> Now it may be the case that you discovered something big enough and so
> personally I will not make a vote.
> If the benefits make the change justifiable is for you guys to decide.
> Shlomi
>
> On Tue, Nov 25, 2014 at 6:43 AM, Sriram Subramanian <
> srsubramanian@linkedin.com.invalid> wrote:
>
> > Looked at the patch. +1 from me.
> >
> > On 11/24/14 8:29 PM, "Gwen Shapira" <gs...@cloudera.com> wrote:
> >
> > >As one of the people who spent too much time building Avro repositories,
> > >+1
> > >on bringing serializer API back.
> > >
> > >I think it will make the new producer easier to work with.
> > >
> > >Gwen
> > >
> > >On Mon, Nov 24, 2014 at 6:13 PM, Jay Kreps <ja...@gmail.com> wrote:
> > >
> > >> This is admittedly late in the release cycle to make a change. To add
> to
> > >> Jun's description the motivation was that we felt it would be better
> to
> > >> change that interface now rather than after the release if it needed
> to
> > >> change.
> > >>
> > >> The motivation for wanting to make a change was the ability to really
> be
> > >> able to develop support for Avro and other serialization formats. The
> > >> current status is pretty scattered--there is a schema repository on an
> > >>Avro
> > >> JIRA and another fork of that on github, and a bunch of people we have
> > >> talked to have done similar things for other serialization systems. It
> > >> would be nice if these things could be packaged in such a way that it
> > >>was
> > >> possible to just change a few configs in the producer and get rich
> > >>metadata
> > >> support for messages.
> > >>
> > >> As we were thinking this through we realized that the new api we were
> > >>about
> > >> to introduce was kind of not very compatable with this since it was
> just
> > >> byte[] oriented.
> > >>
> > >> You can always do this by adding some kind of wrapper api that wraps
> the
> > >> producer. But this puts us back in the position of trying to document
> > >>and
> > >> support multiple interfaces.
> > >>
> > >> This also opens up the possibility of adding a MessageValidator or
> > >> MessageInterceptor plug-in transparently so that you can do other
> custom
> > >> validation on the messages you are sending which obviously requires
> > >>access
> > >> to the original object not the byte array.
> > >>
> > >> This api doesn't prevent using byte[] by configuring the
> > >> ByteArraySerializer it works as it currently does.
> > >>
> > >> -Jay
> > >>
> > >> On Mon, Nov 24, 2014 at 5:58 PM, Jun Rao <ju...@gmail.com> 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
> > >> >
> > >>
> >
> >
>

Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Jun Rao <ju...@gmail.com>.
Shiomi,

Sorry, at that time, I didn't realize that we would be better off with an
api change. Yes, it sucks that we have to break the api. However, if we
have to change it, it's better to do it now rather than later.

Note that if you want to just produce byte[] to Kafka, you can still do
that with the api change. You just need to bind the producer with byte[]
and the default serializer will just work. Yes, there needs to be code
changes. My hope is that right now no one has adopted the new producer api
widely and making such code changes is not very painful yet.

Thanks,

Jun

On Tue, Nov 25, 2014 at 12:12 AM, Shlomi Hazan <sh...@viber.com> wrote:

> Jun, while just a humble user, I would like to recall that it was just 6
> days ago that you told me on the user list that the producer is stable when
> I asked what producer to go with and if the new producer is production
> stable (you can still see that email down the list).
> maybe I miss something, but for me, stable includes the API.
> So it looks rather too big and too late from where I am standing to make
> this change now. this kind of change will introduce generics, add major
> mandatory interface, and make the whole producer more complicated then it
> really has to be when you consider only Kafka and not Avro.
> I can see the obvious benefits for the many other use cases, but once you
> declare something stable it is usually expected that the API will not
> change unless something really big was discovered.
> Now it may be the case that you discovered something big enough and so
> personally I will not make a vote.
> If the benefits make the change justifiable is for you guys to decide.
> Shlomi
>
> On Tue, Nov 25, 2014 at 6:43 AM, Sriram Subramanian <
> srsubramanian@linkedin.com.invalid> wrote:
>
> > Looked at the patch. +1 from me.
> >
> > On 11/24/14 8:29 PM, "Gwen Shapira" <gs...@cloudera.com> wrote:
> >
> > >As one of the people who spent too much time building Avro repositories,
> > >+1
> > >on bringing serializer API back.
> > >
> > >I think it will make the new producer easier to work with.
> > >
> > >Gwen
> > >
> > >On Mon, Nov 24, 2014 at 6:13 PM, Jay Kreps <ja...@gmail.com> wrote:
> > >
> > >> This is admittedly late in the release cycle to make a change. To add
> to
> > >> Jun's description the motivation was that we felt it would be better
> to
> > >> change that interface now rather than after the release if it needed
> to
> > >> change.
> > >>
> > >> The motivation for wanting to make a change was the ability to really
> be
> > >> able to develop support for Avro and other serialization formats. The
> > >> current status is pretty scattered--there is a schema repository on an
> > >>Avro
> > >> JIRA and another fork of that on github, and a bunch of people we have
> > >> talked to have done similar things for other serialization systems. It
> > >> would be nice if these things could be packaged in such a way that it
> > >>was
> > >> possible to just change a few configs in the producer and get rich
> > >>metadata
> > >> support for messages.
> > >>
> > >> As we were thinking this through we realized that the new api we were
> > >>about
> > >> to introduce was kind of not very compatable with this since it was
> just
> > >> byte[] oriented.
> > >>
> > >> You can always do this by adding some kind of wrapper api that wraps
> the
> > >> producer. But this puts us back in the position of trying to document
> > >>and
> > >> support multiple interfaces.
> > >>
> > >> This also opens up the possibility of adding a MessageValidator or
> > >> MessageInterceptor plug-in transparently so that you can do other
> custom
> > >> validation on the messages you are sending which obviously requires
> > >>access
> > >> to the original object not the byte array.
> > >>
> > >> This api doesn't prevent using byte[] by configuring the
> > >> ByteArraySerializer it works as it currently does.
> > >>
> > >> -Jay
> > >>
> > >> On Mon, Nov 24, 2014 at 5:58 PM, Jun Rao <ju...@gmail.com> 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
> > >> >
> > >>
> >
> >
>

Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Shlomi Hazan <sh...@viber.com>.
Jun, while just a humble user, I would like to recall that it was just 6
days ago that you told me on the user list that the producer is stable when
I asked what producer to go with and if the new producer is production
stable (you can still see that email down the list).
maybe I miss something, but for me, stable includes the API.
So it looks rather too big and too late from where I am standing to make
this change now. this kind of change will introduce generics, add major
mandatory interface, and make the whole producer more complicated then it
really has to be when you consider only Kafka and not Avro.
I can see the obvious benefits for the many other use cases, but once you
declare something stable it is usually expected that the API will not
change unless something really big was discovered.
Now it may be the case that you discovered something big enough and so
personally I will not make a vote.
If the benefits make the change justifiable is for you guys to decide.
Shlomi

On Tue, Nov 25, 2014 at 6:43 AM, Sriram Subramanian <
srsubramanian@linkedin.com.invalid> wrote:

> Looked at the patch. +1 from me.
>
> On 11/24/14 8:29 PM, "Gwen Shapira" <gs...@cloudera.com> wrote:
>
> >As one of the people who spent too much time building Avro repositories,
> >+1
> >on bringing serializer API back.
> >
> >I think it will make the new producer easier to work with.
> >
> >Gwen
> >
> >On Mon, Nov 24, 2014 at 6:13 PM, Jay Kreps <ja...@gmail.com> wrote:
> >
> >> This is admittedly late in the release cycle to make a change. To add to
> >> Jun's description the motivation was that we felt it would be better to
> >> change that interface now rather than after the release if it needed to
> >> change.
> >>
> >> The motivation for wanting to make a change was the ability to really be
> >> able to develop support for Avro and other serialization formats. The
> >> current status is pretty scattered--there is a schema repository on an
> >>Avro
> >> JIRA and another fork of that on github, and a bunch of people we have
> >> talked to have done similar things for other serialization systems. It
> >> would be nice if these things could be packaged in such a way that it
> >>was
> >> possible to just change a few configs in the producer and get rich
> >>metadata
> >> support for messages.
> >>
> >> As we were thinking this through we realized that the new api we were
> >>about
> >> to introduce was kind of not very compatable with this since it was just
> >> byte[] oriented.
> >>
> >> You can always do this by adding some kind of wrapper api that wraps the
> >> producer. But this puts us back in the position of trying to document
> >>and
> >> support multiple interfaces.
> >>
> >> This also opens up the possibility of adding a MessageValidator or
> >> MessageInterceptor plug-in transparently so that you can do other custom
> >> validation on the messages you are sending which obviously requires
> >>access
> >> to the original object not the byte array.
> >>
> >> This api doesn't prevent using byte[] by configuring the
> >> ByteArraySerializer it works as it currently does.
> >>
> >> -Jay
> >>
> >> On Mon, Nov 24, 2014 at 5:58 PM, Jun Rao <ju...@gmail.com> 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
> >> >
> >>
>
>

Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Jun Rao <ju...@gmail.com>.
Yes, that will be a separate release. Possibly an 0.8.2 beta-2, followed by
0.8.2 final.

Thanks,

Jun

On Wed, Nov 26, 2014 at 2:24 AM, Shlomi Hazan <sh...@viber.com> wrote:

> Jay, Jun,
> Thank you both for explaining. I understand this is important enough such
> that it must be done, and if so, the sooner the better.
> How will the change be released? a beta-2 or release candidate? I think
> that if possible, it should not overrun the already released version.
> Thank you guys for the hard work.
> Shlomi
>
> On Tue, Nov 25, 2014 at 7:37 PM, Jun Rao <ju...@gmail.com> wrote:
>
> > Bhavesh,
> >
> > This api change doesn't mean you need to change the format of the encoded
> > data. It simply moves the serialization logic from the application to a
> > pluggable serializer. As long as you preserve the serialization logic,
> the
> > consumer should still see the same bytes.
> >
> > If you are talking about how to evolve the data schema over time, that's
> a
> > separate story. Serialization libraries like Avro have better support on
> > schema evolution.
> >
> > Thanks,
> >
> > Jun
> >
> > On Tue, Nov 25, 2014 at 8:41 AM, Bhavesh Mistry <
> > mistry.p.bhavesh@gmail.com>
> > wrote:
> >
> > > How will mix bag will work with Consumer side ?  Entire site can not be
> > > rolled at once so Consumer will have to deals with New and Old
> Serialize
> > > Bytes ?  This could be app team responsibility.  Are you guys targeting
> > > 0.8.2 release, which may break customer who are already using new
> > producer
> > > API (beta version).
> > >
> > > Thanks,
> > >
> > > Bhavesh
> > >
> > > On Tue, Nov 25, 2014 at 8:29 AM, Manikumar Reddy <kumar@nmsworks.co.in
> >
> > > wrote:
> > >
> > > > +1 for this change.
> > > >
> > > > what about de-serializer  class in 0.8.2?  Say i am using new
> producer
> > > with
> > > > Avro and old consumer combination.
> > > > then i need to give custom Decoder implementation for Avro right?.
> > > >
> > > > On Tue, Nov 25, 2014 at 9:19 PM, Joe Stein <jo...@stealth.ly>
> > wrote:
> > > >
> > > > > The serializer is an expected use of the producer/consumer now and
> > > think
> > > > we
> > > > > should continue that support in the new client. As far as breaking
> > the
> > > > API
> > > > > it is why we released the 0.8.2-beta to help get through just these
> > > type
> > > > of
> > > > > blocking issues in a way that the community at large could be
> > involved
> > > in
> > > > > easier with a build/binaries to download and use from maven also.
> > > > >
> > > > > +1 on the change now prior to the 0.8.2 release.
> > > > >
> > > > > - Joe Stein
> > > > >
> > > > >
> > > > > On Mon, Nov 24, 2014 at 11:43 PM, Sriram Subramanian <
> > > > > srsubramanian@linkedin.com.invalid> wrote:
> > > > >
> > > > > > Looked at the patch. +1 from me.
> > > > > >
> > > > > > On 11/24/14 8:29 PM, "Gwen Shapira" <gs...@cloudera.com>
> wrote:
> > > > > >
> > > > > > >As one of the people who spent too much time building Avro
> > > > repositories,
> > > > > > >+1
> > > > > > >on bringing serializer API back.
> > > > > > >
> > > > > > >I think it will make the new producer easier to work with.
> > > > > > >
> > > > > > >Gwen
> > > > > > >
> > > > > > >On Mon, Nov 24, 2014 at 6:13 PM, Jay Kreps <jay.kreps@gmail.com
> >
> > > > wrote:
> > > > > > >
> > > > > > >> This is admittedly late in the release cycle to make a change.
> > To
> > > > add
> > > > > to
> > > > > > >> Jun's description the motivation was that we felt it would be
> > > better
> > > > > to
> > > > > > >> change that interface now rather than after the release if it
> > > needed
> > > > > to
> > > > > > >> change.
> > > > > > >>
> > > > > > >> The motivation for wanting to make a change was the ability to
> > > > really
> > > > > be
> > > > > > >> able to develop support for Avro and other serialization
> > formats.
> > > > The
> > > > > > >> current status is pretty scattered--there is a schema
> repository
> > > on
> > > > an
> > > > > > >>Avro
> > > > > > >> JIRA and another fork of that on github, and a bunch of people
> > we
> > > > have
> > > > > > >> talked to have done similar things for other serialization
> > > systems.
> > > > It
> > > > > > >> would be nice if these things could be packaged in such a way
> > that
> > > > it
> > > > > > >>was
> > > > > > >> possible to just change a few configs in the producer and get
> > rich
> > > > > > >>metadata
> > > > > > >> support for messages.
> > > > > > >>
> > > > > > >> As we were thinking this through we realized that the new api
> we
> > > > were
> > > > > > >>about
> > > > > > >> to introduce was kind of not very compatable with this since
> it
> > > was
> > > > > just
> > > > > > >> byte[] oriented.
> > > > > > >>
> > > > > > >> You can always do this by adding some kind of wrapper api that
> > > wraps
> > > > > the
> > > > > > >> producer. But this puts us back in the position of trying to
> > > > document
> > > > > > >>and
> > > > > > >> support multiple interfaces.
> > > > > > >>
> > > > > > >> This also opens up the possibility of adding a
> MessageValidator
> > or
> > > > > > >> MessageInterceptor plug-in transparently so that you can do
> > other
> > > > > custom
> > > > > > >> validation on the messages you are sending which obviously
> > > requires
> > > > > > >>access
> > > > > > >> to the original object not the byte array.
> > > > > > >>
> > > > > > >> This api doesn't prevent using byte[] by configuring the
> > > > > > >> ByteArraySerializer it works as it currently does.
> > > > > > >>
> > > > > > >> -Jay
> > > > > > >>
> > > > > > >> On Mon, Nov 24, 2014 at 5:58 PM, Jun Rao <ju...@gmail.com>
> > > 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
> > > > > > >> >
> > > > > > >>
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Shlomi Hazan <sh...@viber.com>.
Jay, Jun,
Thank you both for explaining. I understand this is important enough such
that it must be done, and if so, the sooner the better.
How will the change be released? a beta-2 or release candidate? I think
that if possible, it should not overrun the already released version.
Thank you guys for the hard work.
Shlomi

On Tue, Nov 25, 2014 at 7:37 PM, Jun Rao <ju...@gmail.com> wrote:

> Bhavesh,
>
> This api change doesn't mean you need to change the format of the encoded
> data. It simply moves the serialization logic from the application to a
> pluggable serializer. As long as you preserve the serialization logic, the
> consumer should still see the same bytes.
>
> If you are talking about how to evolve the data schema over time, that's a
> separate story. Serialization libraries like Avro have better support on
> schema evolution.
>
> Thanks,
>
> Jun
>
> On Tue, Nov 25, 2014 at 8:41 AM, Bhavesh Mistry <
> mistry.p.bhavesh@gmail.com>
> wrote:
>
> > How will mix bag will work with Consumer side ?  Entire site can not be
> > rolled at once so Consumer will have to deals with New and Old Serialize
> > Bytes ?  This could be app team responsibility.  Are you guys targeting
> > 0.8.2 release, which may break customer who are already using new
> producer
> > API (beta version).
> >
> > Thanks,
> >
> > Bhavesh
> >
> > On Tue, Nov 25, 2014 at 8:29 AM, Manikumar Reddy <ku...@nmsworks.co.in>
> > wrote:
> >
> > > +1 for this change.
> > >
> > > what about de-serializer  class in 0.8.2?  Say i am using new producer
> > with
> > > Avro and old consumer combination.
> > > then i need to give custom Decoder implementation for Avro right?.
> > >
> > > On Tue, Nov 25, 2014 at 9:19 PM, Joe Stein <jo...@stealth.ly>
> wrote:
> > >
> > > > The serializer is an expected use of the producer/consumer now and
> > think
> > > we
> > > > should continue that support in the new client. As far as breaking
> the
> > > API
> > > > it is why we released the 0.8.2-beta to help get through just these
> > type
> > > of
> > > > blocking issues in a way that the community at large could be
> involved
> > in
> > > > easier with a build/binaries to download and use from maven also.
> > > >
> > > > +1 on the change now prior to the 0.8.2 release.
> > > >
> > > > - Joe Stein
> > > >
> > > >
> > > > On Mon, Nov 24, 2014 at 11:43 PM, Sriram Subramanian <
> > > > srsubramanian@linkedin.com.invalid> wrote:
> > > >
> > > > > Looked at the patch. +1 from me.
> > > > >
> > > > > On 11/24/14 8:29 PM, "Gwen Shapira" <gs...@cloudera.com> wrote:
> > > > >
> > > > > >As one of the people who spent too much time building Avro
> > > repositories,
> > > > > >+1
> > > > > >on bringing serializer API back.
> > > > > >
> > > > > >I think it will make the new producer easier to work with.
> > > > > >
> > > > > >Gwen
> > > > > >
> > > > > >On Mon, Nov 24, 2014 at 6:13 PM, Jay Kreps <ja...@gmail.com>
> > > wrote:
> > > > > >
> > > > > >> This is admittedly late in the release cycle to make a change.
> To
> > > add
> > > > to
> > > > > >> Jun's description the motivation was that we felt it would be
> > better
> > > > to
> > > > > >> change that interface now rather than after the release if it
> > needed
> > > > to
> > > > > >> change.
> > > > > >>
> > > > > >> The motivation for wanting to make a change was the ability to
> > > really
> > > > be
> > > > > >> able to develop support for Avro and other serialization
> formats.
> > > The
> > > > > >> current status is pretty scattered--there is a schema repository
> > on
> > > an
> > > > > >>Avro
> > > > > >> JIRA and another fork of that on github, and a bunch of people
> we
> > > have
> > > > > >> talked to have done similar things for other serialization
> > systems.
> > > It
> > > > > >> would be nice if these things could be packaged in such a way
> that
> > > it
> > > > > >>was
> > > > > >> possible to just change a few configs in the producer and get
> rich
> > > > > >>metadata
> > > > > >> support for messages.
> > > > > >>
> > > > > >> As we were thinking this through we realized that the new api we
> > > were
> > > > > >>about
> > > > > >> to introduce was kind of not very compatable with this since it
> > was
> > > > just
> > > > > >> byte[] oriented.
> > > > > >>
> > > > > >> You can always do this by adding some kind of wrapper api that
> > wraps
> > > > the
> > > > > >> producer. But this puts us back in the position of trying to
> > > document
> > > > > >>and
> > > > > >> support multiple interfaces.
> > > > > >>
> > > > > >> This also opens up the possibility of adding a MessageValidator
> or
> > > > > >> MessageInterceptor plug-in transparently so that you can do
> other
> > > > custom
> > > > > >> validation on the messages you are sending which obviously
> > requires
> > > > > >>access
> > > > > >> to the original object not the byte array.
> > > > > >>
> > > > > >> This api doesn't prevent using byte[] by configuring the
> > > > > >> ByteArraySerializer it works as it currently does.
> > > > > >>
> > > > > >> -Jay
> > > > > >>
> > > > > >> On Mon, Nov 24, 2014 at 5:58 PM, Jun Rao <ju...@gmail.com>
> > 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
> > > > > >> >
> > > > > >>
> > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Jun Rao <ju...@gmail.com>.
Bhavesh,

This api change doesn't mean you need to change the format of the encoded
data. It simply moves the serialization logic from the application to a
pluggable serializer. As long as you preserve the serialization logic, the
consumer should still see the same bytes.

If you are talking about how to evolve the data schema over time, that's a
separate story. Serialization libraries like Avro have better support on
schema evolution.

Thanks,

Jun

On Tue, Nov 25, 2014 at 8:41 AM, Bhavesh Mistry <mi...@gmail.com>
wrote:

> How will mix bag will work with Consumer side ?  Entire site can not be
> rolled at once so Consumer will have to deals with New and Old Serialize
> Bytes ?  This could be app team responsibility.  Are you guys targeting
> 0.8.2 release, which may break customer who are already using new producer
> API (beta version).
>
> Thanks,
>
> Bhavesh
>
> On Tue, Nov 25, 2014 at 8:29 AM, Manikumar Reddy <ku...@nmsworks.co.in>
> wrote:
>
> > +1 for this change.
> >
> > what about de-serializer  class in 0.8.2?  Say i am using new producer
> with
> > Avro and old consumer combination.
> > then i need to give custom Decoder implementation for Avro right?.
> >
> > On Tue, Nov 25, 2014 at 9:19 PM, Joe Stein <jo...@stealth.ly> wrote:
> >
> > > The serializer is an expected use of the producer/consumer now and
> think
> > we
> > > should continue that support in the new client. As far as breaking the
> > API
> > > it is why we released the 0.8.2-beta to help get through just these
> type
> > of
> > > blocking issues in a way that the community at large could be involved
> in
> > > easier with a build/binaries to download and use from maven also.
> > >
> > > +1 on the change now prior to the 0.8.2 release.
> > >
> > > - Joe Stein
> > >
> > >
> > > On Mon, Nov 24, 2014 at 11:43 PM, Sriram Subramanian <
> > > srsubramanian@linkedin.com.invalid> wrote:
> > >
> > > > Looked at the patch. +1 from me.
> > > >
> > > > On 11/24/14 8:29 PM, "Gwen Shapira" <gs...@cloudera.com> wrote:
> > > >
> > > > >As one of the people who spent too much time building Avro
> > repositories,
> > > > >+1
> > > > >on bringing serializer API back.
> > > > >
> > > > >I think it will make the new producer easier to work with.
> > > > >
> > > > >Gwen
> > > > >
> > > > >On Mon, Nov 24, 2014 at 6:13 PM, Jay Kreps <ja...@gmail.com>
> > wrote:
> > > > >
> > > > >> This is admittedly late in the release cycle to make a change. To
> > add
> > > to
> > > > >> Jun's description the motivation was that we felt it would be
> better
> > > to
> > > > >> change that interface now rather than after the release if it
> needed
> > > to
> > > > >> change.
> > > > >>
> > > > >> The motivation for wanting to make a change was the ability to
> > really
> > > be
> > > > >> able to develop support for Avro and other serialization formats.
> > The
> > > > >> current status is pretty scattered--there is a schema repository
> on
> > an
> > > > >>Avro
> > > > >> JIRA and another fork of that on github, and a bunch of people we
> > have
> > > > >> talked to have done similar things for other serialization
> systems.
> > It
> > > > >> would be nice if these things could be packaged in such a way that
> > it
> > > > >>was
> > > > >> possible to just change a few configs in the producer and get rich
> > > > >>metadata
> > > > >> support for messages.
> > > > >>
> > > > >> As we were thinking this through we realized that the new api we
> > were
> > > > >>about
> > > > >> to introduce was kind of not very compatable with this since it
> was
> > > just
> > > > >> byte[] oriented.
> > > > >>
> > > > >> You can always do this by adding some kind of wrapper api that
> wraps
> > > the
> > > > >> producer. But this puts us back in the position of trying to
> > document
> > > > >>and
> > > > >> support multiple interfaces.
> > > > >>
> > > > >> This also opens up the possibility of adding a MessageValidator or
> > > > >> MessageInterceptor plug-in transparently so that you can do other
> > > custom
> > > > >> validation on the messages you are sending which obviously
> requires
> > > > >>access
> > > > >> to the original object not the byte array.
> > > > >>
> > > > >> This api doesn't prevent using byte[] by configuring the
> > > > >> ByteArraySerializer it works as it currently does.
> > > > >>
> > > > >> -Jay
> > > > >>
> > > > >> On Mon, Nov 24, 2014 at 5:58 PM, Jun Rao <ju...@gmail.com>
> 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
> > > > >> >
> > > > >>
> > > >
> > > >
> > >
> >
>

Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Bhavesh Mistry <mi...@gmail.com>.
How will mix bag will work with Consumer side ?  Entire site can not be
rolled at once so Consumer will have to deals with New and Old Serialize
Bytes ?  This could be app team responsibility.  Are you guys targeting
0.8.2 release, which may break customer who are already using new producer
API (beta version).

Thanks,

Bhavesh

On Tue, Nov 25, 2014 at 8:29 AM, Manikumar Reddy <ku...@nmsworks.co.in>
wrote:

> +1 for this change.
>
> what about de-serializer  class in 0.8.2?  Say i am using new producer with
> Avro and old consumer combination.
> then i need to give custom Decoder implementation for Avro right?.
>
> On Tue, Nov 25, 2014 at 9:19 PM, Joe Stein <jo...@stealth.ly> wrote:
>
> > The serializer is an expected use of the producer/consumer now and think
> we
> > should continue that support in the new client. As far as breaking the
> API
> > it is why we released the 0.8.2-beta to help get through just these type
> of
> > blocking issues in a way that the community at large could be involved in
> > easier with a build/binaries to download and use from maven also.
> >
> > +1 on the change now prior to the 0.8.2 release.
> >
> > - Joe Stein
> >
> >
> > On Mon, Nov 24, 2014 at 11:43 PM, Sriram Subramanian <
> > srsubramanian@linkedin.com.invalid> wrote:
> >
> > > Looked at the patch. +1 from me.
> > >
> > > On 11/24/14 8:29 PM, "Gwen Shapira" <gs...@cloudera.com> wrote:
> > >
> > > >As one of the people who spent too much time building Avro
> repositories,
> > > >+1
> > > >on bringing serializer API back.
> > > >
> > > >I think it will make the new producer easier to work with.
> > > >
> > > >Gwen
> > > >
> > > >On Mon, Nov 24, 2014 at 6:13 PM, Jay Kreps <ja...@gmail.com>
> wrote:
> > > >
> > > >> This is admittedly late in the release cycle to make a change. To
> add
> > to
> > > >> Jun's description the motivation was that we felt it would be better
> > to
> > > >> change that interface now rather than after the release if it needed
> > to
> > > >> change.
> > > >>
> > > >> The motivation for wanting to make a change was the ability to
> really
> > be
> > > >> able to develop support for Avro and other serialization formats.
> The
> > > >> current status is pretty scattered--there is a schema repository on
> an
> > > >>Avro
> > > >> JIRA and another fork of that on github, and a bunch of people we
> have
> > > >> talked to have done similar things for other serialization systems.
> It
> > > >> would be nice if these things could be packaged in such a way that
> it
> > > >>was
> > > >> possible to just change a few configs in the producer and get rich
> > > >>metadata
> > > >> support for messages.
> > > >>
> > > >> As we were thinking this through we realized that the new api we
> were
> > > >>about
> > > >> to introduce was kind of not very compatable with this since it was
> > just
> > > >> byte[] oriented.
> > > >>
> > > >> You can always do this by adding some kind of wrapper api that wraps
> > the
> > > >> producer. But this puts us back in the position of trying to
> document
> > > >>and
> > > >> support multiple interfaces.
> > > >>
> > > >> This also opens up the possibility of adding a MessageValidator or
> > > >> MessageInterceptor plug-in transparently so that you can do other
> > custom
> > > >> validation on the messages you are sending which obviously requires
> > > >>access
> > > >> to the original object not the byte array.
> > > >>
> > > >> This api doesn't prevent using byte[] by configuring the
> > > >> ByteArraySerializer it works as it currently does.
> > > >>
> > > >> -Jay
> > > >>
> > > >> On Mon, Nov 24, 2014 at 5:58 PM, Jun Rao <ju...@gmail.com> 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
> > > >> >
> > > >>
> > >
> > >
> >
>

Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Jun Rao <ju...@gmail.com>.
The old consumer already takes a deserializer when creating streams. So you
plug in your decoder there.

Thanks,

Jun

On Tue, Nov 25, 2014 at 8:29 AM, Manikumar Reddy <ku...@nmsworks.co.in>
wrote:

> +1 for this change.
>
> what about de-serializer  class in 0.8.2?  Say i am using new producer with
> Avro and old consumer combination.
> then i need to give custom Decoder implementation for Avro right?.
>
> On Tue, Nov 25, 2014 at 9:19 PM, Joe Stein <jo...@stealth.ly> wrote:
>
> > The serializer is an expected use of the producer/consumer now and think
> we
> > should continue that support in the new client. As far as breaking the
> API
> > it is why we released the 0.8.2-beta to help get through just these type
> of
> > blocking issues in a way that the community at large could be involved in
> > easier with a build/binaries to download and use from maven also.
> >
> > +1 on the change now prior to the 0.8.2 release.
> >
> > - Joe Stein
> >
> >
> > On Mon, Nov 24, 2014 at 11:43 PM, Sriram Subramanian <
> > srsubramanian@linkedin.com.invalid> wrote:
> >
> > > Looked at the patch. +1 from me.
> > >
> > > On 11/24/14 8:29 PM, "Gwen Shapira" <gs...@cloudera.com> wrote:
> > >
> > > >As one of the people who spent too much time building Avro
> repositories,
> > > >+1
> > > >on bringing serializer API back.
> > > >
> > > >I think it will make the new producer easier to work with.
> > > >
> > > >Gwen
> > > >
> > > >On Mon, Nov 24, 2014 at 6:13 PM, Jay Kreps <ja...@gmail.com>
> wrote:
> > > >
> > > >> This is admittedly late in the release cycle to make a change. To
> add
> > to
> > > >> Jun's description the motivation was that we felt it would be better
> > to
> > > >> change that interface now rather than after the release if it needed
> > to
> > > >> change.
> > > >>
> > > >> The motivation for wanting to make a change was the ability to
> really
> > be
> > > >> able to develop support for Avro and other serialization formats.
> The
> > > >> current status is pretty scattered--there is a schema repository on
> an
> > > >>Avro
> > > >> JIRA and another fork of that on github, and a bunch of people we
> have
> > > >> talked to have done similar things for other serialization systems.
> It
> > > >> would be nice if these things could be packaged in such a way that
> it
> > > >>was
> > > >> possible to just change a few configs in the producer and get rich
> > > >>metadata
> > > >> support for messages.
> > > >>
> > > >> As we were thinking this through we realized that the new api we
> were
> > > >>about
> > > >> to introduce was kind of not very compatable with this since it was
> > just
> > > >> byte[] oriented.
> > > >>
> > > >> You can always do this by adding some kind of wrapper api that wraps
> > the
> > > >> producer. But this puts us back in the position of trying to
> document
> > > >>and
> > > >> support multiple interfaces.
> > > >>
> > > >> This also opens up the possibility of adding a MessageValidator or
> > > >> MessageInterceptor plug-in transparently so that you can do other
> > custom
> > > >> validation on the messages you are sending which obviously requires
> > > >>access
> > > >> to the original object not the byte array.
> > > >>
> > > >> This api doesn't prevent using byte[] by configuring the
> > > >> ByteArraySerializer it works as it currently does.
> > > >>
> > > >> -Jay
> > > >>
> > > >> On Mon, Nov 24, 2014 at 5:58 PM, Jun Rao <ju...@gmail.com> 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
> > > >> >
> > > >>
> > >
> > >
> >
>

Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Bhavesh Mistry <mi...@gmail.com>.
How will mix bag will work with Consumer side ?  Entire site can not be
rolled at once so Consumer will have to deals with New and Old Serialize
Bytes ?  This could be app team responsibility.  Are you guys targeting
0.8.2 release, which may break customer who are already using new producer
API (beta version).

Thanks,

Bhavesh

On Tue, Nov 25, 2014 at 8:29 AM, Manikumar Reddy <ku...@nmsworks.co.in>
wrote:

> +1 for this change.
>
> what about de-serializer  class in 0.8.2?  Say i am using new producer with
> Avro and old consumer combination.
> then i need to give custom Decoder implementation for Avro right?.
>
> On Tue, Nov 25, 2014 at 9:19 PM, Joe Stein <jo...@stealth.ly> wrote:
>
> > The serializer is an expected use of the producer/consumer now and think
> we
> > should continue that support in the new client. As far as breaking the
> API
> > it is why we released the 0.8.2-beta to help get through just these type
> of
> > blocking issues in a way that the community at large could be involved in
> > easier with a build/binaries to download and use from maven also.
> >
> > +1 on the change now prior to the 0.8.2 release.
> >
> > - Joe Stein
> >
> >
> > On Mon, Nov 24, 2014 at 11:43 PM, Sriram Subramanian <
> > srsubramanian@linkedin.com.invalid> wrote:
> >
> > > Looked at the patch. +1 from me.
> > >
> > > On 11/24/14 8:29 PM, "Gwen Shapira" <gs...@cloudera.com> wrote:
> > >
> > > >As one of the people who spent too much time building Avro
> repositories,
> > > >+1
> > > >on bringing serializer API back.
> > > >
> > > >I think it will make the new producer easier to work with.
> > > >
> > > >Gwen
> > > >
> > > >On Mon, Nov 24, 2014 at 6:13 PM, Jay Kreps <ja...@gmail.com>
> wrote:
> > > >
> > > >> This is admittedly late in the release cycle to make a change. To
> add
> > to
> > > >> Jun's description the motivation was that we felt it would be better
> > to
> > > >> change that interface now rather than after the release if it needed
> > to
> > > >> change.
> > > >>
> > > >> The motivation for wanting to make a change was the ability to
> really
> > be
> > > >> able to develop support for Avro and other serialization formats.
> The
> > > >> current status is pretty scattered--there is a schema repository on
> an
> > > >>Avro
> > > >> JIRA and another fork of that on github, and a bunch of people we
> have
> > > >> talked to have done similar things for other serialization systems.
> It
> > > >> would be nice if these things could be packaged in such a way that
> it
> > > >>was
> > > >> possible to just change a few configs in the producer and get rich
> > > >>metadata
> > > >> support for messages.
> > > >>
> > > >> As we were thinking this through we realized that the new api we
> were
> > > >>about
> > > >> to introduce was kind of not very compatable with this since it was
> > just
> > > >> byte[] oriented.
> > > >>
> > > >> You can always do this by adding some kind of wrapper api that wraps
> > the
> > > >> producer. But this puts us back in the position of trying to
> document
> > > >>and
> > > >> support multiple interfaces.
> > > >>
> > > >> This also opens up the possibility of adding a MessageValidator or
> > > >> MessageInterceptor plug-in transparently so that you can do other
> > custom
> > > >> validation on the messages you are sending which obviously requires
> > > >>access
> > > >> to the original object not the byte array.
> > > >>
> > > >> This api doesn't prevent using byte[] by configuring the
> > > >> ByteArraySerializer it works as it currently does.
> > > >>
> > > >> -Jay
> > > >>
> > > >> On Mon, Nov 24, 2014 at 5:58 PM, Jun Rao <ju...@gmail.com> 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
> > > >> >
> > > >>
> > >
> > >
> >
>

Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Manikumar Reddy <ku...@nmsworks.co.in>.
+1 for this change.

what about de-serializer  class in 0.8.2?  Say i am using new producer with
Avro and old consumer combination.
then i need to give custom Decoder implementation for Avro right?.

On Tue, Nov 25, 2014 at 9:19 PM, Joe Stein <jo...@stealth.ly> wrote:

> The serializer is an expected use of the producer/consumer now and think we
> should continue that support in the new client. As far as breaking the API
> it is why we released the 0.8.2-beta to help get through just these type of
> blocking issues in a way that the community at large could be involved in
> easier with a build/binaries to download and use from maven also.
>
> +1 on the change now prior to the 0.8.2 release.
>
> - Joe Stein
>
>
> On Mon, Nov 24, 2014 at 11:43 PM, Sriram Subramanian <
> srsubramanian@linkedin.com.invalid> wrote:
>
> > Looked at the patch. +1 from me.
> >
> > On 11/24/14 8:29 PM, "Gwen Shapira" <gs...@cloudera.com> wrote:
> >
> > >As one of the people who spent too much time building Avro repositories,
> > >+1
> > >on bringing serializer API back.
> > >
> > >I think it will make the new producer easier to work with.
> > >
> > >Gwen
> > >
> > >On Mon, Nov 24, 2014 at 6:13 PM, Jay Kreps <ja...@gmail.com> wrote:
> > >
> > >> This is admittedly late in the release cycle to make a change. To add
> to
> > >> Jun's description the motivation was that we felt it would be better
> to
> > >> change that interface now rather than after the release if it needed
> to
> > >> change.
> > >>
> > >> The motivation for wanting to make a change was the ability to really
> be
> > >> able to develop support for Avro and other serialization formats. The
> > >> current status is pretty scattered--there is a schema repository on an
> > >>Avro
> > >> JIRA and another fork of that on github, and a bunch of people we have
> > >> talked to have done similar things for other serialization systems. It
> > >> would be nice if these things could be packaged in such a way that it
> > >>was
> > >> possible to just change a few configs in the producer and get rich
> > >>metadata
> > >> support for messages.
> > >>
> > >> As we were thinking this through we realized that the new api we were
> > >>about
> > >> to introduce was kind of not very compatable with this since it was
> just
> > >> byte[] oriented.
> > >>
> > >> You can always do this by adding some kind of wrapper api that wraps
> the
> > >> producer. But this puts us back in the position of trying to document
> > >>and
> > >> support multiple interfaces.
> > >>
> > >> This also opens up the possibility of adding a MessageValidator or
> > >> MessageInterceptor plug-in transparently so that you can do other
> custom
> > >> validation on the messages you are sending which obviously requires
> > >>access
> > >> to the original object not the byte array.
> > >>
> > >> This api doesn't prevent using byte[] by configuring the
> > >> ByteArraySerializer it works as it currently does.
> > >>
> > >> -Jay
> > >>
> > >> On Mon, Nov 24, 2014 at 5:58 PM, Jun Rao <ju...@gmail.com> 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
> > >> >
> > >>
> >
> >
>

Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Manikumar Reddy <ku...@nmsworks.co.in>.
+1 for this change.

what about de-serializer  class in 0.8.2?  Say i am using new producer with
Avro and old consumer combination.
then i need to give custom Decoder implementation for Avro right?.

On Tue, Nov 25, 2014 at 9:19 PM, Joe Stein <jo...@stealth.ly> wrote:

> The serializer is an expected use of the producer/consumer now and think we
> should continue that support in the new client. As far as breaking the API
> it is why we released the 0.8.2-beta to help get through just these type of
> blocking issues in a way that the community at large could be involved in
> easier with a build/binaries to download and use from maven also.
>
> +1 on the change now prior to the 0.8.2 release.
>
> - Joe Stein
>
>
> On Mon, Nov 24, 2014 at 11:43 PM, Sriram Subramanian <
> srsubramanian@linkedin.com.invalid> wrote:
>
> > Looked at the patch. +1 from me.
> >
> > On 11/24/14 8:29 PM, "Gwen Shapira" <gs...@cloudera.com> wrote:
> >
> > >As one of the people who spent too much time building Avro repositories,
> > >+1
> > >on bringing serializer API back.
> > >
> > >I think it will make the new producer easier to work with.
> > >
> > >Gwen
> > >
> > >On Mon, Nov 24, 2014 at 6:13 PM, Jay Kreps <ja...@gmail.com> wrote:
> > >
> > >> This is admittedly late in the release cycle to make a change. To add
> to
> > >> Jun's description the motivation was that we felt it would be better
> to
> > >> change that interface now rather than after the release if it needed
> to
> > >> change.
> > >>
> > >> The motivation for wanting to make a change was the ability to really
> be
> > >> able to develop support for Avro and other serialization formats. The
> > >> current status is pretty scattered--there is a schema repository on an
> > >>Avro
> > >> JIRA and another fork of that on github, and a bunch of people we have
> > >> talked to have done similar things for other serialization systems. It
> > >> would be nice if these things could be packaged in such a way that it
> > >>was
> > >> possible to just change a few configs in the producer and get rich
> > >>metadata
> > >> support for messages.
> > >>
> > >> As we were thinking this through we realized that the new api we were
> > >>about
> > >> to introduce was kind of not very compatable with this since it was
> just
> > >> byte[] oriented.
> > >>
> > >> You can always do this by adding some kind of wrapper api that wraps
> the
> > >> producer. But this puts us back in the position of trying to document
> > >>and
> > >> support multiple interfaces.
> > >>
> > >> This also opens up the possibility of adding a MessageValidator or
> > >> MessageInterceptor plug-in transparently so that you can do other
> custom
> > >> validation on the messages you are sending which obviously requires
> > >>access
> > >> to the original object not the byte array.
> > >>
> > >> This api doesn't prevent using byte[] by configuring the
> > >> ByteArraySerializer it works as it currently does.
> > >>
> > >> -Jay
> > >>
> > >> On Mon, Nov 24, 2014 at 5:58 PM, Jun Rao <ju...@gmail.com> 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
> > >> >
> > >>
> >
> >
>

Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Joe Stein <jo...@stealth.ly>.
The serializer is an expected use of the producer/consumer now and think we
should continue that support in the new client. As far as breaking the API
it is why we released the 0.8.2-beta to help get through just these type of
blocking issues in a way that the community at large could be involved in
easier with a build/binaries to download and use from maven also.

+1 on the change now prior to the 0.8.2 release.

- Joe Stein


On Mon, Nov 24, 2014 at 11:43 PM, Sriram Subramanian <
srsubramanian@linkedin.com.invalid> wrote:

> Looked at the patch. +1 from me.
>
> On 11/24/14 8:29 PM, "Gwen Shapira" <gs...@cloudera.com> wrote:
>
> >As one of the people who spent too much time building Avro repositories,
> >+1
> >on bringing serializer API back.
> >
> >I think it will make the new producer easier to work with.
> >
> >Gwen
> >
> >On Mon, Nov 24, 2014 at 6:13 PM, Jay Kreps <ja...@gmail.com> wrote:
> >
> >> This is admittedly late in the release cycle to make a change. To add to
> >> Jun's description the motivation was that we felt it would be better to
> >> change that interface now rather than after the release if it needed to
> >> change.
> >>
> >> The motivation for wanting to make a change was the ability to really be
> >> able to develop support for Avro and other serialization formats. The
> >> current status is pretty scattered--there is a schema repository on an
> >>Avro
> >> JIRA and another fork of that on github, and a bunch of people we have
> >> talked to have done similar things for other serialization systems. It
> >> would be nice if these things could be packaged in such a way that it
> >>was
> >> possible to just change a few configs in the producer and get rich
> >>metadata
> >> support for messages.
> >>
> >> As we were thinking this through we realized that the new api we were
> >>about
> >> to introduce was kind of not very compatable with this since it was just
> >> byte[] oriented.
> >>
> >> You can always do this by adding some kind of wrapper api that wraps the
> >> producer. But this puts us back in the position of trying to document
> >>and
> >> support multiple interfaces.
> >>
> >> This also opens up the possibility of adding a MessageValidator or
> >> MessageInterceptor plug-in transparently so that you can do other custom
> >> validation on the messages you are sending which obviously requires
> >>access
> >> to the original object not the byte array.
> >>
> >> This api doesn't prevent using byte[] by configuring the
> >> ByteArraySerializer it works as it currently does.
> >>
> >> -Jay
> >>
> >> On Mon, Nov 24, 2014 at 5:58 PM, Jun Rao <ju...@gmail.com> 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
> >> >
> >>
>
>

Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Joe Stein <jo...@stealth.ly>.
The serializer is an expected use of the producer/consumer now and think we
should continue that support in the new client. As far as breaking the API
it is why we released the 0.8.2-beta to help get through just these type of
blocking issues in a way that the community at large could be involved in
easier with a build/binaries to download and use from maven also.

+1 on the change now prior to the 0.8.2 release.

- Joe Stein


On Mon, Nov 24, 2014 at 11:43 PM, Sriram Subramanian <
srsubramanian@linkedin.com.invalid> wrote:

> Looked at the patch. +1 from me.
>
> On 11/24/14 8:29 PM, "Gwen Shapira" <gs...@cloudera.com> wrote:
>
> >As one of the people who spent too much time building Avro repositories,
> >+1
> >on bringing serializer API back.
> >
> >I think it will make the new producer easier to work with.
> >
> >Gwen
> >
> >On Mon, Nov 24, 2014 at 6:13 PM, Jay Kreps <ja...@gmail.com> wrote:
> >
> >> This is admittedly late in the release cycle to make a change. To add to
> >> Jun's description the motivation was that we felt it would be better to
> >> change that interface now rather than after the release if it needed to
> >> change.
> >>
> >> The motivation for wanting to make a change was the ability to really be
> >> able to develop support for Avro and other serialization formats. The
> >> current status is pretty scattered--there is a schema repository on an
> >>Avro
> >> JIRA and another fork of that on github, and a bunch of people we have
> >> talked to have done similar things for other serialization systems. It
> >> would be nice if these things could be packaged in such a way that it
> >>was
> >> possible to just change a few configs in the producer and get rich
> >>metadata
> >> support for messages.
> >>
> >> As we were thinking this through we realized that the new api we were
> >>about
> >> to introduce was kind of not very compatable with this since it was just
> >> byte[] oriented.
> >>
> >> You can always do this by adding some kind of wrapper api that wraps the
> >> producer. But this puts us back in the position of trying to document
> >>and
> >> support multiple interfaces.
> >>
> >> This also opens up the possibility of adding a MessageValidator or
> >> MessageInterceptor plug-in transparently so that you can do other custom
> >> validation on the messages you are sending which obviously requires
> >>access
> >> to the original object not the byte array.
> >>
> >> This api doesn't prevent using byte[] by configuring the
> >> ByteArraySerializer it works as it currently does.
> >>
> >> -Jay
> >>
> >> On Mon, Nov 24, 2014 at 5:58 PM, Jun Rao <ju...@gmail.com> 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
> >> >
> >>
>
>

Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Sriram Subramanian <sr...@linkedin.com.INVALID>.
Looked at the patch. +1 from me.

On 11/24/14 8:29 PM, "Gwen Shapira" <gs...@cloudera.com> wrote:

>As one of the people who spent too much time building Avro repositories,
>+1
>on bringing serializer API back.
>
>I think it will make the new producer easier to work with.
>
>Gwen
>
>On Mon, Nov 24, 2014 at 6:13 PM, Jay Kreps <ja...@gmail.com> wrote:
>
>> This is admittedly late in the release cycle to make a change. To add to
>> Jun's description the motivation was that we felt it would be better to
>> change that interface now rather than after the release if it needed to
>> change.
>>
>> The motivation for wanting to make a change was the ability to really be
>> able to develop support for Avro and other serialization formats. The
>> current status is pretty scattered--there is a schema repository on an
>>Avro
>> JIRA and another fork of that on github, and a bunch of people we have
>> talked to have done similar things for other serialization systems. It
>> would be nice if these things could be packaged in such a way that it
>>was
>> possible to just change a few configs in the producer and get rich
>>metadata
>> support for messages.
>>
>> As we were thinking this through we realized that the new api we were
>>about
>> to introduce was kind of not very compatable with this since it was just
>> byte[] oriented.
>>
>> You can always do this by adding some kind of wrapper api that wraps the
>> producer. But this puts us back in the position of trying to document
>>and
>> support multiple interfaces.
>>
>> This also opens up the possibility of adding a MessageValidator or
>> MessageInterceptor plug-in transparently so that you can do other custom
>> validation on the messages you are sending which obviously requires
>>access
>> to the original object not the byte array.
>>
>> This api doesn't prevent using byte[] by configuring the
>> ByteArraySerializer it works as it currently does.
>>
>> -Jay
>>
>> On Mon, Nov 24, 2014 at 5:58 PM, Jun Rao <ju...@gmail.com> 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
>> >
>>


Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Gwen Shapira <gs...@cloudera.com>.
As one of the people who spent too much time building Avro repositories, +1
on bringing serializer API back.

I think it will make the new producer easier to work with.

Gwen

On Mon, Nov 24, 2014 at 6:13 PM, Jay Kreps <ja...@gmail.com> wrote:

> This is admittedly late in the release cycle to make a change. To add to
> Jun's description the motivation was that we felt it would be better to
> change that interface now rather than after the release if it needed to
> change.
>
> The motivation for wanting to make a change was the ability to really be
> able to develop support for Avro and other serialization formats. The
> current status is pretty scattered--there is a schema repository on an Avro
> JIRA and another fork of that on github, and a bunch of people we have
> talked to have done similar things for other serialization systems. It
> would be nice if these things could be packaged in such a way that it was
> possible to just change a few configs in the producer and get rich metadata
> support for messages.
>
> As we were thinking this through we realized that the new api we were about
> to introduce was kind of not very compatable with this since it was just
> byte[] oriented.
>
> You can always do this by adding some kind of wrapper api that wraps the
> producer. But this puts us back in the position of trying to document and
> support multiple interfaces.
>
> This also opens up the possibility of adding a MessageValidator or
> MessageInterceptor plug-in transparently so that you can do other custom
> validation on the messages you are sending which obviously requires access
> to the original object not the byte array.
>
> This api doesn't prevent using byte[] by configuring the
> ByteArraySerializer it works as it currently does.
>
> -Jay
>
> On Mon, Nov 24, 2014 at 5:58 PM, Jun Rao <ju...@gmail.com> 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
> >
>

Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Jay Kreps <ja...@gmail.com>.
This is admittedly late in the release cycle to make a change. To add to
Jun's description the motivation was that we felt it would be better to
change that interface now rather than after the release if it needed to
change.

The motivation for wanting to make a change was the ability to really be
able to develop support for Avro and other serialization formats. The
current status is pretty scattered--there is a schema repository on an Avro
JIRA and another fork of that on github, and a bunch of people we have
talked to have done similar things for other serialization systems. It
would be nice if these things could be packaged in such a way that it was
possible to just change a few configs in the producer and get rich metadata
support for messages.

As we were thinking this through we realized that the new api we were about
to introduce was kind of not very compatable with this since it was just
byte[] oriented.

You can always do this by adding some kind of wrapper api that wraps the
producer. But this puts us back in the position of trying to document and
support multiple interfaces.

This also opens up the possibility of adding a MessageValidator or
MessageInterceptor plug-in transparently so that you can do other custom
validation on the messages you are sending which obviously requires access
to the original object not the byte array.

This api doesn't prevent using byte[] by configuring the
ByteArraySerializer it works as it currently does.

-Jay

On Mon, Nov 24, 2014 at 5:58 PM, Jun Rao <ju...@gmail.com> 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
>

Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Gwen Shapira <gs...@cloudera.com>.
Can you elaborate a bit on what an object API wrapper will look like?

Since the serialization API already exists today, its very easy to
know how I'll use the new producer with serialization - exactly the
same way I use the existing one.
If we are proposing a change that will require significant changes in
how we serialize / deserialize, I'd like to see the API so I can
estimate the impact.

Gwen

On Thu, Dec 4, 2014 at 10:19 AM, Guozhang Wang <wa...@gmail.com> wrote:
> I would prefer making the kafka producer as is and wrap the object API on
> top rather than wiring the serializer configs into producers. Some thoughts:
>
> 1. For code sharing, I think it may only be effective for though simple
> functions such as string serialization, etc. For Avro / Shrift / PB, the
> serialization logic would be quite hard to share across organizations:
> imagine some people wants to use Avro 1.7 while others are still staying
> with 1.4 which are not API compatible, while some people use a schema
> registry server for clients to communicate while others compile the schemas
> into source code, etc. So I think in the end having those simple object
> serialization code into kafka.api package and letting applications write
> their own complicated serialization wrapper would be as beneficial as this
> approach.
>
> 2. For code simplicity I do not see a huge difference between a wired
> serializer, which will call serializer.encode() inside the producer, with a
> wrapper, which will call the same outside the producer, or a typed record,
> which will call record.encode() inside the producer.
>
> 3. For less error-proneness, people always mess with the config settings
> especially when they use hierarchical / nested wiring of configs, and such
> mistakes will only be detected on runtime but not compilation time. In the
> past we have seem a lot of such cases with the old producer APIs that
> wire-in the serializer class. If we move this to a SerDe interface, for
> example KafkaProducer<K, V>(KeySer<K>, ValueSer<V>) such errors will be
> detected at compilation.
>
> 4. For data type flexibility, the current approach bind one producer
> instance to a fixed record type. This may be OK in most cases as people
> usually only use a single data type but there are some cases where we would
> like to have a single producer to be able to send multiple typed messages,
> like control messages, or even with a single serialization like Avro we
> would sometimes want to have GenericaRecord and IndexedRecord for some
> specific types.
>
>
> Guozhang
>
> On Wed, Dec 3, 2014 at 2:54 PM, Jun Rao <ju...@confluent.io> wrote:
>
>> Jan, Jason,
>>
>> First, within an Kafka cluster, it's unlikely that each topic has a
>> different type serializer. Like Jason mentioned, Square standardizes on
>> protocol. Many other places such as LinkedIn standardize on Avro.
>>
>> Second, dealing with bytes only has limited use cases. Other than copying
>> bytes around, there isn't much else that one can do. Even for the case of
>> copying data from Kafka into HDFS, often you will need to (1) extract the
>> timestamp so that you can partition the data properly; (2) extract
>> individual fields if you want to put the data in a column-oriented storage
>> format. So, most interesting clients likely need to deal with objects
>> instead of bytes.
>>
>> Finally, the generic api doesn't prevent one from using just the bytes. The
>> additional overhead is just a method call, which the old clients are
>> already paying. Having both a raw bytes and a generic api is probably going
>> to confuse the users more.
>>
>> Thanks,
>>
>> Jun
>>
>>
>>
>> On Tue, Dec 2, 2014 at 6:50 PM, Jan Filipiak <Ja...@trivago.com>
>> wrote:
>>
>> > Hello Everyone,
>> >
>> > I would very much appreciate if someone could provide me a real world
>> > examplewhere it is more convenient to implement the serializers instead
>> of
>> > just making sure to provide bytearrays.
>> >
>> > The code we came up with explicitly avoids the serializer api. I think it
>> > is common understanding that if you want to transport data you need to
>> have
>> > it as a bytearray.
>> >
>> > If at all I personally would like to have a serializer interface that
>> > takes the same types as the producer
>> >
>> > public interface Serializer<K,V> extends Configurable {
>> >     public byte[] serializeKey(K data);
>> >     public byte[] serializeValue(V data);
>> >     public void close();
>> > }
>> >
>> > this would avoid long serialize implementations with branches like
>> > "switch(topic)" or "if(isKey)". Further serializer per topic makes more
>> > sense in my opinion. It feels natural to have a one to one relationship
>> > from types to topics or at least only a few partition per type. But as we
>> > inherit the type from the producer we would have to create many
>> producers.
>> > This would create additional unnecessary connections to the brokers. With
>> > the serializers we create a one type to all topics relationship and the
>> > only type that satisfies that is the bytearray or Object. Am I missing
>> > something here? As said in the beginning I would like to that usecase
>> that
>> > really benefits from using the serializers. I think in theory they sound
>> > great but they cause real practical issues that may lead users to wrong
>> > decisions.
>> >
>> > -1 for putting the serializers back in.
>> >
>> > Looking forward to replies that can show me the benefit of serializes and
>> > especially how the
>> > Type => topic relationship can be handled nicely.
>> >
>> > Best
>> > Jan
>> >
>> >
>> >
>> >
>> > On 25.11.2014 02:58, 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
>> >>
>> >>
>> >
>>
>
>
>
> --
> -- Guozhang

Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Jay Kreps <ja...@gmail.com>.
Hey Guozhang,

These are good points, let me try to address them.

1. Our goal is to be able to provide a best-of-breed serialization package
that works out of the box that does most of the magic. This best-of-breed
plugin would allow schemas, schema evolution, compatibility checks, etc. We
think if this is good enough most people will use it. We spent the last few
months talking with Kafka users and this is an area where there really is a
lot of room for improvement (seriously many people are just sending csv
data or have no standard at all). Some people may want to customize this
logic, but still they will be able to easily bundle up their customized
logic using this api and have every application in their organization
easily plug it in. Our primary goal is to have all applications in an
organization be able to share an approach to data serialization while still
programming against the public Kafka api.

2. I think what you are saying is that there isn't a big functional
difference between
     producer.send(encoder.encode(key), encoder.encode(value)
and
    producer.send(key, value)
I agree functionally these are equivalent. The only real differences are
(a) the byte[] interface doesn't encourage the use of a serializer (you
have to communicate the org standard via email)
(b) there is no easy way to reuse the serializer on the server side for
message format validation
(c) there is no way to allow plug in other validators in the client that
need to see the original object (without having these reserialize the
object to do their validation).

3. Agree. Part of the problem in the old producer that made it error prone
was that we have a default serializer that gives insane errors when used
with the wrong input types...which irrespective of what we do here we
should probably fix. There is value in having both a constructor which
takes the serializers and not. The value of allowing instantiation from
config is to make it easier to inherent the serializers from an environment
config that does the right thing.

4. Agreed. I addressed this a bit in the other email.

-Jay



On Thu, Dec 4, 2014 at 10:19 AM, Guozhang Wang <wa...@gmail.com> wrote:

> I would prefer making the kafka producer as is and wrap the object API on
> top rather than wiring the serializer configs into producers. Some
> thoughts:
>
> 1. For code sharing, I think it may only be effective for though simple
> functions such as string serialization, etc. For Avro / Shrift / PB, the
> serialization logic would be quite hard to share across organizations:
> imagine some people wants to use Avro 1.7 while others are still staying
> with 1.4 which are not API compatible, while some people use a schema
> registry server for clients to communicate while others compile the schemas
> into source code, etc. So I think in the end having those simple object
> serialization code into kafka.api package and letting applications write
> their own complicated serialization wrapper would be as beneficial as this
> approach.
>
> 2. For code simplicity I do not see a huge difference between a wired
> serializer, which will call serializer.encode() inside the producer, with a
> wrapper, which will call the same outside the producer, or a typed record,
> which will call record.encode() inside the producer.
>
> 3. For less error-proneness, people always mess with the config settings
> especially when they use hierarchical / nested wiring of configs, and such
> mistakes will only be detected on runtime but not compilation time. In the
> past we have seem a lot of such cases with the old producer APIs that
> wire-in the serializer class. If we move this to a SerDe interface, for
> example KafkaProducer<K, V>(KeySer<K>, ValueSer<V>) such errors will be
> detected at compilation.
>
> 4. For data type flexibility, the current approach bind one producer
> instance to a fixed record type. This may be OK in most cases as people
> usually only use a single data type but there are some cases where we would
> like to have a single producer to be able to send multiple typed messages,
> like control messages, or even with a single serialization like Avro we
> would sometimes want to have GenericaRecord and IndexedRecord for some
> specific types.
>
>
> Guozhang
>
> On Wed, Dec 3, 2014 at 2:54 PM, Jun Rao <ju...@confluent.io> wrote:
>
> > Jan, Jason,
> >
> > First, within an Kafka cluster, it's unlikely that each topic has a
> > different type serializer. Like Jason mentioned, Square standardizes on
> > protocol. Many other places such as LinkedIn standardize on Avro.
> >
> > Second, dealing with bytes only has limited use cases. Other than copying
> > bytes around, there isn't much else that one can do. Even for the case of
> > copying data from Kafka into HDFS, often you will need to (1) extract the
> > timestamp so that you can partition the data properly; (2) extract
> > individual fields if you want to put the data in a column-oriented
> storage
> > format. So, most interesting clients likely need to deal with objects
> > instead of bytes.
> >
> > Finally, the generic api doesn't prevent one from using just the bytes.
> The
> > additional overhead is just a method call, which the old clients are
> > already paying. Having both a raw bytes and a generic api is probably
> going
> > to confuse the users more.
> >
> > Thanks,
> >
> > Jun
> >
> >
> >
> > On Tue, Dec 2, 2014 at 6:50 PM, Jan Filipiak <Ja...@trivago.com>
> > wrote:
> >
> > > Hello Everyone,
> > >
> > > I would very much appreciate if someone could provide me a real world
> > > examplewhere it is more convenient to implement the serializers instead
> > of
> > > just making sure to provide bytearrays.
> > >
> > > The code we came up with explicitly avoids the serializer api. I think
> it
> > > is common understanding that if you want to transport data you need to
> > have
> > > it as a bytearray.
> > >
> > > If at all I personally would like to have a serializer interface that
> > > takes the same types as the producer
> > >
> > > public interface Serializer<K,V> extends Configurable {
> > >     public byte[] serializeKey(K data);
> > >     public byte[] serializeValue(V data);
> > >     public void close();
> > > }
> > >
> > > this would avoid long serialize implementations with branches like
> > > "switch(topic)" or "if(isKey)". Further serializer per topic makes more
> > > sense in my opinion. It feels natural to have a one to one relationship
> > > from types to topics or at least only a few partition per type. But as
> we
> > > inherit the type from the producer we would have to create many
> > producers.
> > > This would create additional unnecessary connections to the brokers.
> With
> > > the serializers we create a one type to all topics relationship and the
> > > only type that satisfies that is the bytearray or Object. Am I missing
> > > something here? As said in the beginning I would like to that usecase
> > that
> > > really benefits from using the serializers. I think in theory they
> sound
> > > great but they cause real practical issues that may lead users to wrong
> > > decisions.
> > >
> > > -1 for putting the serializers back in.
> > >
> > > Looking forward to replies that can show me the benefit of serializes
> and
> > > especially how the
> > > Type => topic relationship can be handled nicely.
> > >
> > > Best
> > > Jan
> > >
> > >
> > >
> > >
> > > On 25.11.2014 02:58, 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
> > >>
> > >>
> > >
> >
>
>
>
> --
> -- Guozhang
>

Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Guozhang Wang <wa...@gmail.com>.
I would prefer making the kafka producer as is and wrap the object API on
top rather than wiring the serializer configs into producers. Some thoughts:

1. For code sharing, I think it may only be effective for though simple
functions such as string serialization, etc. For Avro / Shrift / PB, the
serialization logic would be quite hard to share across organizations:
imagine some people wants to use Avro 1.7 while others are still staying
with 1.4 which are not API compatible, while some people use a schema
registry server for clients to communicate while others compile the schemas
into source code, etc. So I think in the end having those simple object
serialization code into kafka.api package and letting applications write
their own complicated serialization wrapper would be as beneficial as this
approach.

2. For code simplicity I do not see a huge difference between a wired
serializer, which will call serializer.encode() inside the producer, with a
wrapper, which will call the same outside the producer, or a typed record,
which will call record.encode() inside the producer.

3. For less error-proneness, people always mess with the config settings
especially when they use hierarchical / nested wiring of configs, and such
mistakes will only be detected on runtime but not compilation time. In the
past we have seem a lot of such cases with the old producer APIs that
wire-in the serializer class. If we move this to a SerDe interface, for
example KafkaProducer<K, V>(KeySer<K>, ValueSer<V>) such errors will be
detected at compilation.

4. For data type flexibility, the current approach bind one producer
instance to a fixed record type. This may be OK in most cases as people
usually only use a single data type but there are some cases where we would
like to have a single producer to be able to send multiple typed messages,
like control messages, or even with a single serialization like Avro we
would sometimes want to have GenericaRecord and IndexedRecord for some
specific types.


Guozhang

On Wed, Dec 3, 2014 at 2:54 PM, Jun Rao <ju...@confluent.io> wrote:

> Jan, Jason,
>
> First, within an Kafka cluster, it's unlikely that each topic has a
> different type serializer. Like Jason mentioned, Square standardizes on
> protocol. Many other places such as LinkedIn standardize on Avro.
>
> Second, dealing with bytes only has limited use cases. Other than copying
> bytes around, there isn't much else that one can do. Even for the case of
> copying data from Kafka into HDFS, often you will need to (1) extract the
> timestamp so that you can partition the data properly; (2) extract
> individual fields if you want to put the data in a column-oriented storage
> format. So, most interesting clients likely need to deal with objects
> instead of bytes.
>
> Finally, the generic api doesn't prevent one from using just the bytes. The
> additional overhead is just a method call, which the old clients are
> already paying. Having both a raw bytes and a generic api is probably going
> to confuse the users more.
>
> Thanks,
>
> Jun
>
>
>
> On Tue, Dec 2, 2014 at 6:50 PM, Jan Filipiak <Ja...@trivago.com>
> wrote:
>
> > Hello Everyone,
> >
> > I would very much appreciate if someone could provide me a real world
> > examplewhere it is more convenient to implement the serializers instead
> of
> > just making sure to provide bytearrays.
> >
> > The code we came up with explicitly avoids the serializer api. I think it
> > is common understanding that if you want to transport data you need to
> have
> > it as a bytearray.
> >
> > If at all I personally would like to have a serializer interface that
> > takes the same types as the producer
> >
> > public interface Serializer<K,V> extends Configurable {
> >     public byte[] serializeKey(K data);
> >     public byte[] serializeValue(V data);
> >     public void close();
> > }
> >
> > this would avoid long serialize implementations with branches like
> > "switch(topic)" or "if(isKey)". Further serializer per topic makes more
> > sense in my opinion. It feels natural to have a one to one relationship
> > from types to topics or at least only a few partition per type. But as we
> > inherit the type from the producer we would have to create many
> producers.
> > This would create additional unnecessary connections to the brokers. With
> > the serializers we create a one type to all topics relationship and the
> > only type that satisfies that is the bytearray or Object. Am I missing
> > something here? As said in the beginning I would like to that usecase
> that
> > really benefits from using the serializers. I think in theory they sound
> > great but they cause real practical issues that may lead users to wrong
> > decisions.
> >
> > -1 for putting the serializers back in.
> >
> > Looking forward to replies that can show me the benefit of serializes and
> > especially how the
> > Type => topic relationship can be handled nicely.
> >
> > Best
> > Jan
> >
> >
> >
> >
> > On 25.11.2014 02:58, 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
> >>
> >>
> >
>



-- 
-- Guozhang

Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Jun Rao <ju...@confluent.io>.
Jan, Jason,

First, within an Kafka cluster, it's unlikely that each topic has a
different type serializer. Like Jason mentioned, Square standardizes on
protocol. Many other places such as LinkedIn standardize on Avro.

Second, dealing with bytes only has limited use cases. Other than copying
bytes around, there isn't much else that one can do. Even for the case of
copying data from Kafka into HDFS, often you will need to (1) extract the
timestamp so that you can partition the data properly; (2) extract
individual fields if you want to put the data in a column-oriented storage
format. So, most interesting clients likely need to deal with objects
instead of bytes.

Finally, the generic api doesn't prevent one from using just the bytes. The
additional overhead is just a method call, which the old clients are
already paying. Having both a raw bytes and a generic api is probably going
to confuse the users more.

Thanks,

Jun



On Tue, Dec 2, 2014 at 6:50 PM, Jan Filipiak <Ja...@trivago.com>
wrote:

> Hello Everyone,
>
> I would very much appreciate if someone could provide me a real world
> examplewhere it is more convenient to implement the serializers instead of
> just making sure to provide bytearrays.
>
> The code we came up with explicitly avoids the serializer api. I think it
> is common understanding that if you want to transport data you need to have
> it as a bytearray.
>
> If at all I personally would like to have a serializer interface that
> takes the same types as the producer
>
> public interface Serializer<K,V> extends Configurable {
>     public byte[] serializeKey(K data);
>     public byte[] serializeValue(V data);
>     public void close();
> }
>
> this would avoid long serialize implementations with branches like
> "switch(topic)" or "if(isKey)". Further serializer per topic makes more
> sense in my opinion. It feels natural to have a one to one relationship
> from types to topics or at least only a few partition per type. But as we
> inherit the type from the producer we would have to create many producers.
> This would create additional unnecessary connections to the brokers. With
> the serializers we create a one type to all topics relationship and the
> only type that satisfies that is the bytearray or Object. Am I missing
> something here? As said in the beginning I would like to that usecase that
> really benefits from using the serializers. I think in theory they sound
> great but they cause real practical issues that may lead users to wrong
> decisions.
>
> -1 for putting the serializers back in.
>
> Looking forward to replies that can show me the benefit of serializes and
> especially how the
> Type => topic relationship can be handled nicely.
>
> Best
> Jan
>
>
>
>
> On 25.11.2014 02:58, 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
>>
>>
>

Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Jason Rosenberg <jb...@squareup.com>.
In our case, we use protocol buffers for all messages, and these have
simple serialization/deserialization builtin to the protobuf libraries
(e.g. MyProtobufMessage.toByteArray()).  Also, we often produce/consume
messages without conversion to/from protobuf Objects (e.g. in cases where
we are just forwarding messages on to other topics, or if we are consuming
directly to a binary blob store like hdfs).  There's a huge efficiency in
not over synthesizing new Objects.

Thus, it's nice to only deal with bytes directly in all messages, and keep
things simple.  Having the overhead of having to dummy in a default,
generically parameterized, no-op serializer (and the overhead of having
that extra no-op method call, seems unnecessary).

I'd suggest that maybe it could work seamlessly either way (which it
probably does now, for the case where no serializer is provided, but not
sure if it efficiently will elide the call to the no-op serializer after
JIT?)....Alternatively, I do think it's important to preserve the
efficiency of sending raw bytes directly, so if necessary, maybe expose
both apis (one which explicitly bypasses any serialization).

Finally, I've wondered in the past about enabling some sort of streaming
serialization, whereby you hook up a producer to a long living stream
class, which could integrate compression in line, and allow more control of
the pipeline.  The stream would implement an iterator to get the next
serialized message, etc.  For me, something like this might be a reason to
have a serialization/deserialization abstraction built into the
producer/consumer api's.

But if I have a vote, I'd be in favor of keeping the api simple and have it
take bytes directly.

Jason

On Tue, Dec 2, 2014 at 9:50 PM, Jan Filipiak <Ja...@trivago.com>
wrote:

> Hello Everyone,
>
> I would very much appreciate if someone could provide me a real world
> examplewhere it is more convenient to implement the serializers instead of
> just making sure to provide bytearrays.
>
> The code we came up with explicitly avoids the serializer api. I think it
> is common understanding that if you want to transport data you need to have
> it as a bytearray.
>
> If at all I personally would like to have a serializer interface that
> takes the same types as the producer
>
> public interface Serializer<K,V> extends Configurable {
>     public byte[] serializeKey(K data);
>     public byte[] serializeValue(V data);
>     public void close();
> }
>
> this would avoid long serialize implementations with branches like
> "switch(topic)" or "if(isKey)". Further serializer per topic makes more
> sense in my opinion. It feels natural to have a one to one relationship
> from types to topics or at least only a few partition per type. But as we
> inherit the type from the producer we would have to create many producers.
> This would create additional unnecessary connections to the brokers. With
> the serializers we create a one type to all topics relationship and the
> only type that satisfies that is the bytearray or Object. Am I missing
> something here? As said in the beginning I would like to that usecase that
> really benefits from using the serializers. I think in theory they sound
> great but they cause real practical issues that may lead users to wrong
> decisions.
>
> -1 for putting the serializers back in.
>
> Looking forward to replies that can show me the benefit of serializes and
> especially how the
> Type => topic relationship can be handled nicely.
>
> Best
> Jan
>
>
>
>
> On 25.11.2014 02:58, 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
>>
>>
>

Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Philippe Laflamme <pl...@hopper.com>.
Sorry for adding noise, but I think Jan has a very good point: applications
shouldn't be forced to create multiple producers simply to wire-in the
proper Serializer. It's an artificial restriction that wastes resources.

It's a common thing for us to create a single producer and slap different
"views" on top for each topic it writes to.

Furthermore, requiring that a producer specify both a K and a V type is
clumsy for topics that don't have a key. The signature would look like
KafkaProducer<Void, MyObject> where the Void type is unnecessary noise that
also pollutes other types like ProducerRecord.

The less opinions Kafka has about application-level concerns, the better.

Cheers,
Philippe


On Tue, Dec 2, 2014 at 9:50 PM, Jan Filipiak <Ja...@trivago.com>
wrote:

> Hello Everyone,
>
> I would very much appreciate if someone could provide me a real world
> examplewhere it is more convenient to implement the serializers instead of
> just making sure to provide bytearrays.
>
> The code we came up with explicitly avoids the serializer api. I think it
> is common understanding that if you want to transport data you need to have
> it as a bytearray.
>
> If at all I personally would like to have a serializer interface that
> takes the same types as the producer
>
> public interface Serializer<K,V> extends Configurable {
>     public byte[] serializeKey(K data);
>     public byte[] serializeValue(V data);
>     public void close();
> }
>
> this would avoid long serialize implementations with branches like
> "switch(topic)" or "if(isKey)". Further serializer per topic makes more
> sense in my opinion. It feels natural to have a one to one relationship
> from types to topics or at least only a few partition per type. But as we
> inherit the type from the producer we would have to create many producers.
> This would create additional unnecessary connections to the brokers. With
> the serializers we create a one type to all topics relationship and the
> only type that satisfies that is the bytearray or Object. Am I missing
> something here? As said in the beginning I would like to that usecase that
> really benefits from using the serializers. I think in theory they sound
> great but they cause real practical issues that may lead users to wrong
> decisions.
>
> -1 for putting the serializers back in.
>
> Looking forward to replies that can show me the benefit of serializes and
> especially how the
> Type => topic relationship can be handled nicely.
>
> Best
> Jan
>
>
>
>
> On 25.11.2014 02:58, 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
>>
>>
>

Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Jan Filipiak <Ja...@trivago.com>.
Hello Everyone,

I would very much appreciate if someone could provide me a real world 
examplewhere it is more convenient to implement the serializers instead 
of just making sure to provide bytearrays.

The code we came up with explicitly avoids the serializer api. I think 
it is common understanding that if you want to transport data you need 
to have it as a bytearray.

If at all I personally would like to have a serializer interface that 
takes the same types as the producer

public interface Serializer<K,V> extends Configurable {
     public byte[] serializeKey(K data);
     public byte[] serializeValue(V data);
     public void close();
}

this would avoid long serialize implementations with branches like 
"switch(topic)" or "if(isKey)". Further serializer per topic makes more 
sense in my opinion. It feels natural to have a one to one relationship 
from types to topics or at least only a few partition per type. But as 
we inherit the type from the producer we would have to create many 
producers. This would create additional unnecessary connections to the 
brokers. With the serializers we create a one type to all topics 
relationship and the only type that satisfies that is the bytearray or 
Object. Am I missing something here? As said in the beginning I would 
like to that usecase that really benefits from using the serializers. I 
think in theory they sound great but they cause real practical issues 
that may lead users to wrong decisions.

-1 for putting the serializers back in.

Looking forward to replies that can show me the benefit of serializes 
and especially how the
Type => topic relationship can be handled nicely.

Best
Jan



On 25.11.2014 02:58, 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
>


Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Roger Hoover <ro...@gmail.com>.
"It also makes it possible to do validation on the server
side or make other tools that inspect or display messages (e.g. the various
command line tools) and do this in an easily pluggable way across tools."

I agree that it's valuable to have a standard way to plugin serialization
across many tools, especially for producers.  For example, the Kafka
producer might get wrapped by JRuby and exposed as a Logstash plugin
<https://github.com/joekiller/logstash-kafka>.  With a standard method for
plugging in serdes, one can reuse a serde with any tool that wraps the
standard producer API.  This won't be possible if we rely on custom
wrappers.

On Tue, Dec 2, 2014 at 1:49 PM, Jay Kreps <ja...@confluent.io> wrote:

> Yeah totally, far from preventing it, making it easy to specify/encourage a
> custom serializer across your org is exactly the kind of thing I was hoping
> to make work well. If there is a config that gives the serializer you can
> just default this to what you want people to use as some kind of
> environment default or just tell people to set the property. A person who
> wants to ignore this can, of course, but the easy thing to do will be to
> use an off-the-shelf serialization method.
>
> If you really want to enforce it, having an interface for serialization
> would also let us optionally check this on the server side (e.g. if you
> specify a serializer on the server side we validate that messages are in
> this format).
>
> If the api is just bytes of course you can make a serializer you want
> people to use, and you can send around an email asking people to use it,
> but the easy thing to do will remain "my string".getBytes() or whatever and
> lots of people will do that instead.
>
> Here the advantage of config is that (assuming your config system allows
> it) you should be able to have some kind of global environment default for
> these settings and easily grep across applications to determine what is in
> use.
>
> I think in all of this there is no hard and fast technical difference
> between these approaches, i.e. there is nothing you can do one way that is
> impossible the other way.
>
> But I do think that having a nice way to plug in serialization makes it
> much more straight-forward and intuitive to package these things up inside
> an organization. It also makes it possible to do validation on the server
> side or make other tools that inspect or display messages (e.g. the various
> command line tools) and do this in an easily pluggable way across tools.
>
> The concern I was expressing was that in the absence of support for
> serialization, what everyone will do is just make a wrapper api that
> handles these things (since no one can actually use the producer without
> serialization, and you will want to encourage use of the proper thing). The
> problem I have with wrapper apis is that they defeat common documentation
> and tend to made without as much thought as the primary api.
>
> The advantage of having serialization handled internally is that all you
> need to do is know the right config for your organization and any example
> usage remains the same.
>
> Hopefully that helps explain the rationale a little more.
>
> -Jay
>
> On Tue, Dec 2, 2014 at 11:53 AM, Joel Koshy <jj...@gmail.com> wrote:
>
> > Thanks for the follow-up Jay.  I still don't quite see the issue here
> > but maybe I just need to process this a bit more. To me "packaging up
> > the best practice and plug it in" seems to be to expose a simple
> > low-level API and give people the option to plug in a (possibly
> > shared) standard serializer in their application configs (or a custom
> > one if they choose) and invoke that from code. The additional
> > serialization call is a minor drawback but a very clear and easily
> > understood step that can be documented.  The serializer can obviously
> > also do other things such as schema registration. I'm actually not (or
> > at least I think I'm not) influenced very much by LinkedIn's wrapper.
> > It's just that I think it is reasonable to expect that in practice
> > most organizations (big and small) tend to have at least some specific
> > organization-specific detail that warrants a custom serializer anyway;
> > and it's going to be easier to override a serializer than an entire
> > producer API.
> >
> > Joel
> >
> > On Tue, Dec 02, 2014 at 11:09:55AM -0800, Jay Kreps wrote:
> > > Hey Joel, you are right, we discussed this, but I think we didn't think
> > > about it as deeply as we should have. I think our take was strongly
> > shaped
> > > by having a wrapper api at LinkedIn that DOES do the serialization
> > > transparently so I think you are thinking of the producer as just an
> > > implementation detail of that wrapper. Imagine a world where every
> > > application at LinkedIn had to figure that part out themselves. That
> is,
> > > imagine that what you guys supported was just the raw producer api and
> > that
> > > that just handled bytes. I think in that world the types of data you
> > would
> > > see would be totally funky and standardizing correct usage would be a
> > > massive pain.
> > >
> > > Conversely, you could imagine advocating the LinkedIn approach where
> you
> > > just say, well, every org should wrap up the clients in a way that does
> > > things like serialization and other data checks. The problem with that
> is
> > > that it (1) it is kind of redundant work and it is likely that the
> > wrapper
> > > will goof some nuances of the apis, and (2) it makes documentation and
> > code
> > > sharing really hard. That is, rather than being able to go to a central
> > > place and read how to use the producer, LinkedIn people need to
> document
> > > the LinkedIn producer wrapper, and users at LinkedIn need to read about
> > > LinkedIn's wrapper for the producer to understand how to use it. Now
> > > imagine this multiplied over every user.
> > >
> > > The idea is that since everyone needs to do this we should just make it
> > > easy to package up the best practice and plug it in. That way the
> > > "contract" your application programs to is just the normal producer
> api.
> > >
> > > -Jay
> > >
> > > On Tue, Dec 2, 2014 at 10:06 AM, Joel Koshy <jj...@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 <jj...@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
> > > > > >
> > > > > >
> > > >
> > > >
> >
> >
>

Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Roger Hoover <ro...@gmail.com>.
"It also makes it possible to do validation on the server
side or make other tools that inspect or display messages (e.g. the various
command line tools) and do this in an easily pluggable way across tools."

I agree that it's valuable to have a standard way to plugin serialization
across many tools, especially for producers.  For example, the Kafka
producer might get wrapped by JRuby and exposed as a Logstash plugin
<https://github.com/joekiller/logstash-kafka>.  With a standard method for
plugging in serdes, one can reuse a serde with any tool that wraps the
standard producer API.  This won't be possible if we rely on custom
wrappers.

On Tue, Dec 2, 2014 at 1:49 PM, Jay Kreps <ja...@confluent.io> wrote:

> Yeah totally, far from preventing it, making it easy to specify/encourage a
> custom serializer across your org is exactly the kind of thing I was hoping
> to make work well. If there is a config that gives the serializer you can
> just default this to what you want people to use as some kind of
> environment default or just tell people to set the property. A person who
> wants to ignore this can, of course, but the easy thing to do will be to
> use an off-the-shelf serialization method.
>
> If you really want to enforce it, having an interface for serialization
> would also let us optionally check this on the server side (e.g. if you
> specify a serializer on the server side we validate that messages are in
> this format).
>
> If the api is just bytes of course you can make a serializer you want
> people to use, and you can send around an email asking people to use it,
> but the easy thing to do will remain "my string".getBytes() or whatever and
> lots of people will do that instead.
>
> Here the advantage of config is that (assuming your config system allows
> it) you should be able to have some kind of global environment default for
> these settings and easily grep across applications to determine what is in
> use.
>
> I think in all of this there is no hard and fast technical difference
> between these approaches, i.e. there is nothing you can do one way that is
> impossible the other way.
>
> But I do think that having a nice way to plug in serialization makes it
> much more straight-forward and intuitive to package these things up inside
> an organization. It also makes it possible to do validation on the server
> side or make other tools that inspect or display messages (e.g. the various
> command line tools) and do this in an easily pluggable way across tools.
>
> The concern I was expressing was that in the absence of support for
> serialization, what everyone will do is just make a wrapper api that
> handles these things (since no one can actually use the producer without
> serialization, and you will want to encourage use of the proper thing). The
> problem I have with wrapper apis is that they defeat common documentation
> and tend to made without as much thought as the primary api.
>
> The advantage of having serialization handled internally is that all you
> need to do is know the right config for your organization and any example
> usage remains the same.
>
> Hopefully that helps explain the rationale a little more.
>
> -Jay
>
> On Tue, Dec 2, 2014 at 11:53 AM, Joel Koshy <jj...@gmail.com> wrote:
>
> > Thanks for the follow-up Jay.  I still don't quite see the issue here
> > but maybe I just need to process this a bit more. To me "packaging up
> > the best practice and plug it in" seems to be to expose a simple
> > low-level API and give people the option to plug in a (possibly
> > shared) standard serializer in their application configs (or a custom
> > one if they choose) and invoke that from code. The additional
> > serialization call is a minor drawback but a very clear and easily
> > understood step that can be documented.  The serializer can obviously
> > also do other things such as schema registration. I'm actually not (or
> > at least I think I'm not) influenced very much by LinkedIn's wrapper.
> > It's just that I think it is reasonable to expect that in practice
> > most organizations (big and small) tend to have at least some specific
> > organization-specific detail that warrants a custom serializer anyway;
> > and it's going to be easier to override a serializer than an entire
> > producer API.
> >
> > Joel
> >
> > On Tue, Dec 02, 2014 at 11:09:55AM -0800, Jay Kreps wrote:
> > > Hey Joel, you are right, we discussed this, but I think we didn't think
> > > about it as deeply as we should have. I think our take was strongly
> > shaped
> > > by having a wrapper api at LinkedIn that DOES do the serialization
> > > transparently so I think you are thinking of the producer as just an
> > > implementation detail of that wrapper. Imagine a world where every
> > > application at LinkedIn had to figure that part out themselves. That
> is,
> > > imagine that what you guys supported was just the raw producer api and
> > that
> > > that just handled bytes. I think in that world the types of data you
> > would
> > > see would be totally funky and standardizing correct usage would be a
> > > massive pain.
> > >
> > > Conversely, you could imagine advocating the LinkedIn approach where
> you
> > > just say, well, every org should wrap up the clients in a way that does
> > > things like serialization and other data checks. The problem with that
> is
> > > that it (1) it is kind of redundant work and it is likely that the
> > wrapper
> > > will goof some nuances of the apis, and (2) it makes documentation and
> > code
> > > sharing really hard. That is, rather than being able to go to a central
> > > place and read how to use the producer, LinkedIn people need to
> document
> > > the LinkedIn producer wrapper, and users at LinkedIn need to read about
> > > LinkedIn's wrapper for the producer to understand how to use it. Now
> > > imagine this multiplied over every user.
> > >
> > > The idea is that since everyone needs to do this we should just make it
> > > easy to package up the best practice and plug it in. That way the
> > > "contract" your application programs to is just the normal producer
> api.
> > >
> > > -Jay
> > >
> > > On Tue, Dec 2, 2014 at 10:06 AM, Joel Koshy <jj...@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 <jj...@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
> > > > > >
> > > > > >
> > > >
> > > >
> >
> >
>

Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Jay Kreps <ja...@confluent.io>.
Yeah totally, far from preventing it, making it easy to specify/encourage a
custom serializer across your org is exactly the kind of thing I was hoping
to make work well. If there is a config that gives the serializer you can
just default this to what you want people to use as some kind of
environment default or just tell people to set the property. A person who
wants to ignore this can, of course, but the easy thing to do will be to
use an off-the-shelf serialization method.

If you really want to enforce it, having an interface for serialization
would also let us optionally check this on the server side (e.g. if you
specify a serializer on the server side we validate that messages are in
this format).

If the api is just bytes of course you can make a serializer you want
people to use, and you can send around an email asking people to use it,
but the easy thing to do will remain "my string".getBytes() or whatever and
lots of people will do that instead.

Here the advantage of config is that (assuming your config system allows
it) you should be able to have some kind of global environment default for
these settings and easily grep across applications to determine what is in
use.

I think in all of this there is no hard and fast technical difference
between these approaches, i.e. there is nothing you can do one way that is
impossible the other way.

But I do think that having a nice way to plug in serialization makes it
much more straight-forward and intuitive to package these things up inside
an organization. It also makes it possible to do validation on the server
side or make other tools that inspect or display messages (e.g. the various
command line tools) and do this in an easily pluggable way across tools.

The concern I was expressing was that in the absence of support for
serialization, what everyone will do is just make a wrapper api that
handles these things (since no one can actually use the producer without
serialization, and you will want to encourage use of the proper thing). The
problem I have with wrapper apis is that they defeat common documentation
and tend to made without as much thought as the primary api.

The advantage of having serialization handled internally is that all you
need to do is know the right config for your organization and any example
usage remains the same.

Hopefully that helps explain the rationale a little more.

-Jay

On Tue, Dec 2, 2014 at 11:53 AM, Joel Koshy <jj...@gmail.com> wrote:

> Thanks for the follow-up Jay.  I still don't quite see the issue here
> but maybe I just need to process this a bit more. To me "packaging up
> the best practice and plug it in" seems to be to expose a simple
> low-level API and give people the option to plug in a (possibly
> shared) standard serializer in their application configs (or a custom
> one if they choose) and invoke that from code. The additional
> serialization call is a minor drawback but a very clear and easily
> understood step that can be documented.  The serializer can obviously
> also do other things such as schema registration. I'm actually not (or
> at least I think I'm not) influenced very much by LinkedIn's wrapper.
> It's just that I think it is reasonable to expect that in practice
> most organizations (big and small) tend to have at least some specific
> organization-specific detail that warrants a custom serializer anyway;
> and it's going to be easier to override a serializer than an entire
> producer API.
>
> Joel
>
> On Tue, Dec 02, 2014 at 11:09:55AM -0800, Jay Kreps wrote:
> > Hey Joel, you are right, we discussed this, but I think we didn't think
> > about it as deeply as we should have. I think our take was strongly
> shaped
> > by having a wrapper api at LinkedIn that DOES do the serialization
> > transparently so I think you are thinking of the producer as just an
> > implementation detail of that wrapper. Imagine a world where every
> > application at LinkedIn had to figure that part out themselves. That is,
> > imagine that what you guys supported was just the raw producer api and
> that
> > that just handled bytes. I think in that world the types of data you
> would
> > see would be totally funky and standardizing correct usage would be a
> > massive pain.
> >
> > Conversely, you could imagine advocating the LinkedIn approach where you
> > just say, well, every org should wrap up the clients in a way that does
> > things like serialization and other data checks. The problem with that is
> > that it (1) it is kind of redundant work and it is likely that the
> wrapper
> > will goof some nuances of the apis, and (2) it makes documentation and
> code
> > sharing really hard. That is, rather than being able to go to a central
> > place and read how to use the producer, LinkedIn people need to document
> > the LinkedIn producer wrapper, and users at LinkedIn need to read about
> > LinkedIn's wrapper for the producer to understand how to use it. Now
> > imagine this multiplied over every user.
> >
> > The idea is that since everyone needs to do this we should just make it
> > easy to package up the best practice and plug it in. That way the
> > "contract" your application programs to is just the normal producer api.
> >
> > -Jay
> >
> > On Tue, Dec 2, 2014 at 10:06 AM, Joel Koshy <jj...@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 <jj...@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
> > > > >
> > > > >
> > >
> > >
>
>

Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Jay Kreps <ja...@confluent.io>.
Yeah totally, far from preventing it, making it easy to specify/encourage a
custom serializer across your org is exactly the kind of thing I was hoping
to make work well. If there is a config that gives the serializer you can
just default this to what you want people to use as some kind of
environment default or just tell people to set the property. A person who
wants to ignore this can, of course, but the easy thing to do will be to
use an off-the-shelf serialization method.

If you really want to enforce it, having an interface for serialization
would also let us optionally check this on the server side (e.g. if you
specify a serializer on the server side we validate that messages are in
this format).

If the api is just bytes of course you can make a serializer you want
people to use, and you can send around an email asking people to use it,
but the easy thing to do will remain "my string".getBytes() or whatever and
lots of people will do that instead.

Here the advantage of config is that (assuming your config system allows
it) you should be able to have some kind of global environment default for
these settings and easily grep across applications to determine what is in
use.

I think in all of this there is no hard and fast technical difference
between these approaches, i.e. there is nothing you can do one way that is
impossible the other way.

But I do think that having a nice way to plug in serialization makes it
much more straight-forward and intuitive to package these things up inside
an organization. It also makes it possible to do validation on the server
side or make other tools that inspect or display messages (e.g. the various
command line tools) and do this in an easily pluggable way across tools.

The concern I was expressing was that in the absence of support for
serialization, what everyone will do is just make a wrapper api that
handles these things (since no one can actually use the producer without
serialization, and you will want to encourage use of the proper thing). The
problem I have with wrapper apis is that they defeat common documentation
and tend to made without as much thought as the primary api.

The advantage of having serialization handled internally is that all you
need to do is know the right config for your organization and any example
usage remains the same.

Hopefully that helps explain the rationale a little more.

-Jay

On Tue, Dec 2, 2014 at 11:53 AM, Joel Koshy <jj...@gmail.com> wrote:

> Thanks for the follow-up Jay.  I still don't quite see the issue here
> but maybe I just need to process this a bit more. To me "packaging up
> the best practice and plug it in" seems to be to expose a simple
> low-level API and give people the option to plug in a (possibly
> shared) standard serializer in their application configs (or a custom
> one if they choose) and invoke that from code. The additional
> serialization call is a minor drawback but a very clear and easily
> understood step that can be documented.  The serializer can obviously
> also do other things such as schema registration. I'm actually not (or
> at least I think I'm not) influenced very much by LinkedIn's wrapper.
> It's just that I think it is reasonable to expect that in practice
> most organizations (big and small) tend to have at least some specific
> organization-specific detail that warrants a custom serializer anyway;
> and it's going to be easier to override a serializer than an entire
> producer API.
>
> Joel
>
> On Tue, Dec 02, 2014 at 11:09:55AM -0800, Jay Kreps wrote:
> > Hey Joel, you are right, we discussed this, but I think we didn't think
> > about it as deeply as we should have. I think our take was strongly
> shaped
> > by having a wrapper api at LinkedIn that DOES do the serialization
> > transparently so I think you are thinking of the producer as just an
> > implementation detail of that wrapper. Imagine a world where every
> > application at LinkedIn had to figure that part out themselves. That is,
> > imagine that what you guys supported was just the raw producer api and
> that
> > that just handled bytes. I think in that world the types of data you
> would
> > see would be totally funky and standardizing correct usage would be a
> > massive pain.
> >
> > Conversely, you could imagine advocating the LinkedIn approach where you
> > just say, well, every org should wrap up the clients in a way that does
> > things like serialization and other data checks. The problem with that is
> > that it (1) it is kind of redundant work and it is likely that the
> wrapper
> > will goof some nuances of the apis, and (2) it makes documentation and
> code
> > sharing really hard. That is, rather than being able to go to a central
> > place and read how to use the producer, LinkedIn people need to document
> > the LinkedIn producer wrapper, and users at LinkedIn need to read about
> > LinkedIn's wrapper for the producer to understand how to use it. Now
> > imagine this multiplied over every user.
> >
> > The idea is that since everyone needs to do this we should just make it
> > easy to package up the best practice and plug it in. That way the
> > "contract" your application programs to is just the normal producer api.
> >
> > -Jay
> >
> > On Tue, Dec 2, 2014 at 10:06 AM, Joel Koshy <jj...@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 <jj...@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
> > > > >
> > > > >
> > >
> > >
>
>

Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Joel Koshy <jj...@gmail.com>.
Thanks for the follow-up Jay.  I still don't quite see the issue here
but maybe I just need to process this a bit more. To me "packaging up
the best practice and plug it in" seems to be to expose a simple
low-level API and give people the option to plug in a (possibly
shared) standard serializer in their application configs (or a custom
one if they choose) and invoke that from code. The additional
serialization call is a minor drawback but a very clear and easily
understood step that can be documented.  The serializer can obviously
also do other things such as schema registration. I'm actually not (or
at least I think I'm not) influenced very much by LinkedIn's wrapper.
It's just that I think it is reasonable to expect that in practice
most organizations (big and small) tend to have at least some specific
organization-specific detail that warrants a custom serializer anyway;
and it's going to be easier to override a serializer than an entire
producer API.

Joel

On Tue, Dec 02, 2014 at 11:09:55AM -0800, Jay Kreps wrote:
> Hey Joel, you are right, we discussed this, but I think we didn't think
> about it as deeply as we should have. I think our take was strongly shaped
> by having a wrapper api at LinkedIn that DOES do the serialization
> transparently so I think you are thinking of the producer as just an
> implementation detail of that wrapper. Imagine a world where every
> application at LinkedIn had to figure that part out themselves. That is,
> imagine that what you guys supported was just the raw producer api and that
> that just handled bytes. I think in that world the types of data you would
> see would be totally funky and standardizing correct usage would be a
> massive pain.
> 
> Conversely, you could imagine advocating the LinkedIn approach where you
> just say, well, every org should wrap up the clients in a way that does
> things like serialization and other data checks. The problem with that is
> that it (1) it is kind of redundant work and it is likely that the wrapper
> will goof some nuances of the apis, and (2) it makes documentation and code
> sharing really hard. That is, rather than being able to go to a central
> place and read how to use the producer, LinkedIn people need to document
> the LinkedIn producer wrapper, and users at LinkedIn need to read about
> LinkedIn's wrapper for the producer to understand how to use it. Now
> imagine this multiplied over every user.
> 
> The idea is that since everyone needs to do this we should just make it
> easy to package up the best practice and plug it in. That way the
> "contract" your application programs to is just the normal producer api.
> 
> -Jay
> 
> On Tue, Dec 2, 2014 at 10:06 AM, Joel Koshy <jj...@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 <jj...@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
> > > >
> > > >
> >
> >


RE: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Thunder Stumpges <ts...@ntent.com>.
I'm not sure I agree with this. I feel that the need to have a consistent, well documented, shared serialization approach at the organization level is important no matter what. How you structure the API doesn't change that or make it any easier or "automatic" than before. It is still possible for users on different projects to "plug in" the wrong serializer or to "be totally funky". In order to make this consistent and completely encapsulated from users, a company would *still* need to write a shim layer that configures the correct serializer in a consistent way, and *that* still needs to be documented and understood.

Regards,
Thunder

-----Original Message-----
From: Jay Kreps [mailto:jay@confluent.io] 
Sent: Tuesday, December 02, 2014 11:10 AM
To: dev@kafka.apache.org
Cc: users@kafka.apache.org
Subject: Re: [DISCUSSION] adding the serializer api back to the new java producer

Hey Joel, you are right, we discussed this, but I think we didn't think about it as deeply as we should have. I think our take was strongly shaped by having a wrapper api at LinkedIn that DOES do the serialization transparently so I think you are thinking of the producer as just an implementation detail of that wrapper. Imagine a world where every application at LinkedIn had to figure that part out themselves. That is, imagine that what you guys supported was just the raw producer api and that that just handled bytes. I think in that world the types of data you would see would be totally funky and standardizing correct usage would be a massive pain.

Conversely, you could imagine advocating the LinkedIn approach where you just say, well, every org should wrap up the clients in a way that does things like serialization and other data checks. The problem with that is that it (1) it is kind of redundant work and it is likely that the wrapper will goof some nuances of the apis, and (2) it makes documentation and code sharing really hard. That is, rather than being able to go to a central place and read how to use the producer, LinkedIn people need to document the LinkedIn producer wrapper, and users at LinkedIn need to read about LinkedIn's wrapper for the producer to understand how to use it. Now imagine this multiplied over every user.

The idea is that since everyone needs to do this we should just make it easy to package up the best practice and plug it in. That way the "contract" your application programs to is just the normal producer api.

-Jay

On Tue, Dec 2, 2014 at 10:06 AM, Joel Koshy <jj...@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 <jj...@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
> > >
> > >
>
>

RE: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Thunder Stumpges <ts...@ntent.com>.
I'm not sure I agree with this. I feel that the need to have a consistent, well documented, shared serialization approach at the organization level is important no matter what. How you structure the API doesn't change that or make it any easier or "automatic" than before. It is still possible for users on different projects to "plug in" the wrong serializer or to "be totally funky". In order to make this consistent and completely encapsulated from users, a company would *still* need to write a shim layer that configures the correct serializer in a consistent way, and *that* still needs to be documented and understood.

Regards,
Thunder

-----Original Message-----
From: Jay Kreps [mailto:jay@confluent.io] 
Sent: Tuesday, December 02, 2014 11:10 AM
To: dev@kafka.apache.org
Cc: users@kafka.apache.org
Subject: Re: [DISCUSSION] adding the serializer api back to the new java producer

Hey Joel, you are right, we discussed this, but I think we didn't think about it as deeply as we should have. I think our take was strongly shaped by having a wrapper api at LinkedIn that DOES do the serialization transparently so I think you are thinking of the producer as just an implementation detail of that wrapper. Imagine a world where every application at LinkedIn had to figure that part out themselves. That is, imagine that what you guys supported was just the raw producer api and that that just handled bytes. I think in that world the types of data you would see would be totally funky and standardizing correct usage would be a massive pain.

Conversely, you could imagine advocating the LinkedIn approach where you just say, well, every org should wrap up the clients in a way that does things like serialization and other data checks. The problem with that is that it (1) it is kind of redundant work and it is likely that the wrapper will goof some nuances of the apis, and (2) it makes documentation and code sharing really hard. That is, rather than being able to go to a central place and read how to use the producer, LinkedIn people need to document the LinkedIn producer wrapper, and users at LinkedIn need to read about LinkedIn's wrapper for the producer to understand how to use it. Now imagine this multiplied over every user.

The idea is that since everyone needs to do this we should just make it easy to package up the best practice and plug it in. That way the "contract" your application programs to is just the normal producer api.

-Jay

On Tue, Dec 2, 2014 at 10:06 AM, Joel Koshy <jj...@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 <jj...@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
> > >
> > >
>
>

Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Joel Koshy <jj...@gmail.com>.
Thanks for the follow-up Jay.  I still don't quite see the issue here
but maybe I just need to process this a bit more. To me "packaging up
the best practice and plug it in" seems to be to expose a simple
low-level API and give people the option to plug in a (possibly
shared) standard serializer in their application configs (or a custom
one if they choose) and invoke that from code. The additional
serialization call is a minor drawback but a very clear and easily
understood step that can be documented.  The serializer can obviously
also do other things such as schema registration. I'm actually not (or
at least I think I'm not) influenced very much by LinkedIn's wrapper.
It's just that I think it is reasonable to expect that in practice
most organizations (big and small) tend to have at least some specific
organization-specific detail that warrants a custom serializer anyway;
and it's going to be easier to override a serializer than an entire
producer API.

Joel

On Tue, Dec 02, 2014 at 11:09:55AM -0800, Jay Kreps wrote:
> Hey Joel, you are right, we discussed this, but I think we didn't think
> about it as deeply as we should have. I think our take was strongly shaped
> by having a wrapper api at LinkedIn that DOES do the serialization
> transparently so I think you are thinking of the producer as just an
> implementation detail of that wrapper. Imagine a world where every
> application at LinkedIn had to figure that part out themselves. That is,
> imagine that what you guys supported was just the raw producer api and that
> that just handled bytes. I think in that world the types of data you would
> see would be totally funky and standardizing correct usage would be a
> massive pain.
> 
> Conversely, you could imagine advocating the LinkedIn approach where you
> just say, well, every org should wrap up the clients in a way that does
> things like serialization and other data checks. The problem with that is
> that it (1) it is kind of redundant work and it is likely that the wrapper
> will goof some nuances of the apis, and (2) it makes documentation and code
> sharing really hard. That is, rather than being able to go to a central
> place and read how to use the producer, LinkedIn people need to document
> the LinkedIn producer wrapper, and users at LinkedIn need to read about
> LinkedIn's wrapper for the producer to understand how to use it. Now
> imagine this multiplied over every user.
> 
> The idea is that since everyone needs to do this we should just make it
> easy to package up the best practice and plug it in. That way the
> "contract" your application programs to is just the normal producer api.
> 
> -Jay
> 
> On Tue, Dec 2, 2014 at 10:06 AM, Joel Koshy <jj...@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 <jj...@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
> > > >
> > > >
> >
> >


Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Jay Kreps <ja...@confluent.io>.
Hey Joel, you are right, we discussed this, but I think we didn't think
about it as deeply as we should have. I think our take was strongly shaped
by having a wrapper api at LinkedIn that DOES do the serialization
transparently so I think you are thinking of the producer as just an
implementation detail of that wrapper. Imagine a world where every
application at LinkedIn had to figure that part out themselves. That is,
imagine that what you guys supported was just the raw producer api and that
that just handled bytes. I think in that world the types of data you would
see would be totally funky and standardizing correct usage would be a
massive pain.

Conversely, you could imagine advocating the LinkedIn approach where you
just say, well, every org should wrap up the clients in a way that does
things like serialization and other data checks. The problem with that is
that it (1) it is kind of redundant work and it is likely that the wrapper
will goof some nuances of the apis, and (2) it makes documentation and code
sharing really hard. That is, rather than being able to go to a central
place and read how to use the producer, LinkedIn people need to document
the LinkedIn producer wrapper, and users at LinkedIn need to read about
LinkedIn's wrapper for the producer to understand how to use it. Now
imagine this multiplied over every user.

The idea is that since everyone needs to do this we should just make it
easy to package up the best practice and plug it in. That way the
"contract" your application programs to is just the normal producer api.

-Jay

On Tue, Dec 2, 2014 at 10:06 AM, Joel Koshy <jj...@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 <jj...@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
> > >
> > >
>
>

Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Jay Kreps <ja...@confluent.io>.
Hey Joel, you are right, we discussed this, but I think we didn't think
about it as deeply as we should have. I think our take was strongly shaped
by having a wrapper api at LinkedIn that DOES do the serialization
transparently so I think you are thinking of the producer as just an
implementation detail of that wrapper. Imagine a world where every
application at LinkedIn had to figure that part out themselves. That is,
imagine that what you guys supported was just the raw producer api and that
that just handled bytes. I think in that world the types of data you would
see would be totally funky and standardizing correct usage would be a
massive pain.

Conversely, you could imagine advocating the LinkedIn approach where you
just say, well, every org should wrap up the clients in a way that does
things like serialization and other data checks. The problem with that is
that it (1) it is kind of redundant work and it is likely that the wrapper
will goof some nuances of the apis, and (2) it makes documentation and code
sharing really hard. That is, rather than being able to go to a central
place and read how to use the producer, LinkedIn people need to document
the LinkedIn producer wrapper, and users at LinkedIn need to read about
LinkedIn's wrapper for the producer to understand how to use it. Now
imagine this multiplied over every user.

The idea is that since everyone needs to do this we should just make it
easy to package up the best practice and plug it in. That way the
"contract" your application programs to is just the normal producer api.

-Jay

On Tue, Dec 2, 2014 at 10:06 AM, Joel Koshy <jj...@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 <jj...@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
> > >
> > >
>
>

Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Jun Rao <ju...@gmail.com>.
Rajiv,

Yes, that's possible within an organization. However, if you want to share
that implementation with other organizations, they will have to make code
changes, instead of just a config change.

Thanks,

Jun

On Tue, Dec 2, 2014 at 1:06 PM, Rajiv Kurian <ra...@signalfuse.com> wrote:

> Why can't the organization package the Avro implementation with a kafka
> client and distribute that library though? The risk of different users
> supplying the kafka client with different serializer/deserializer
> implementations still exists.
>
> On Tue, Dec 2, 2014 at 12:11 PM, Jun Rao <ju...@gmail.com> wrote:
>
> > Joel, Rajiv, Thunder,
> >
> > 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.
> >
> > Joel,
> >
> > 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.
> > Since the consumer client is the entry point where an application gets
> the
> > data,  if the type is enforced there, it makes it clear to all down
> stream
> > consumers.
> >
> > Thanks,
> >
> > Jun
> >
> > On Tue, Dec 2, 2014 at 10:06 AM, Joel Koshy <jj...@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 <jj...@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
> > > > >
> > > > >
> > >
> > >
> >
>

Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Rajiv Kurian <ra...@signalfuse.com>.
Why can't the organization package the Avro implementation with a kafka
client and distribute that library though? The risk of different users
supplying the kafka client with different serializer/deserializer
implementations still exists.

On Tue, Dec 2, 2014 at 12:11 PM, Jun Rao <ju...@gmail.com> wrote:

> Joel, Rajiv, Thunder,
>
> 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.
>
> Joel,
>
> 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.
> Since the consumer client is the entry point where an application gets the
> data,  if the type is enforced there, it makes it clear to all down stream
> consumers.
>
> Thanks,
>
> Jun
>
> On Tue, Dec 2, 2014 at 10:06 AM, Joel Koshy <jj...@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 <jj...@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
> > > >
> > > >
> >
> >
>

Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Rajiv Kurian <ra...@signalfuse.com>.
Yeah I am kind of sad about that :(. I just mentioned it to show that there
are material use cases for applications where you expose the underlying
ByteBuffer (I know we were talking about byte arrays) instead of
serializing/deserializing objects -  performance is a big one.


On Tue, Dec 2, 2014 at 5:42 PM, Jun Rao <ju...@confluent.io> wrote:

> Rajiv,
>
> That's probably a very special use case. Note that even in the new consumer
> api w/o the generics, the client is only going to get the byte array back.
> So, you won't be able to take advantage of reusing the ByteBuffer in the
> underlying responses.
>
> Thanks,
>
> Jun
>
> On Tue, Dec 2, 2014 at 5:26 PM, Rajiv Kurian <ra...@signalfuse.com> wrote:
>
> > 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 <jj...@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 <jj...@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 <jjkoshy.w@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 <
> > jjkoshy.w@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
> > > > >
> > >
> > >
> >
>

Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Rajiv Kurian <ra...@signalfuse.com>.
Yeah I am kind of sad about that :(. I just mentioned it to show that there
are material use cases for applications where you expose the underlying
ByteBuffer (I know we were talking about byte arrays) instead of
serializing/deserializing objects -  performance is a big one.


On Tue, Dec 2, 2014 at 5:42 PM, Jun Rao <ju...@confluent.io> wrote:

> Rajiv,
>
> That's probably a very special use case. Note that even in the new consumer
> api w/o the generics, the client is only going to get the byte array back.
> So, you won't be able to take advantage of reusing the ByteBuffer in the
> underlying responses.
>
> Thanks,
>
> Jun
>
> On Tue, Dec 2, 2014 at 5:26 PM, Rajiv Kurian <ra...@signalfuse.com> wrote:
>
> > 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 <jj...@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 <jj...@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 <jjkoshy.w@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 <
> > jjkoshy.w@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
> > > > >
> > >
> > >
> >
>

Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Jun Rao <ju...@confluent.io>.
Rajiv,

That's probably a very special use case. Note that even in the new consumer
api w/o the generics, the client is only going to get the byte array back.
So, you won't be able to take advantage of reusing the ByteBuffer in the
underlying responses.

Thanks,

Jun

On Tue, Dec 2, 2014 at 5:26 PM, Rajiv Kurian <ra...@signalfuse.com> wrote:

> 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 <jj...@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 <jj...@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 <jj...@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 <
> jjkoshy.w@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
> > > >
> >
> >
>

Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Jun Rao <ju...@confluent.io>.
Rajiv,

That's probably a very special use case. Note that even in the new consumer
api w/o the generics, the client is only going to get the byte array back.
So, you won't be able to take advantage of reusing the ByteBuffer in the
underlying responses.

Thanks,

Jun

On Tue, Dec 2, 2014 at 5:26 PM, Rajiv Kurian <ra...@signalfuse.com> wrote:

> 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 <jj...@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 <jj...@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 <jj...@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 <
> jjkoshy.w@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
> > > >
> >
> >
>

Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Rajiv Kurian <ra...@signalfuse.com>.
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 <jj...@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 <jj...@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 <jj...@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 <jjkoshy.w@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
> > >
>
>

Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Rajiv Kurian <ra...@signalfuse.com>.
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 <jj...@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 <jj...@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 <jj...@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 <jjkoshy.w@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
> > >
>
>

Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Joel Koshy <jj...@gmail.com>.
> 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 <jj...@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 <jj...@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 <jj...@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
> >


Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Joel Koshy <jj...@gmail.com>.
> 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 <jj...@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 <jj...@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 <jj...@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
> >


Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Jun Rao <ju...@gmail.com>.
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.

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.

Thanks,

Jun

On Tue, Dec 2, 2014 at 2:05 PM, Joel Koshy <jj...@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 <jj...@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 <jj...@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
>

Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Jun Rao <ju...@gmail.com>.
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.

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.

Thanks,

Jun

On Tue, Dec 2, 2014 at 2:05 PM, Joel Koshy <jj...@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 <jj...@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 <jj...@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
>

Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Joel Koshy <jj...@gmail.com>.
> 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 <jj...@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 <jj...@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

Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Joel Koshy <jj...@gmail.com>.
> 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 <jj...@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 <jj...@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

Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Rajiv Kurian <ra...@signalfuse.com>.
Why can't the organization package the Avro implementation with a kafka
client and distribute that library though? The risk of different users
supplying the kafka client with different serializer/deserializer
implementations still exists.

On Tue, Dec 2, 2014 at 12:11 PM, Jun Rao <ju...@gmail.com> wrote:

> Joel, Rajiv, Thunder,
>
> 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.
>
> Joel,
>
> 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.
> Since the consumer client is the entry point where an application gets the
> data,  if the type is enforced there, it makes it clear to all down stream
> consumers.
>
> Thanks,
>
> Jun
>
> On Tue, Dec 2, 2014 at 10:06 AM, Joel Koshy <jj...@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 <jj...@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
> > > >
> > > >
> >
> >
>

Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Jun Rao <ju...@gmail.com>.
Joel, Rajiv, Thunder,

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.

Joel,

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.
Since the consumer client is the entry point where an application gets the
data,  if the type is enforced there, it makes it clear to all down stream
consumers.

Thanks,

Jun

On Tue, Dec 2, 2014 at 10:06 AM, Joel Koshy <jj...@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 <jj...@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
> > >
> > >
>
>

Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Jun Rao <ju...@gmail.com>.
Joel, Rajiv, Thunder,

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.

Joel,

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.
Since the consumer client is the entry point where an application gets the
data,  if the type is enforced there, it makes it clear to all down stream
consumers.

Thanks,

Jun

On Tue, Dec 2, 2014 at 10:06 AM, Joel Koshy <jj...@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 <jj...@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
> > >
> > >
>
>

RE: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Thunder Stumpges <ts...@ntent.com>.
Hello, while we do not currently use the Java API, we are writing a C#/.net client (https://github.com/ntent-ad/kafka4net). FWIW, we also chose to keep the API simpler accepting just byte arrays. We did not want to impose even a simple interface onto users of the library, feeling that users will have their own serialization requirements (or not), and if desired, can write their own shim to handle serialization in the way they would like.  

Cheers,
Thunder


-----Original Message-----
From: Rajiv Kurian [mailto:rajiv@signalfuse.com] 
Sent: Tuesday, December 02, 2014 10:22 AM
To: users@kafka.apache.org
Subject: Re: [DISCUSSION] adding the serializer api back to the new java producer

It's not clear to me from your initial email what exactly can't be done with the raw accept bytes API. Serialization libraries should be share able outside of kafka. I honestly like the simplicity of the raw bytes API and feel like serialization should just remain outside of the base Kafka APIs.
Any one who wants them bundled could then create a higher level API themselves.

On Tue, Dec 2, 2014 at 10:06 AM, Joel Koshy <jj...@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 <jj...@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
> > >
> > >
>
>

Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Rajiv Kurian <ra...@signalfuse.com>.
It's not clear to me from your initial email what exactly can't be done
with the raw accept bytes API. Serialization libraries should be share able
outside of kafka. I honestly like the simplicity of the raw bytes API and
feel like serialization should just remain outside of the base Kafka APIs.
Any one who wants them bundled could then create a higher level API
themselves.

On Tue, Dec 2, 2014 at 10:06 AM, Joel Koshy <jj...@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 <jj...@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
> > >
> > >
>
>

Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Joel Koshy <jj...@gmail.com>.
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 <jj...@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
> >
> >


Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Joel Koshy <jj...@gmail.com>.
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 <jj...@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
> >
> >


Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Jun Rao <ju...@gmail.com>.
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 <jj...@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
>
>

Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Jun Rao <ju...@gmail.com>.
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 <jj...@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
>
>

Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Joel Koshy <jj...@gmail.com>.
> 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


Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Jonathan Weeks <jo...@gmail.com>.
+1 on this change — APIs are forever. As much as we’d love to see 0.8.2 release ASAP, it is important to get this right.

-JW

> On Nov 24, 2014, at 5:58 PM, Jun Rao <ju...@gmail.com> 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


Re: [DISCUSSION] adding the serializer api back to the new java producer

Posted by Joel Koshy <jj...@gmail.com>.
> 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