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 <jjkosh...@gmail.com> wrote: > > > Re: pushing complexity of dealing with objects: we're talking about > > just a call to a serialize method to convert the object to a byte > > array right? Or is there more to it? (To me) that seems less > > cumbersome than having to interact with parameterized types. Actually, > > can you explain more clearly what you mean by <q>reason about what > > type of data is being sent</q> in your original email? I have some > > notion of what that means but it is a bit vague and you might have > > meant something else. > > > > Thanks, > > > > Joel > > > > On Tue, Dec 02, 2014 at 09:15:19AM -0800, Jun Rao wrote: > > > Joel, > > > > > > Thanks for the feedback. > > > > > > Yes, the raw bytes interface is simpler than the Generic api. However, it > > > just pushes the complexity of dealing with the objects to the > > application. > > > We also thought about the layered approach. However, this may confuse the > > > users since there is no single entry point and it's not clear which > > layer a > > > user should be using. > > > > > > Jun > > > > > > > > > On Tue, Dec 2, 2014 at 12:34 AM, Joel Koshy <jjkosh...@gmail.com> wrote: > > > > > > > > makes it hard to reason about what type of data is being sent to > > Kafka > > > > and > > > > > also makes it hard to share an implementation of the serializer. For > > > > > example, to support Avro, the serialization logic could be quite > > involved > > > > > since it might need to register the Avro schema in some remote > > registry > > > > and > > > > > maintain a schema cache locally, etc. Without a serialization api, > > it's > > > > > impossible to share such an implementation so that people can easily > > > > reuse. > > > > > We sort of overlooked this implication during the initial discussion > > of > > > > the > > > > > producer api. > > > > > > > > Thanks for bringing this up and the patch. My take on this is that > > > > any reasoning about the data itself is more appropriately handled > > > > outside of the core producer API. FWIW, I don't think this was > > > > _overlooked_ during the initial discussion of the producer API > > > > (especially since it was a significant change from the old producer). > > > > IIRC we believed at the time that there is elegance and flexibility in > > > > a simple API that deals with raw bytes. I think it is more accurate to > > > > say that this is a reversal of opinion for some (which is fine) but > > > > personally I'm still in the old camp :) i.e., I really like the > > > > simplicity of the current 0.8.2 producer API and find parameterized > > > > types/generics to be distracting and annoying; and IMO any > > > > data-specific handling is better absorbed at a higher-level than the > > > > core Kafka APIs - possibly by a (very thin) wrapper producer library. > > > > I don't quite see why it is difficult to share different wrapper > > > > implementations; or even ser-de libraries for that matter that people > > > > can invoke before sending to/reading from Kafka. > > > > > > > > That said I'm not opposed to the change - it's just that I prefer > > > > what's currently there. So I'm +0 on the proposal. > > > > > > > > Thanks, > > > > > > > > Joel > > > > > > > > On Mon, Nov 24, 2014 at 05:58:50PM -0800, Jun Rao wrote: > > > > > Hi, Everyone, > > > > > > > > > > I'd like to start a discussion on whether it makes sense to add the > > > > > serializer api back to the new java producer. Currently, the new java > > > > > producer takes a byte array for both the key and the value. While > > this > > > > api > > > > > is simple, it pushes the serialization logic into the application. > > This > > > > > makes it hard to reason about what type of data is being sent to > > Kafka > > > > and > > > > > also makes it hard to share an implementation of the serializer. For > > > > > example, to support Avro, the serialization logic could be quite > > involved > > > > > since it might need to register the Avro schema in some remote > > registry > > > > and > > > > > maintain a schema cache locally, etc. Without a serialization api, > > it's > > > > > impossible to share such an implementation so that people can easily > > > > reuse. > > > > > We sort of overlooked this implication during the initial discussion > > of > > > > the > > > > > producer api. > > > > > > > > > > So, I'd like to propose an api change to the new producer by adding > > back > > > > > the serializer api similar to what we had in the old producer. > > Specially, > > > > > the proposed api changes are the following. > > > > > > > > > > First, we change KafkaProducer to take generic types K and V for the > > key > > > > > and the value, respectively. > > > > > > > > > > public class KafkaProducer<K,V> implements Producer<K,V> { > > > > > > > > > > public Future<RecordMetadata> send(ProducerRecord<K,V> record, > > > > Callback > > > > > callback); > > > > > > > > > > public Future<RecordMetadata> send(ProducerRecord<K,V> record); > > > > > } > > > > > > > > > > Second, we add two new configs, one for the key serializer and > > another > > > > for > > > > > the value serializer. Both serializers will default to the byte array > > > > > implementation. > > > > > > > > > > public class ProducerConfig extends AbstractConfig { > > > > > > > > > > .define(KEY_SERIALIZER_CLASS_CONFIG, Type.CLASS, > > > > > "org.apache.kafka.clients.producer.ByteArraySerializer", > > Importance.HIGH, > > > > > KEY_SERIALIZER_CLASS_DOC) > > > > > .define(VALUE_SERIALIZER_CLASS_CONFIG, Type.CLASS, > > > > > "org.apache.kafka.clients.producer.ByteArraySerializer", > > Importance.HIGH, > > > > > VALUE_SERIALIZER_CLASS_DOC); > > > > > } > > > > > > > > > > Both serializers will implement the following interface. > > > > > > > > > > public interface Serializer<T> extends Configurable { > > > > > public byte[] serialize(String topic, T data, boolean isKey); > > > > > > > > > > public void close(); > > > > > } > > > > > > > > > > This is more or less the same as what's in the old producer. The > > slight > > > > > differences are (1) the serializer now only requires a parameter-less > > > > > constructor; (2) the serializer has a configure() and a close() > > method > > > > for > > > > > initialization and cleanup, respectively; (3) the serialize() method > > > > > additionally takes the topic and an isKey indicator, both of which > > are > > > > > useful for things like schema registration. > > > > > > > > > > The detailed changes are included in KAFKA-1797. For completeness, I > > also > > > > > made the corresponding changes for the new java consumer api as well. > > > > > > > > > > Note that the proposed api changes are incompatible with what's in > > the > > > > > 0.8.2 branch. However, if those api changes are beneficial, it's > > probably > > > > > better to include them now in the 0.8.2 release, rather than later. > > > > > > > > > > I'd like to discuss mainly two things in this thread. > > > > > 1. Do people feel that the proposed api changes are reasonable? > > > > > 2. Are there any concerns of including the api changes in the 0.8.2 > > final > > > > > release? > > > > > > > > > > Thanks, > > > > > > > > > > Jun > > > > > > > > > > > >