You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by Jay Kreps <ja...@gmail.com> on 2014/01/24 20:54:13 UTC

New Producer Public API

As mentioned in a previous email we are working on a re-implementation of
the producer. I would like to use this email thread to discuss the details
of the public API and the configuration. I would love for us to be
incredibly picky about this public api now so it is as good as possible and
we don't need to break it in the future.

The best way to get a feel for the API is actually to take a look at the
javadoc, my hope is to get the api docs good enough so that it is
self-explanatory:
http://empathybox.com/kafka-javadoc/index.html?kafka/clients/producer/KafkaProducer.html

Please take a look at this API and give me any thoughts you may have!

It may also be reasonable to take a look at the configs:
http://empathybox.com/kafka-javadoc/kafka/clients/producer/ProducerConfig.html

The actual code is posted here:
https://issues.apache.org/jira/browse/KAFKA-1227

A few questions or comments to kick things off:
1. We need to make a decision on whether serialization of the user's key
and value should be done by the user (with our api just taking byte[]) or
if we should take an object and allow the user to configure a Serializer
class which we instantiate via reflection. We take the later approach in
the current producer, and I have carried this through to this prototype.
The tradeoff I see is this: taking byte[] is actually simpler, the user can
directly do whatever serialization they like. The complication is actually
partitioning. Currently partitioning is done by a similar plug-in api
(Partitioner) which the user can implement and configure to override how
partitions are assigned. If we take byte[] as input then we have no access
to the original object and partitioning MUST be done on the byte[]. This is
fine for hash partitioning. However for various types of semantic
partitioning (range partitioning, or whatever) you would want access to the
original object. In the current approach a producer who wishes to send
byte[] they have serialized in their own code can configure the
BytesSerialization we supply which is just a "no op" serialization.
2. We should obsess over naming and make sure each of the class names are
good.
3. Jun has already pointed out that we need to include the topic and
partition in the response, which is absolutely right. I haven't done that
yet but that definitely needs to be there.
4. Currently RecordSend.await will throw an exception if the request
failed. The intention here is that producer.send(message).await() exactly
simulates a synchronous call. Guozhang has noted that this is a little
annoying since the user must then catch exceptions. However if we remove
this then if the user doesn't check for errors they won't know one has
occurred, which I predict will be a common mistake.
5. Perhaps there is more we could do to make the async callbacks and future
we give back intuitive and easy to program against?

Some background info on implementation:

At a high level the primary difference in this producer is that it removes
the distinction between the "sync" and "async" producer. Effectively all
requests are sent asynchronously but always return a future response object
that gives the offset as well as any error that may have occurred when the
request is complete. The batching that is done in the async producer only
today is done whenever possible now. This means that the sync producer,
under load, can get performance as good as the async producer (preliminary
results show the producer getting 1m messages/sec). This works similar to
group commit in databases but with respect to the actual network
transmission--any messages that arrive while a send is in progress are
batched together. It is also possible to encourage batching even under low
load to save server resources by introducing a delay on the send to allow
more messages to accumulate; this is done using the linger.ms config (this
is similar to Nagle's algorithm in TCP).

This producer does all network communication asynchronously and in parallel
to all servers so the performance penalty for acks=-1 and waiting on
replication should be much reduced. I haven't done much benchmarking on
this yet, though.

The high level design is described a little here, though this is now a
little out of date:
https://cwiki.apache.org/confluence/display/KAFKA/Client+Rewrite

-Jay

Re: New Producer Public API

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

I think this should perform okay. We already create a number of objects per
message sent, one more shouldn't have too much performance impact if it is
just thousands per second.

-Jay


On Fri, Jan 24, 2014 at 2:28 PM, Andrey Yegorov <an...@gmail.com>wrote:

> So for each message that I need to send asynchronously I have to create a
> new instance of callback and hold on to the message?
> This looks nice in theory but in case of few thousands of request/sec this
> could use up too much extra memory and push too much to garbage collector,
> especially in case connection breaks for a few seconds and all this piles
> up as result of retry logic.
>
> I guess I can pool callbacks and do something like setMessage() on callback
> but this looks like an attempt to workaround limitations of the API.
> I'd prefer to create one instance of callback per app or per thread and
> reuse it. Since kafka producer already have messages in the batch and knows
> the batch that failed, it can pass the message to the onError() callback.
>
> Am I over-thinking this?
>
>
> ----------
> Andrey Yegorov
>
>
> On Fri, Jan 24, 2014 at 1:15 PM, Jay Kreps <ja...@gmail.com> wrote:
>
> > If I understand your use case I think usage would be something like
> >
> >   producer.send(message, new Callback() {
> >     public void onCompletion(RecordSend send) {
> >        if(send.hasError())
> >           log.write(message);
> >     }
> >   });
> >
> > Reasonable?
> >
> > In other words you can include references to any variables you like in
> the
> > callback. We could provide the message for you but that would require us
> to
> > hang on to the message object for the duration of the call which has
> memory
> > implications so I think it is better for people to only do this if they
> > want to use it.
> >
> > -Jay
> >
> >
> > On Fri, Jan 24, 2014 at 1:05 PM, Andrey Yegorov <
> andrey.yegorov@gmail.com
> > >wrote:
> >
> > > I love the callback in send() but I do not see how it helps in case of
> an
> > > error.
> > >
> > > Imagine the usecase: I want to write messages to the log so I can
> replay
> > > them to kafka later in case if async send failed.
> > > From a brief look at the API I see that I'll get back RecordSend object
> > > (which is not true already - it was not send in case of error.) From
> that
> > > object I can get some info about the error and offset. How d I get the
> > > original message back so I can write it to the log? Can you please
> > provide
> > > an example?
> > >
> > >
> > >
> > > ----------
> > > Andrey Yegorov
> > >
> > >
> > > On Fri, Jan 24, 2014 at 11:54 AM, Jay Kreps <ja...@gmail.com>
> wrote:
> > >
> > > > As mentioned in a previous email we are working on a
> re-implementation
> > of
> > > > the producer. I would like to use this email thread to discuss the
> > > details
> > > > of the public API and the configuration. I would love for us to be
> > > > incredibly picky about this public api now so it is as good as
> possible
> > > and
> > > > we don't need to break it in the future.
> > > >
> > > > The best way to get a feel for the API is actually to take a look at
> > the
> > > > javadoc, my hope is to get the api docs good enough so that it is
> > > > self-explanatory:
> > > >
> > > >
> > >
> >
> http://empathybox.com/kafka-javadoc/index.html?kafka/clients/producer/KafkaProducer.html
> > > >
> > > > Please take a look at this API and give me any thoughts you may have!
> > > >
> > > > It may also be reasonable to take a look at the configs:
> > > >
> > > >
> > >
> >
> http://empathybox.com/kafka-javadoc/kafka/clients/producer/ProducerConfig.html
> > > >
> > > > The actual code is posted here:
> > > > https://issues.apache.org/jira/browse/KAFKA-1227
> > > >
> > > > A few questions or comments to kick things off:
> > > > 1. We need to make a decision on whether serialization of the user's
> > key
> > > > and value should be done by the user (with our api just taking
> byte[])
> > or
> > > > if we should take an object and allow the user to configure a
> > Serializer
> > > > class which we instantiate via reflection. We take the later approach
> > in
> > > > the current producer, and I have carried this through to this
> > prototype.
> > > > The tradeoff I see is this: taking byte[] is actually simpler, the
> user
> > > can
> > > > directly do whatever serialization they like. The complication is
> > > actually
> > > > partitioning. Currently partitioning is done by a similar plug-in api
> > > > (Partitioner) which the user can implement and configure to override
> > how
> > > > partitions are assigned. If we take byte[] as input then we have no
> > > access
> > > > to the original object and partitioning MUST be done on the byte[].
> > This
> > > is
> > > > fine for hash partitioning. However for various types of semantic
> > > > partitioning (range partitioning, or whatever) you would want access
> to
> > > the
> > > > original object. In the current approach a producer who wishes to
> send
> > > > byte[] they have serialized in their own code can configure the
> > > > BytesSerialization we supply which is just a "no op" serialization.
> > > > 2. We should obsess over naming and make sure each of the class names
> > are
> > > > good.
> > > > 3. Jun has already pointed out that we need to include the topic and
> > > > partition in the response, which is absolutely right. I haven't done
> > that
> > > > yet but that definitely needs to be there.
> > > > 4. Currently RecordSend.await will throw an exception if the request
> > > > failed. The intention here is that producer.send(message).await()
> > exactly
> > > > simulates a synchronous call. Guozhang has noted that this is a
> little
> > > > annoying since the user must then catch exceptions. However if we
> > remove
> > > > this then if the user doesn't check for errors they won't know one
> has
> > > > occurred, which I predict will be a common mistake.
> > > > 5. Perhaps there is more we could do to make the async callbacks and
> > > future
> > > > we give back intuitive and easy to program against?
> > > >
> > > > Some background info on implementation:
> > > >
> > > > At a high level the primary difference in this producer is that it
> > > removes
> > > > the distinction between the "sync" and "async" producer. Effectively
> > all
> > > > requests are sent asynchronously but always return a future response
> > > object
> > > > that gives the offset as well as any error that may have occurred
> when
> > > the
> > > > request is complete. The batching that is done in the async producer
> > only
> > > > today is done whenever possible now. This means that the sync
> producer,
> > > > under load, can get performance as good as the async producer
> > > (preliminary
> > > > results show the producer getting 1m messages/sec). This works
> similar
> > to
> > > > group commit in databases but with respect to the actual network
> > > > transmission--any messages that arrive while a send is in progress
> are
> > > > batched together. It is also possible to encourage batching even
> under
> > > low
> > > > load to save server resources by introducing a delay on the send to
> > allow
> > > > more messages to accumulate; this is done using the linger.ms config
> > > (this
> > > > is similar to Nagle's algorithm in TCP).
> > > >
> > > > This producer does all network communication asynchronously and in
> > > parallel
> > > > to all servers so the performance penalty for acks=-1 and waiting on
> > > > replication should be much reduced. I haven't done much benchmarking
> on
> > > > this yet, though.
> > > >
> > > > The high level design is described a little here, though this is now
> a
> > > > little out of date:
> > > > https://cwiki.apache.org/confluence/display/KAFKA/Client+Rewrite
> > > >
> > > > -Jay
> > > >
> > >
> >
>

Re: New Producer Public API

Posted by Andrey Yegorov <an...@gmail.com>.
So for each message that I need to send asynchronously I have to create a
new instance of callback and hold on to the message?
This looks nice in theory but in case of few thousands of request/sec this
could use up too much extra memory and push too much to garbage collector,
especially in case connection breaks for a few seconds and all this piles
up as result of retry logic.

I guess I can pool callbacks and do something like setMessage() on callback
but this looks like an attempt to workaround limitations of the API.
I'd prefer to create one instance of callback per app or per thread and
reuse it. Since kafka producer already have messages in the batch and knows
the batch that failed, it can pass the message to the onError() callback.

Am I over-thinking this?


----------
Andrey Yegorov


On Fri, Jan 24, 2014 at 1:15 PM, Jay Kreps <ja...@gmail.com> wrote:

> If I understand your use case I think usage would be something like
>
>   producer.send(message, new Callback() {
>     public void onCompletion(RecordSend send) {
>        if(send.hasError())
>           log.write(message);
>     }
>   });
>
> Reasonable?
>
> In other words you can include references to any variables you like in the
> callback. We could provide the message for you but that would require us to
> hang on to the message object for the duration of the call which has memory
> implications so I think it is better for people to only do this if they
> want to use it.
>
> -Jay
>
>
> On Fri, Jan 24, 2014 at 1:05 PM, Andrey Yegorov <andrey.yegorov@gmail.com
> >wrote:
>
> > I love the callback in send() but I do not see how it helps in case of an
> > error.
> >
> > Imagine the usecase: I want to write messages to the log so I can replay
> > them to kafka later in case if async send failed.
> > From a brief look at the API I see that I'll get back RecordSend object
> > (which is not true already - it was not send in case of error.) From that
> > object I can get some info about the error and offset. How d I get the
> > original message back so I can write it to the log? Can you please
> provide
> > an example?
> >
> >
> >
> > ----------
> > Andrey Yegorov
> >
> >
> > On Fri, Jan 24, 2014 at 11:54 AM, Jay Kreps <ja...@gmail.com> wrote:
> >
> > > As mentioned in a previous email we are working on a re-implementation
> of
> > > the producer. I would like to use this email thread to discuss the
> > details
> > > of the public API and the configuration. I would love for us to be
> > > incredibly picky about this public api now so it is as good as possible
> > and
> > > we don't need to break it in the future.
> > >
> > > The best way to get a feel for the API is actually to take a look at
> the
> > > javadoc, my hope is to get the api docs good enough so that it is
> > > self-explanatory:
> > >
> > >
> >
> http://empathybox.com/kafka-javadoc/index.html?kafka/clients/producer/KafkaProducer.html
> > >
> > > Please take a look at this API and give me any thoughts you may have!
> > >
> > > It may also be reasonable to take a look at the configs:
> > >
> > >
> >
> http://empathybox.com/kafka-javadoc/kafka/clients/producer/ProducerConfig.html
> > >
> > > The actual code is posted here:
> > > https://issues.apache.org/jira/browse/KAFKA-1227
> > >
> > > A few questions or comments to kick things off:
> > > 1. We need to make a decision on whether serialization of the user's
> key
> > > and value should be done by the user (with our api just taking byte[])
> or
> > > if we should take an object and allow the user to configure a
> Serializer
> > > class which we instantiate via reflection. We take the later approach
> in
> > > the current producer, and I have carried this through to this
> prototype.
> > > The tradeoff I see is this: taking byte[] is actually simpler, the user
> > can
> > > directly do whatever serialization they like. The complication is
> > actually
> > > partitioning. Currently partitioning is done by a similar plug-in api
> > > (Partitioner) which the user can implement and configure to override
> how
> > > partitions are assigned. If we take byte[] as input then we have no
> > access
> > > to the original object and partitioning MUST be done on the byte[].
> This
> > is
> > > fine for hash partitioning. However for various types of semantic
> > > partitioning (range partitioning, or whatever) you would want access to
> > the
> > > original object. In the current approach a producer who wishes to send
> > > byte[] they have serialized in their own code can configure the
> > > BytesSerialization we supply which is just a "no op" serialization.
> > > 2. We should obsess over naming and make sure each of the class names
> are
> > > good.
> > > 3. Jun has already pointed out that we need to include the topic and
> > > partition in the response, which is absolutely right. I haven't done
> that
> > > yet but that definitely needs to be there.
> > > 4. Currently RecordSend.await will throw an exception if the request
> > > failed. The intention here is that producer.send(message).await()
> exactly
> > > simulates a synchronous call. Guozhang has noted that this is a little
> > > annoying since the user must then catch exceptions. However if we
> remove
> > > this then if the user doesn't check for errors they won't know one has
> > > occurred, which I predict will be a common mistake.
> > > 5. Perhaps there is more we could do to make the async callbacks and
> > future
> > > we give back intuitive and easy to program against?
> > >
> > > Some background info on implementation:
> > >
> > > At a high level the primary difference in this producer is that it
> > removes
> > > the distinction between the "sync" and "async" producer. Effectively
> all
> > > requests are sent asynchronously but always return a future response
> > object
> > > that gives the offset as well as any error that may have occurred when
> > the
> > > request is complete. The batching that is done in the async producer
> only
> > > today is done whenever possible now. This means that the sync producer,
> > > under load, can get performance as good as the async producer
> > (preliminary
> > > results show the producer getting 1m messages/sec). This works similar
> to
> > > group commit in databases but with respect to the actual network
> > > transmission--any messages that arrive while a send is in progress are
> > > batched together. It is also possible to encourage batching even under
> > low
> > > load to save server resources by introducing a delay on the send to
> allow
> > > more messages to accumulate; this is done using the linger.ms config
> > (this
> > > is similar to Nagle's algorithm in TCP).
> > >
> > > This producer does all network communication asynchronously and in
> > parallel
> > > to all servers so the performance penalty for acks=-1 and waiting on
> > > replication should be much reduced. I haven't done much benchmarking on
> > > this yet, though.
> > >
> > > The high level design is described a little here, though this is now a
> > > little out of date:
> > > https://cwiki.apache.org/confluence/display/KAFKA/Client+Rewrite
> > >
> > > -Jay
> > >
> >
>

Re: New Producer Public API

Posted by Jay Kreps <ja...@gmail.com>.
If I understand your use case I think usage would be something like

  producer.send(message, new Callback() {
    public void onCompletion(RecordSend send) {
       if(send.hasError())
          log.write(message);
    }
  });

Reasonable?

In other words you can include references to any variables you like in the
callback. We could provide the message for you but that would require us to
hang on to the message object for the duration of the call which has memory
implications so I think it is better for people to only do this if they
want to use it.

-Jay


On Fri, Jan 24, 2014 at 1:05 PM, Andrey Yegorov <an...@gmail.com>wrote:

> I love the callback in send() but I do not see how it helps in case of an
> error.
>
> Imagine the usecase: I want to write messages to the log so I can replay
> them to kafka later in case if async send failed.
> From a brief look at the API I see that I'll get back RecordSend object
> (which is not true already - it was not send in case of error.) From that
> object I can get some info about the error and offset. How d I get the
> original message back so I can write it to the log? Can you please provide
> an example?
>
>
>
> ----------
> Andrey Yegorov
>
>
> On Fri, Jan 24, 2014 at 11:54 AM, Jay Kreps <ja...@gmail.com> wrote:
>
> > As mentioned in a previous email we are working on a re-implementation of
> > the producer. I would like to use this email thread to discuss the
> details
> > of the public API and the configuration. I would love for us to be
> > incredibly picky about this public api now so it is as good as possible
> and
> > we don't need to break it in the future.
> >
> > The best way to get a feel for the API is actually to take a look at the
> > javadoc, my hope is to get the api docs good enough so that it is
> > self-explanatory:
> >
> >
> http://empathybox.com/kafka-javadoc/index.html?kafka/clients/producer/KafkaProducer.html
> >
> > Please take a look at this API and give me any thoughts you may have!
> >
> > It may also be reasonable to take a look at the configs:
> >
> >
> http://empathybox.com/kafka-javadoc/kafka/clients/producer/ProducerConfig.html
> >
> > The actual code is posted here:
> > https://issues.apache.org/jira/browse/KAFKA-1227
> >
> > A few questions or comments to kick things off:
> > 1. We need to make a decision on whether serialization of the user's key
> > and value should be done by the user (with our api just taking byte[]) or
> > if we should take an object and allow the user to configure a Serializer
> > class which we instantiate via reflection. We take the later approach in
> > the current producer, and I have carried this through to this prototype.
> > The tradeoff I see is this: taking byte[] is actually simpler, the user
> can
> > directly do whatever serialization they like. The complication is
> actually
> > partitioning. Currently partitioning is done by a similar plug-in api
> > (Partitioner) which the user can implement and configure to override how
> > partitions are assigned. If we take byte[] as input then we have no
> access
> > to the original object and partitioning MUST be done on the byte[]. This
> is
> > fine for hash partitioning. However for various types of semantic
> > partitioning (range partitioning, or whatever) you would want access to
> the
> > original object. In the current approach a producer who wishes to send
> > byte[] they have serialized in their own code can configure the
> > BytesSerialization we supply which is just a "no op" serialization.
> > 2. We should obsess over naming and make sure each of the class names are
> > good.
> > 3. Jun has already pointed out that we need to include the topic and
> > partition in the response, which is absolutely right. I haven't done that
> > yet but that definitely needs to be there.
> > 4. Currently RecordSend.await will throw an exception if the request
> > failed. The intention here is that producer.send(message).await() exactly
> > simulates a synchronous call. Guozhang has noted that this is a little
> > annoying since the user must then catch exceptions. However if we remove
> > this then if the user doesn't check for errors they won't know one has
> > occurred, which I predict will be a common mistake.
> > 5. Perhaps there is more we could do to make the async callbacks and
> future
> > we give back intuitive and easy to program against?
> >
> > Some background info on implementation:
> >
> > At a high level the primary difference in this producer is that it
> removes
> > the distinction between the "sync" and "async" producer. Effectively all
> > requests are sent asynchronously but always return a future response
> object
> > that gives the offset as well as any error that may have occurred when
> the
> > request is complete. The batching that is done in the async producer only
> > today is done whenever possible now. This means that the sync producer,
> > under load, can get performance as good as the async producer
> (preliminary
> > results show the producer getting 1m messages/sec). This works similar to
> > group commit in databases but with respect to the actual network
> > transmission--any messages that arrive while a send is in progress are
> > batched together. It is also possible to encourage batching even under
> low
> > load to save server resources by introducing a delay on the send to allow
> > more messages to accumulate; this is done using the linger.ms config
> (this
> > is similar to Nagle's algorithm in TCP).
> >
> > This producer does all network communication asynchronously and in
> parallel
> > to all servers so the performance penalty for acks=-1 and waiting on
> > replication should be much reduced. I haven't done much benchmarking on
> > this yet, though.
> >
> > The high level design is described a little here, though this is now a
> > little out of date:
> > https://cwiki.apache.org/confluence/display/KAFKA/Client+Rewrite
> >
> > -Jay
> >
>

Re: New Producer Public API

Posted by Andrey Yegorov <an...@gmail.com>.
I love the callback in send() but I do not see how it helps in case of an
error.

Imagine the usecase: I want to write messages to the log so I can replay
them to kafka later in case if async send failed.
>From a brief look at the API I see that I'll get back RecordSend object
(which is not true already - it was not send in case of error.) From that
object I can get some info about the error and offset. How d I get the
original message back so I can write it to the log? Can you please provide
an example?



----------
Andrey Yegorov


On Fri, Jan 24, 2014 at 11:54 AM, Jay Kreps <ja...@gmail.com> wrote:

> As mentioned in a previous email we are working on a re-implementation of
> the producer. I would like to use this email thread to discuss the details
> of the public API and the configuration. I would love for us to be
> incredibly picky about this public api now so it is as good as possible and
> we don't need to break it in the future.
>
> The best way to get a feel for the API is actually to take a look at the
> javadoc, my hope is to get the api docs good enough so that it is
> self-explanatory:
>
> http://empathybox.com/kafka-javadoc/index.html?kafka/clients/producer/KafkaProducer.html
>
> Please take a look at this API and give me any thoughts you may have!
>
> It may also be reasonable to take a look at the configs:
>
> http://empathybox.com/kafka-javadoc/kafka/clients/producer/ProducerConfig.html
>
> The actual code is posted here:
> https://issues.apache.org/jira/browse/KAFKA-1227
>
> A few questions or comments to kick things off:
> 1. We need to make a decision on whether serialization of the user's key
> and value should be done by the user (with our api just taking byte[]) or
> if we should take an object and allow the user to configure a Serializer
> class which we instantiate via reflection. We take the later approach in
> the current producer, and I have carried this through to this prototype.
> The tradeoff I see is this: taking byte[] is actually simpler, the user can
> directly do whatever serialization they like. The complication is actually
> partitioning. Currently partitioning is done by a similar plug-in api
> (Partitioner) which the user can implement and configure to override how
> partitions are assigned. If we take byte[] as input then we have no access
> to the original object and partitioning MUST be done on the byte[]. This is
> fine for hash partitioning. However for various types of semantic
> partitioning (range partitioning, or whatever) you would want access to the
> original object. In the current approach a producer who wishes to send
> byte[] they have serialized in their own code can configure the
> BytesSerialization we supply which is just a "no op" serialization.
> 2. We should obsess over naming and make sure each of the class names are
> good.
> 3. Jun has already pointed out that we need to include the topic and
> partition in the response, which is absolutely right. I haven't done that
> yet but that definitely needs to be there.
> 4. Currently RecordSend.await will throw an exception if the request
> failed. The intention here is that producer.send(message).await() exactly
> simulates a synchronous call. Guozhang has noted that this is a little
> annoying since the user must then catch exceptions. However if we remove
> this then if the user doesn't check for errors they won't know one has
> occurred, which I predict will be a common mistake.
> 5. Perhaps there is more we could do to make the async callbacks and future
> we give back intuitive and easy to program against?
>
> Some background info on implementation:
>
> At a high level the primary difference in this producer is that it removes
> the distinction between the "sync" and "async" producer. Effectively all
> requests are sent asynchronously but always return a future response object
> that gives the offset as well as any error that may have occurred when the
> request is complete. The batching that is done in the async producer only
> today is done whenever possible now. This means that the sync producer,
> under load, can get performance as good as the async producer (preliminary
> results show the producer getting 1m messages/sec). This works similar to
> group commit in databases but with respect to the actual network
> transmission--any messages that arrive while a send is in progress are
> batched together. It is also possible to encourage batching even under low
> load to save server resources by introducing a delay on the send to allow
> more messages to accumulate; this is done using the linger.ms config (this
> is similar to Nagle's algorithm in TCP).
>
> This producer does all network communication asynchronously and in parallel
> to all servers so the performance penalty for acks=-1 and waiting on
> replication should be much reduced. I haven't done much benchmarking on
> this yet, though.
>
> The high level design is described a little here, though this is now a
> little out of date:
> https://cwiki.apache.org/confluence/display/KAFKA/Client+Rewrite
>
> -Jay
>

Re: New Producer Public API

Posted by David Arthur <mu...@gmail.com>.
On 1/24/14 7:41 PM, Jay Kreps wrote:
> Yeah I'll fix that name.
>
> Hmm, yeah, I agree that often you want to be able delay network
> connectivity until you have started everything up. But at the same time I
> kind of loath special init() methods because you always forget to call them
> and get one round of error every time.
One pattern I've used in the past is to use lazy initialization but also 
provide a method to eagerly do it. E.g., if init() wasn't called, the 
first call of send() would call it for you.
> I wonder if in those cases one could
> just avoid creating the producer instance until you are ready to connect.
> Basically if you think of the producer instance as the equivalent of a
> socket connection or whatever this kind of makes sense.
>
> -Jay
>
>
> On Fri, Jan 24, 2014 at 4:34 PM, Roger Hoover <ro...@gmail.com>wrote:
>
>> Jay,
>>
>> Thanks for the explanation.  I didn't realize that the broker list was for
>> bootstrapping and was not required to be a complete list of all brokers
>> (although I see now that it's clearly stated in the text description of the
>> parameter).  Nonetheless, does it still make sense to make the config
>> parameter more clear?  Instead of BROKER_LIST_CONFIG, it could be something
>> like BROKER_LIST_INITIAL_CONFIG or BROKER_DISCOVERY_LIST_CONFIG or
>> BROKER_BOOTSTRAP_LIST_CONFIG?
>>
>>   I like the idea of proactively checking that at least one broker url is
>> working and failing fast if it is not.  My 2 cents is that it should be
>> triggered by a method call like initialize() rather than doing it in the
>> constructor.  Sometimes for unit tests or other purposes, you want to be
>> able to create objects without triggering network dependencies.
>>
>> Cheers,
>>
>> Roger
>>
>>
>> On Fri, Jan 24, 2014 at 4:13 PM, Jay Kreps <ja...@gmail.com> wrote:
>>
>>> Roger,
>>>
>>> These are good questions.
>>>
>>> 1. The producer since 0.8 is actually zookeeper free, so this is not new
>> to
>>> this client it is true for the current client as well. Our experience was
>>> that direct zookeeper connections from zillions of producers wasn't a
>> good
>>> idea for a number of reasons. Our intention is to remove this dependency
>>> from the consumer as well. The configuration in the producer doesn't need
>>> the full set of brokers, though, just one or two machines to bootstrap
>> the
>>> state of the cluster from--in other words it isn't like you need to
>>> reconfigure your clients every time you add some servers. This is exactly
>>> how zookeeper works too--if we used zookeeper you would need to give a
>> list
>>> of zk urls in case a particular zk server was down. Basically either way
>>> you need a few statically configured nodes to go to discover the full
>> state
>>> of the cluster. For people who don't like hard coding hosts you can use a
>>> VIP or dns or something instead.
>>>
>>> 2. Yes this is a good point and was a concern I had too--the current
>>> behavior is that with bad urls the client would start normally and then
>>> hang trying to fetch metadata when the first message is sent and finally
>>> give up and throw an exception. This is not ideal.
>>>
>>> The challenge is this: we use the bootstrap urls to fetch metadata for
>>> particular topics but we don't know which until we start getting messages
>>> for them. We have the option of fetching metadata for all topics but the
>>> problem is that for a cluster hosting tens of thousands of topics that is
>>> actually a ton of data.
>>>
>>> An alternative that this just made me think of is that we could
>> proactively
>>> connect to bootstrap urls sequentially until one succeeds when the
>> producer
>>> is first created and fail fast if we can't establish a connection. This
>>> would not be wasted work as we could use the connection for the metadata
>>> request when the first message is sent. I like this solution and will
>>> implement it. So thanks for asking!
>>>
>>> -Jay
>>>
>>>
>>>
>>> On Fri, Jan 24, 2014 at 2:46 PM, Roger Hoover <roger.hoover@gmail.com
>>>> wrote:
>>>> A couple comments:
>>>>
>>>> 1) Why does the config use a broker list instead of discovering the
>>> brokers
>>>> in ZooKeeper?  It doesn't match the HighLevelConsumer API.
>>>>
>>>> 2) It looks like broker connections are created on demand.  I'm
>> wondering
>>>> if sometimes you might want to flush out config or network connectivity
>>>> issues before pushing the first message through.
>>>>
>>>> Should there also be a KafkaProducer.connect() or .open() method or
>>>> connectAll()?  I guess it would try to connect to all brokers in the
>>>> BROKER_LIST_CONFIG
>>>>
>>>> HTH,
>>>>
>>>> Roger
>>>>
>>>>
>>>> On Fri, Jan 24, 2014 at 11:54 AM, Jay Kreps <ja...@gmail.com>
>> wrote:
>>>>> As mentioned in a previous email we are working on a
>> re-implementation
>>> of
>>>>> the producer. I would like to use this email thread to discuss the
>>>> details
>>>>> of the public API and the configuration. I would love for us to be
>>>>> incredibly picky about this public api now so it is as good as
>> possible
>>>> and
>>>>> we don't need to break it in the future.
>>>>>
>>>>> The best way to get a feel for the API is actually to take a look at
>>> the
>>>>> javadoc, my hope is to get the api docs good enough so that it is
>>>>> self-explanatory:
>>>>>
>>>>>
>> http://empathybox.com/kafka-javadoc/index.html?kafka/clients/producer/KafkaProducer.html
>>>>> Please take a look at this API and give me any thoughts you may have!
>>>>>
>>>>> It may also be reasonable to take a look at the configs:
>>>>>
>>>>>
>> http://empathybox.com/kafka-javadoc/kafka/clients/producer/ProducerConfig.html
>>>>> The actual code is posted here:
>>>>> https://issues.apache.org/jira/browse/KAFKA-1227
>>>>>
>>>>> A few questions or comments to kick things off:
>>>>> 1. We need to make a decision on whether serialization of the user's
>>> key
>>>>> and value should be done by the user (with our api just taking
>> byte[])
>>> or
>>>>> if we should take an object and allow the user to configure a
>>> Serializer
>>>>> class which we instantiate via reflection. We take the later approach
>>> in
>>>>> the current producer, and I have carried this through to this
>>> prototype.
>>>>> The tradeoff I see is this: taking byte[] is actually simpler, the
>> user
>>>> can
>>>>> directly do whatever serialization they like. The complication is
>>>> actually
>>>>> partitioning. Currently partitioning is done by a similar plug-in api
>>>>> (Partitioner) which the user can implement and configure to override
>>> how
>>>>> partitions are assigned. If we take byte[] as input then we have no
>>>> access
>>>>> to the original object and partitioning MUST be done on the byte[].
>>> This
>>>> is
>>>>> fine for hash partitioning. However for various types of semantic
>>>>> partitioning (range partitioning, or whatever) you would want access
>> to
>>>> the
>>>>> original object. In the current approach a producer who wishes to
>> send
>>>>> byte[] they have serialized in their own code can configure the
>>>>> BytesSerialization we supply which is just a "no op" serialization.
>>>>> 2. We should obsess over naming and make sure each of the class names
>>> are
>>>>> good.
>>>>> 3. Jun has already pointed out that we need to include the topic and
>>>>> partition in the response, which is absolutely right. I haven't done
>>> that
>>>>> yet but that definitely needs to be there.
>>>>> 4. Currently RecordSend.await will throw an exception if the request
>>>>> failed. The intention here is that producer.send(message).await()
>>> exactly
>>>>> simulates a synchronous call. Guozhang has noted that this is a
>> little
>>>>> annoying since the user must then catch exceptions. However if we
>>> remove
>>>>> this then if the user doesn't check for errors they won't know one
>> has
>>>>> occurred, which I predict will be a common mistake.
>>>>> 5. Perhaps there is more we could do to make the async callbacks and
>>>> future
>>>>> we give back intuitive and easy to program against?
>>>>>
>>>>> Some background info on implementation:
>>>>>
>>>>> At a high level the primary difference in this producer is that it
>>>> removes
>>>>> the distinction between the "sync" and "async" producer. Effectively
>>> all
>>>>> requests are sent asynchronously but always return a future response
>>>> object
>>>>> that gives the offset as well as any error that may have occurred
>> when
>>>> the
>>>>> request is complete. The batching that is done in the async producer
>>> only
>>>>> today is done whenever possible now. This means that the sync
>> producer,
>>>>> under load, can get performance as good as the async producer
>>>> (preliminary
>>>>> results show the producer getting 1m messages/sec). This works
>> similar
>>> to
>>>>> group commit in databases but with respect to the actual network
>>>>> transmission--any messages that arrive while a send is in progress
>> are
>>>>> batched together. It is also possible to encourage batching even
>> under
>>>> low
>>>>> load to save server resources by introducing a delay on the send to
>>> allow
>>>>> more messages to accumulate; this is done using the linger.ms config
>>>> (this
>>>>> is similar to Nagle's algorithm in TCP).
>>>>>
>>>>> This producer does all network communication asynchronously and in
>>>> parallel
>>>>> to all servers so the performance penalty for acks=-1 and waiting on
>>>>> replication should be much reduced. I haven't done much benchmarking
>> on
>>>>> this yet, though.
>>>>>
>>>>> The high level design is described a little here, though this is now
>> a
>>>>> little out of date:
>>>>> https://cwiki.apache.org/confluence/display/KAFKA/Client+Rewrite
>>>>>
>>>>> -Jay
>>>>>


Re: New Producer Public API

Posted by Jay Kreps <ja...@gmail.com>.
Yeah I'll fix that name.

Hmm, yeah, I agree that often you want to be able delay network
connectivity until you have started everything up. But at the same time I
kind of loath special init() methods because you always forget to call them
and get one round of error every time. I wonder if in those cases one could
just avoid creating the producer instance until you are ready to connect.
Basically if you think of the producer instance as the equivalent of a
socket connection or whatever this kind of makes sense.

-Jay


On Fri, Jan 24, 2014 at 4:34 PM, Roger Hoover <ro...@gmail.com>wrote:

> Jay,
>
> Thanks for the explanation.  I didn't realize that the broker list was for
> bootstrapping and was not required to be a complete list of all brokers
> (although I see now that it's clearly stated in the text description of the
> parameter).  Nonetheless, does it still make sense to make the config
> parameter more clear?  Instead of BROKER_LIST_CONFIG, it could be something
> like BROKER_LIST_INITIAL_CONFIG or BROKER_DISCOVERY_LIST_CONFIG or
> BROKER_BOOTSTRAP_LIST_CONFIG?
>
>  I like the idea of proactively checking that at least one broker url is
> working and failing fast if it is not.  My 2 cents is that it should be
> triggered by a method call like initialize() rather than doing it in the
> constructor.  Sometimes for unit tests or other purposes, you want to be
> able to create objects without triggering network dependencies.
>
> Cheers,
>
> Roger
>
>
> On Fri, Jan 24, 2014 at 4:13 PM, Jay Kreps <ja...@gmail.com> wrote:
>
> > Roger,
> >
> > These are good questions.
> >
> > 1. The producer since 0.8 is actually zookeeper free, so this is not new
> to
> > this client it is true for the current client as well. Our experience was
> > that direct zookeeper connections from zillions of producers wasn't a
> good
> > idea for a number of reasons. Our intention is to remove this dependency
> > from the consumer as well. The configuration in the producer doesn't need
> > the full set of brokers, though, just one or two machines to bootstrap
> the
> > state of the cluster from--in other words it isn't like you need to
> > reconfigure your clients every time you add some servers. This is exactly
> > how zookeeper works too--if we used zookeeper you would need to give a
> list
> > of zk urls in case a particular zk server was down. Basically either way
> > you need a few statically configured nodes to go to discover the full
> state
> > of the cluster. For people who don't like hard coding hosts you can use a
> > VIP or dns or something instead.
> >
> > 2. Yes this is a good point and was a concern I had too--the current
> > behavior is that with bad urls the client would start normally and then
> > hang trying to fetch metadata when the first message is sent and finally
> > give up and throw an exception. This is not ideal.
> >
> > The challenge is this: we use the bootstrap urls to fetch metadata for
> > particular topics but we don't know which until we start getting messages
> > for them. We have the option of fetching metadata for all topics but the
> > problem is that for a cluster hosting tens of thousands of topics that is
> > actually a ton of data.
> >
> > An alternative that this just made me think of is that we could
> proactively
> > connect to bootstrap urls sequentially until one succeeds when the
> producer
> > is first created and fail fast if we can't establish a connection. This
> > would not be wasted work as we could use the connection for the metadata
> > request when the first message is sent. I like this solution and will
> > implement it. So thanks for asking!
> >
> > -Jay
> >
> >
> >
> > On Fri, Jan 24, 2014 at 2:46 PM, Roger Hoover <roger.hoover@gmail.com
> > >wrote:
> >
> > > A couple comments:
> > >
> > > 1) Why does the config use a broker list instead of discovering the
> > brokers
> > > in ZooKeeper?  It doesn't match the HighLevelConsumer API.
> > >
> > > 2) It looks like broker connections are created on demand.  I'm
> wondering
> > > if sometimes you might want to flush out config or network connectivity
> > > issues before pushing the first message through.
> > >
> > > Should there also be a KafkaProducer.connect() or .open() method or
> > > connectAll()?  I guess it would try to connect to all brokers in the
> > > BROKER_LIST_CONFIG
> > >
> > > HTH,
> > >
> > > Roger
> > >
> > >
> > > On Fri, Jan 24, 2014 at 11:54 AM, Jay Kreps <ja...@gmail.com>
> wrote:
> > >
> > > > As mentioned in a previous email we are working on a
> re-implementation
> > of
> > > > the producer. I would like to use this email thread to discuss the
> > > details
> > > > of the public API and the configuration. I would love for us to be
> > > > incredibly picky about this public api now so it is as good as
> possible
> > > and
> > > > we don't need to break it in the future.
> > > >
> > > > The best way to get a feel for the API is actually to take a look at
> > the
> > > > javadoc, my hope is to get the api docs good enough so that it is
> > > > self-explanatory:
> > > >
> > > >
> > >
> >
> http://empathybox.com/kafka-javadoc/index.html?kafka/clients/producer/KafkaProducer.html
> > > >
> > > > Please take a look at this API and give me any thoughts you may have!
> > > >
> > > > It may also be reasonable to take a look at the configs:
> > > >
> > > >
> > >
> >
> http://empathybox.com/kafka-javadoc/kafka/clients/producer/ProducerConfig.html
> > > >
> > > > The actual code is posted here:
> > > > https://issues.apache.org/jira/browse/KAFKA-1227
> > > >
> > > > A few questions or comments to kick things off:
> > > > 1. We need to make a decision on whether serialization of the user's
> > key
> > > > and value should be done by the user (with our api just taking
> byte[])
> > or
> > > > if we should take an object and allow the user to configure a
> > Serializer
> > > > class which we instantiate via reflection. We take the later approach
> > in
> > > > the current producer, and I have carried this through to this
> > prototype.
> > > > The tradeoff I see is this: taking byte[] is actually simpler, the
> user
> > > can
> > > > directly do whatever serialization they like. The complication is
> > > actually
> > > > partitioning. Currently partitioning is done by a similar plug-in api
> > > > (Partitioner) which the user can implement and configure to override
> > how
> > > > partitions are assigned. If we take byte[] as input then we have no
> > > access
> > > > to the original object and partitioning MUST be done on the byte[].
> > This
> > > is
> > > > fine for hash partitioning. However for various types of semantic
> > > > partitioning (range partitioning, or whatever) you would want access
> to
> > > the
> > > > original object. In the current approach a producer who wishes to
> send
> > > > byte[] they have serialized in their own code can configure the
> > > > BytesSerialization we supply which is just a "no op" serialization.
> > > > 2. We should obsess over naming and make sure each of the class names
> > are
> > > > good.
> > > > 3. Jun has already pointed out that we need to include the topic and
> > > > partition in the response, which is absolutely right. I haven't done
> > that
> > > > yet but that definitely needs to be there.
> > > > 4. Currently RecordSend.await will throw an exception if the request
> > > > failed. The intention here is that producer.send(message).await()
> > exactly
> > > > simulates a synchronous call. Guozhang has noted that this is a
> little
> > > > annoying since the user must then catch exceptions. However if we
> > remove
> > > > this then if the user doesn't check for errors they won't know one
> has
> > > > occurred, which I predict will be a common mistake.
> > > > 5. Perhaps there is more we could do to make the async callbacks and
> > > future
> > > > we give back intuitive and easy to program against?
> > > >
> > > > Some background info on implementation:
> > > >
> > > > At a high level the primary difference in this producer is that it
> > > removes
> > > > the distinction between the "sync" and "async" producer. Effectively
> > all
> > > > requests are sent asynchronously but always return a future response
> > > object
> > > > that gives the offset as well as any error that may have occurred
> when
> > > the
> > > > request is complete. The batching that is done in the async producer
> > only
> > > > today is done whenever possible now. This means that the sync
> producer,
> > > > under load, can get performance as good as the async producer
> > > (preliminary
> > > > results show the producer getting 1m messages/sec). This works
> similar
> > to
> > > > group commit in databases but with respect to the actual network
> > > > transmission--any messages that arrive while a send is in progress
> are
> > > > batched together. It is also possible to encourage batching even
> under
> > > low
> > > > load to save server resources by introducing a delay on the send to
> > allow
> > > > more messages to accumulate; this is done using the linger.ms config
> > > (this
> > > > is similar to Nagle's algorithm in TCP).
> > > >
> > > > This producer does all network communication asynchronously and in
> > > parallel
> > > > to all servers so the performance penalty for acks=-1 and waiting on
> > > > replication should be much reduced. I haven't done much benchmarking
> on
> > > > this yet, though.
> > > >
> > > > The high level design is described a little here, though this is now
> a
> > > > little out of date:
> > > > https://cwiki.apache.org/confluence/display/KAFKA/Client+Rewrite
> > > >
> > > > -Jay
> > > >
> > >
> >
>

Re: New Producer Public API

Posted by Jay Kreps <ja...@gmail.com>.
Yeah I'll fix that name.

Hmm, yeah, I agree that often you want to be able delay network
connectivity until you have started everything up. But at the same time I
kind of loath special init() methods because you always forget to call them
and get one round of error every time. I wonder if in those cases one could
just avoid creating the producer instance until you are ready to connect.
Basically if you think of the producer instance as the equivalent of a
socket connection or whatever this kind of makes sense.

-Jay


On Fri, Jan 24, 2014 at 4:34 PM, Roger Hoover <ro...@gmail.com>wrote:

> Jay,
>
> Thanks for the explanation.  I didn't realize that the broker list was for
> bootstrapping and was not required to be a complete list of all brokers
> (although I see now that it's clearly stated in the text description of the
> parameter).  Nonetheless, does it still make sense to make the config
> parameter more clear?  Instead of BROKER_LIST_CONFIG, it could be something
> like BROKER_LIST_INITIAL_CONFIG or BROKER_DISCOVERY_LIST_CONFIG or
> BROKER_BOOTSTRAP_LIST_CONFIG?
>
>  I like the idea of proactively checking that at least one broker url is
> working and failing fast if it is not.  My 2 cents is that it should be
> triggered by a method call like initialize() rather than doing it in the
> constructor.  Sometimes for unit tests or other purposes, you want to be
> able to create objects without triggering network dependencies.
>
> Cheers,
>
> Roger
>
>
> On Fri, Jan 24, 2014 at 4:13 PM, Jay Kreps <ja...@gmail.com> wrote:
>
> > Roger,
> >
> > These are good questions.
> >
> > 1. The producer since 0.8 is actually zookeeper free, so this is not new
> to
> > this client it is true for the current client as well. Our experience was
> > that direct zookeeper connections from zillions of producers wasn't a
> good
> > idea for a number of reasons. Our intention is to remove this dependency
> > from the consumer as well. The configuration in the producer doesn't need
> > the full set of brokers, though, just one or two machines to bootstrap
> the
> > state of the cluster from--in other words it isn't like you need to
> > reconfigure your clients every time you add some servers. This is exactly
> > how zookeeper works too--if we used zookeeper you would need to give a
> list
> > of zk urls in case a particular zk server was down. Basically either way
> > you need a few statically configured nodes to go to discover the full
> state
> > of the cluster. For people who don't like hard coding hosts you can use a
> > VIP or dns or something instead.
> >
> > 2. Yes this is a good point and was a concern I had too--the current
> > behavior is that with bad urls the client would start normally and then
> > hang trying to fetch metadata when the first message is sent and finally
> > give up and throw an exception. This is not ideal.
> >
> > The challenge is this: we use the bootstrap urls to fetch metadata for
> > particular topics but we don't know which until we start getting messages
> > for them. We have the option of fetching metadata for all topics but the
> > problem is that for a cluster hosting tens of thousands of topics that is
> > actually a ton of data.
> >
> > An alternative that this just made me think of is that we could
> proactively
> > connect to bootstrap urls sequentially until one succeeds when the
> producer
> > is first created and fail fast if we can't establish a connection. This
> > would not be wasted work as we could use the connection for the metadata
> > request when the first message is sent. I like this solution and will
> > implement it. So thanks for asking!
> >
> > -Jay
> >
> >
> >
> > On Fri, Jan 24, 2014 at 2:46 PM, Roger Hoover <roger.hoover@gmail.com
> > >wrote:
> >
> > > A couple comments:
> > >
> > > 1) Why does the config use a broker list instead of discovering the
> > brokers
> > > in ZooKeeper?  It doesn't match the HighLevelConsumer API.
> > >
> > > 2) It looks like broker connections are created on demand.  I'm
> wondering
> > > if sometimes you might want to flush out config or network connectivity
> > > issues before pushing the first message through.
> > >
> > > Should there also be a KafkaProducer.connect() or .open() method or
> > > connectAll()?  I guess it would try to connect to all brokers in the
> > > BROKER_LIST_CONFIG
> > >
> > > HTH,
> > >
> > > Roger
> > >
> > >
> > > On Fri, Jan 24, 2014 at 11:54 AM, Jay Kreps <ja...@gmail.com>
> wrote:
> > >
> > > > As mentioned in a previous email we are working on a
> re-implementation
> > of
> > > > the producer. I would like to use this email thread to discuss the
> > > details
> > > > of the public API and the configuration. I would love for us to be
> > > > incredibly picky about this public api now so it is as good as
> possible
> > > and
> > > > we don't need to break it in the future.
> > > >
> > > > The best way to get a feel for the API is actually to take a look at
> > the
> > > > javadoc, my hope is to get the api docs good enough so that it is
> > > > self-explanatory:
> > > >
> > > >
> > >
> >
> http://empathybox.com/kafka-javadoc/index.html?kafka/clients/producer/KafkaProducer.html
> > > >
> > > > Please take a look at this API and give me any thoughts you may have!
> > > >
> > > > It may also be reasonable to take a look at the configs:
> > > >
> > > >
> > >
> >
> http://empathybox.com/kafka-javadoc/kafka/clients/producer/ProducerConfig.html
> > > >
> > > > The actual code is posted here:
> > > > https://issues.apache.org/jira/browse/KAFKA-1227
> > > >
> > > > A few questions or comments to kick things off:
> > > > 1. We need to make a decision on whether serialization of the user's
> > key
> > > > and value should be done by the user (with our api just taking
> byte[])
> > or
> > > > if we should take an object and allow the user to configure a
> > Serializer
> > > > class which we instantiate via reflection. We take the later approach
> > in
> > > > the current producer, and I have carried this through to this
> > prototype.
> > > > The tradeoff I see is this: taking byte[] is actually simpler, the
> user
> > > can
> > > > directly do whatever serialization they like. The complication is
> > > actually
> > > > partitioning. Currently partitioning is done by a similar plug-in api
> > > > (Partitioner) which the user can implement and configure to override
> > how
> > > > partitions are assigned. If we take byte[] as input then we have no
> > > access
> > > > to the original object and partitioning MUST be done on the byte[].
> > This
> > > is
> > > > fine for hash partitioning. However for various types of semantic
> > > > partitioning (range partitioning, or whatever) you would want access
> to
> > > the
> > > > original object. In the current approach a producer who wishes to
> send
> > > > byte[] they have serialized in their own code can configure the
> > > > BytesSerialization we supply which is just a "no op" serialization.
> > > > 2. We should obsess over naming and make sure each of the class names
> > are
> > > > good.
> > > > 3. Jun has already pointed out that we need to include the topic and
> > > > partition in the response, which is absolutely right. I haven't done
> > that
> > > > yet but that definitely needs to be there.
> > > > 4. Currently RecordSend.await will throw an exception if the request
> > > > failed. The intention here is that producer.send(message).await()
> > exactly
> > > > simulates a synchronous call. Guozhang has noted that this is a
> little
> > > > annoying since the user must then catch exceptions. However if we
> > remove
> > > > this then if the user doesn't check for errors they won't know one
> has
> > > > occurred, which I predict will be a common mistake.
> > > > 5. Perhaps there is more we could do to make the async callbacks and
> > > future
> > > > we give back intuitive and easy to program against?
> > > >
> > > > Some background info on implementation:
> > > >
> > > > At a high level the primary difference in this producer is that it
> > > removes
> > > > the distinction between the "sync" and "async" producer. Effectively
> > all
> > > > requests are sent asynchronously but always return a future response
> > > object
> > > > that gives the offset as well as any error that may have occurred
> when
> > > the
> > > > request is complete. The batching that is done in the async producer
> > only
> > > > today is done whenever possible now. This means that the sync
> producer,
> > > > under load, can get performance as good as the async producer
> > > (preliminary
> > > > results show the producer getting 1m messages/sec). This works
> similar
> > to
> > > > group commit in databases but with respect to the actual network
> > > > transmission--any messages that arrive while a send is in progress
> are
> > > > batched together. It is also possible to encourage batching even
> under
> > > low
> > > > load to save server resources by introducing a delay on the send to
> > allow
> > > > more messages to accumulate; this is done using the linger.ms config
> > > (this
> > > > is similar to Nagle's algorithm in TCP).
> > > >
> > > > This producer does all network communication asynchronously and in
> > > parallel
> > > > to all servers so the performance penalty for acks=-1 and waiting on
> > > > replication should be much reduced. I haven't done much benchmarking
> on
> > > > this yet, though.
> > > >
> > > > The high level design is described a little here, though this is now
> a
> > > > little out of date:
> > > > https://cwiki.apache.org/confluence/display/KAFKA/Client+Rewrite
> > > >
> > > > -Jay
> > > >
> > >
> >
>

Re: New Producer Public API

Posted by Roger Hoover <ro...@gmail.com>.
Jay,

Thanks for the explanation.  I didn't realize that the broker list was for
bootstrapping and was not required to be a complete list of all brokers
(although I see now that it's clearly stated in the text description of the
parameter).  Nonetheless, does it still make sense to make the config
parameter more clear?  Instead of BROKER_LIST_CONFIG, it could be something
like BROKER_LIST_INITIAL_CONFIG or BROKER_DISCOVERY_LIST_CONFIG or
BROKER_BOOTSTRAP_LIST_CONFIG?

 I like the idea of proactively checking that at least one broker url is
working and failing fast if it is not.  My 2 cents is that it should be
triggered by a method call like initialize() rather than doing it in the
constructor.  Sometimes for unit tests or other purposes, you want to be
able to create objects without triggering network dependencies.

Cheers,

Roger


On Fri, Jan 24, 2014 at 4:13 PM, Jay Kreps <ja...@gmail.com> wrote:

> Roger,
>
> These are good questions.
>
> 1. The producer since 0.8 is actually zookeeper free, so this is not new to
> this client it is true for the current client as well. Our experience was
> that direct zookeeper connections from zillions of producers wasn't a good
> idea for a number of reasons. Our intention is to remove this dependency
> from the consumer as well. The configuration in the producer doesn't need
> the full set of brokers, though, just one or two machines to bootstrap the
> state of the cluster from--in other words it isn't like you need to
> reconfigure your clients every time you add some servers. This is exactly
> how zookeeper works too--if we used zookeeper you would need to give a list
> of zk urls in case a particular zk server was down. Basically either way
> you need a few statically configured nodes to go to discover the full state
> of the cluster. For people who don't like hard coding hosts you can use a
> VIP or dns or something instead.
>
> 2. Yes this is a good point and was a concern I had too--the current
> behavior is that with bad urls the client would start normally and then
> hang trying to fetch metadata when the first message is sent and finally
> give up and throw an exception. This is not ideal.
>
> The challenge is this: we use the bootstrap urls to fetch metadata for
> particular topics but we don't know which until we start getting messages
> for them. We have the option of fetching metadata for all topics but the
> problem is that for a cluster hosting tens of thousands of topics that is
> actually a ton of data.
>
> An alternative that this just made me think of is that we could proactively
> connect to bootstrap urls sequentially until one succeeds when the producer
> is first created and fail fast if we can't establish a connection. This
> would not be wasted work as we could use the connection for the metadata
> request when the first message is sent. I like this solution and will
> implement it. So thanks for asking!
>
> -Jay
>
>
>
> On Fri, Jan 24, 2014 at 2:46 PM, Roger Hoover <roger.hoover@gmail.com
> >wrote:
>
> > A couple comments:
> >
> > 1) Why does the config use a broker list instead of discovering the
> brokers
> > in ZooKeeper?  It doesn't match the HighLevelConsumer API.
> >
> > 2) It looks like broker connections are created on demand.  I'm wondering
> > if sometimes you might want to flush out config or network connectivity
> > issues before pushing the first message through.
> >
> > Should there also be a KafkaProducer.connect() or .open() method or
> > connectAll()?  I guess it would try to connect to all brokers in the
> > BROKER_LIST_CONFIG
> >
> > HTH,
> >
> > Roger
> >
> >
> > On Fri, Jan 24, 2014 at 11:54 AM, Jay Kreps <ja...@gmail.com> wrote:
> >
> > > As mentioned in a previous email we are working on a re-implementation
> of
> > > the producer. I would like to use this email thread to discuss the
> > details
> > > of the public API and the configuration. I would love for us to be
> > > incredibly picky about this public api now so it is as good as possible
> > and
> > > we don't need to break it in the future.
> > >
> > > The best way to get a feel for the API is actually to take a look at
> the
> > > javadoc, my hope is to get the api docs good enough so that it is
> > > self-explanatory:
> > >
> > >
> >
> http://empathybox.com/kafka-javadoc/index.html?kafka/clients/producer/KafkaProducer.html
> > >
> > > Please take a look at this API and give me any thoughts you may have!
> > >
> > > It may also be reasonable to take a look at the configs:
> > >
> > >
> >
> http://empathybox.com/kafka-javadoc/kafka/clients/producer/ProducerConfig.html
> > >
> > > The actual code is posted here:
> > > https://issues.apache.org/jira/browse/KAFKA-1227
> > >
> > > A few questions or comments to kick things off:
> > > 1. We need to make a decision on whether serialization of the user's
> key
> > > and value should be done by the user (with our api just taking byte[])
> or
> > > if we should take an object and allow the user to configure a
> Serializer
> > > class which we instantiate via reflection. We take the later approach
> in
> > > the current producer, and I have carried this through to this
> prototype.
> > > The tradeoff I see is this: taking byte[] is actually simpler, the user
> > can
> > > directly do whatever serialization they like. The complication is
> > actually
> > > partitioning. Currently partitioning is done by a similar plug-in api
> > > (Partitioner) which the user can implement and configure to override
> how
> > > partitions are assigned. If we take byte[] as input then we have no
> > access
> > > to the original object and partitioning MUST be done on the byte[].
> This
> > is
> > > fine for hash partitioning. However for various types of semantic
> > > partitioning (range partitioning, or whatever) you would want access to
> > the
> > > original object. In the current approach a producer who wishes to send
> > > byte[] they have serialized in their own code can configure the
> > > BytesSerialization we supply which is just a "no op" serialization.
> > > 2. We should obsess over naming and make sure each of the class names
> are
> > > good.
> > > 3. Jun has already pointed out that we need to include the topic and
> > > partition in the response, which is absolutely right. I haven't done
> that
> > > yet but that definitely needs to be there.
> > > 4. Currently RecordSend.await will throw an exception if the request
> > > failed. The intention here is that producer.send(message).await()
> exactly
> > > simulates a synchronous call. Guozhang has noted that this is a little
> > > annoying since the user must then catch exceptions. However if we
> remove
> > > this then if the user doesn't check for errors they won't know one has
> > > occurred, which I predict will be a common mistake.
> > > 5. Perhaps there is more we could do to make the async callbacks and
> > future
> > > we give back intuitive and easy to program against?
> > >
> > > Some background info on implementation:
> > >
> > > At a high level the primary difference in this producer is that it
> > removes
> > > the distinction between the "sync" and "async" producer. Effectively
> all
> > > requests are sent asynchronously but always return a future response
> > object
> > > that gives the offset as well as any error that may have occurred when
> > the
> > > request is complete. The batching that is done in the async producer
> only
> > > today is done whenever possible now. This means that the sync producer,
> > > under load, can get performance as good as the async producer
> > (preliminary
> > > results show the producer getting 1m messages/sec). This works similar
> to
> > > group commit in databases but with respect to the actual network
> > > transmission--any messages that arrive while a send is in progress are
> > > batched together. It is also possible to encourage batching even under
> > low
> > > load to save server resources by introducing a delay on the send to
> allow
> > > more messages to accumulate; this is done using the linger.ms config
> > (this
> > > is similar to Nagle's algorithm in TCP).
> > >
> > > This producer does all network communication asynchronously and in
> > parallel
> > > to all servers so the performance penalty for acks=-1 and waiting on
> > > replication should be much reduced. I haven't done much benchmarking on
> > > this yet, though.
> > >
> > > The high level design is described a little here, though this is now a
> > > little out of date:
> > > https://cwiki.apache.org/confluence/display/KAFKA/Client+Rewrite
> > >
> > > -Jay
> > >
> >
>

Re: New Producer Public API

Posted by Roger Hoover <ro...@gmail.com>.
Jay,

Thanks for the explanation.  I didn't realize that the broker list was for
bootstrapping and was not required to be a complete list of all brokers
(although I see now that it's clearly stated in the text description of the
parameter).  Nonetheless, does it still make sense to make the config
parameter more clear?  Instead of BROKER_LIST_CONFIG, it could be something
like BROKER_LIST_INITIAL_CONFIG or BROKER_DISCOVERY_LIST_CONFIG or
BROKER_BOOTSTRAP_LIST_CONFIG?

 I like the idea of proactively checking that at least one broker url is
working and failing fast if it is not.  My 2 cents is that it should be
triggered by a method call like initialize() rather than doing it in the
constructor.  Sometimes for unit tests or other purposes, you want to be
able to create objects without triggering network dependencies.

Cheers,

Roger


On Fri, Jan 24, 2014 at 4:13 PM, Jay Kreps <ja...@gmail.com> wrote:

> Roger,
>
> These are good questions.
>
> 1. The producer since 0.8 is actually zookeeper free, so this is not new to
> this client it is true for the current client as well. Our experience was
> that direct zookeeper connections from zillions of producers wasn't a good
> idea for a number of reasons. Our intention is to remove this dependency
> from the consumer as well. The configuration in the producer doesn't need
> the full set of brokers, though, just one or two machines to bootstrap the
> state of the cluster from--in other words it isn't like you need to
> reconfigure your clients every time you add some servers. This is exactly
> how zookeeper works too--if we used zookeeper you would need to give a list
> of zk urls in case a particular zk server was down. Basically either way
> you need a few statically configured nodes to go to discover the full state
> of the cluster. For people who don't like hard coding hosts you can use a
> VIP or dns or something instead.
>
> 2. Yes this is a good point and was a concern I had too--the current
> behavior is that with bad urls the client would start normally and then
> hang trying to fetch metadata when the first message is sent and finally
> give up and throw an exception. This is not ideal.
>
> The challenge is this: we use the bootstrap urls to fetch metadata for
> particular topics but we don't know which until we start getting messages
> for them. We have the option of fetching metadata for all topics but the
> problem is that for a cluster hosting tens of thousands of topics that is
> actually a ton of data.
>
> An alternative that this just made me think of is that we could proactively
> connect to bootstrap urls sequentially until one succeeds when the producer
> is first created and fail fast if we can't establish a connection. This
> would not be wasted work as we could use the connection for the metadata
> request when the first message is sent. I like this solution and will
> implement it. So thanks for asking!
>
> -Jay
>
>
>
> On Fri, Jan 24, 2014 at 2:46 PM, Roger Hoover <roger.hoover@gmail.com
> >wrote:
>
> > A couple comments:
> >
> > 1) Why does the config use a broker list instead of discovering the
> brokers
> > in ZooKeeper?  It doesn't match the HighLevelConsumer API.
> >
> > 2) It looks like broker connections are created on demand.  I'm wondering
> > if sometimes you might want to flush out config or network connectivity
> > issues before pushing the first message through.
> >
> > Should there also be a KafkaProducer.connect() or .open() method or
> > connectAll()?  I guess it would try to connect to all brokers in the
> > BROKER_LIST_CONFIG
> >
> > HTH,
> >
> > Roger
> >
> >
> > On Fri, Jan 24, 2014 at 11:54 AM, Jay Kreps <ja...@gmail.com> wrote:
> >
> > > As mentioned in a previous email we are working on a re-implementation
> of
> > > the producer. I would like to use this email thread to discuss the
> > details
> > > of the public API and the configuration. I would love for us to be
> > > incredibly picky about this public api now so it is as good as possible
> > and
> > > we don't need to break it in the future.
> > >
> > > The best way to get a feel for the API is actually to take a look at
> the
> > > javadoc, my hope is to get the api docs good enough so that it is
> > > self-explanatory:
> > >
> > >
> >
> http://empathybox.com/kafka-javadoc/index.html?kafka/clients/producer/KafkaProducer.html
> > >
> > > Please take a look at this API and give me any thoughts you may have!
> > >
> > > It may also be reasonable to take a look at the configs:
> > >
> > >
> >
> http://empathybox.com/kafka-javadoc/kafka/clients/producer/ProducerConfig.html
> > >
> > > The actual code is posted here:
> > > https://issues.apache.org/jira/browse/KAFKA-1227
> > >
> > > A few questions or comments to kick things off:
> > > 1. We need to make a decision on whether serialization of the user's
> key
> > > and value should be done by the user (with our api just taking byte[])
> or
> > > if we should take an object and allow the user to configure a
> Serializer
> > > class which we instantiate via reflection. We take the later approach
> in
> > > the current producer, and I have carried this through to this
> prototype.
> > > The tradeoff I see is this: taking byte[] is actually simpler, the user
> > can
> > > directly do whatever serialization they like. The complication is
> > actually
> > > partitioning. Currently partitioning is done by a similar plug-in api
> > > (Partitioner) which the user can implement and configure to override
> how
> > > partitions are assigned. If we take byte[] as input then we have no
> > access
> > > to the original object and partitioning MUST be done on the byte[].
> This
> > is
> > > fine for hash partitioning. However for various types of semantic
> > > partitioning (range partitioning, or whatever) you would want access to
> > the
> > > original object. In the current approach a producer who wishes to send
> > > byte[] they have serialized in their own code can configure the
> > > BytesSerialization we supply which is just a "no op" serialization.
> > > 2. We should obsess over naming and make sure each of the class names
> are
> > > good.
> > > 3. Jun has already pointed out that we need to include the topic and
> > > partition in the response, which is absolutely right. I haven't done
> that
> > > yet but that definitely needs to be there.
> > > 4. Currently RecordSend.await will throw an exception if the request
> > > failed. The intention here is that producer.send(message).await()
> exactly
> > > simulates a synchronous call. Guozhang has noted that this is a little
> > > annoying since the user must then catch exceptions. However if we
> remove
> > > this then if the user doesn't check for errors they won't know one has
> > > occurred, which I predict will be a common mistake.
> > > 5. Perhaps there is more we could do to make the async callbacks and
> > future
> > > we give back intuitive and easy to program against?
> > >
> > > Some background info on implementation:
> > >
> > > At a high level the primary difference in this producer is that it
> > removes
> > > the distinction between the "sync" and "async" producer. Effectively
> all
> > > requests are sent asynchronously but always return a future response
> > object
> > > that gives the offset as well as any error that may have occurred when
> > the
> > > request is complete. The batching that is done in the async producer
> only
> > > today is done whenever possible now. This means that the sync producer,
> > > under load, can get performance as good as the async producer
> > (preliminary
> > > results show the producer getting 1m messages/sec). This works similar
> to
> > > group commit in databases but with respect to the actual network
> > > transmission--any messages that arrive while a send is in progress are
> > > batched together. It is also possible to encourage batching even under
> > low
> > > load to save server resources by introducing a delay on the send to
> allow
> > > more messages to accumulate; this is done using the linger.ms config
> > (this
> > > is similar to Nagle's algorithm in TCP).
> > >
> > > This producer does all network communication asynchronously and in
> > parallel
> > > to all servers so the performance penalty for acks=-1 and waiting on
> > > replication should be much reduced. I haven't done much benchmarking on
> > > this yet, though.
> > >
> > > The high level design is described a little here, though this is now a
> > > little out of date:
> > > https://cwiki.apache.org/confluence/display/KAFKA/Client+Rewrite
> > >
> > > -Jay
> > >
> >
>

Re: New Producer Public API

Posted by Scott Clasen <sc...@heroku.com>.
+1  to zk bootstrap + close as an option at least


On Tue, Jan 28, 2014 at 10:09 AM, Neha Narkhede <ne...@gmail.com>wrote:

> >> The producer since 0.8 is actually zookeeper free, so this is not new to
> this client it is true for the current client as well. Our experience was
> that direct zookeeper connections from zillions of producers wasn't a good
> idea for a number of reasons.
>
> The problem with several thousand connections to zookeeper is mainly the
> long lived sessions causing overhead on zookeeper.
> This further degrades zookeeper performance causing it to be flaky and
> expire sessions/disconnect clients and so on. That being said,
> I don't see why we can't use zookeeper *just* for the bootstrap on client
> startup and close the connection right after the bootstrap is done.
> IMO, this is more intuitive and convenient as it will allow users to the
> same "bootstrap config" across producers, consumers and brokers and
> will not cause any performance/operational issues on zookeeper. This is
> assuming that all the zillion clients don't bootstrap at the same time,
> which is rare in practice.
>
> Thanks,
> Neha
>
>
> On Tue, Jan 28, 2014 at 8:02 AM, Mattijs Ugen (DT) <mattijs@holmes.nl
> >wrote:
>
> > Sorry to tune in a bit late, but here goes.
> >
> > > 1. The producer since 0.8 is actually zookeeper free, so this is not
> new
> > to
> > > this client it is true for the current client as well. Our experience
> was
> > > that direct zookeeper connections from zillions of producers wasn't a
> > good
> > > idea for a number of reasons. Our intention is to remove this
> dependency
> > > from the consumer as well. The configuration in the producer doesn't
> need
> > > the full set of brokers, though, just one or two machines to bootstrap
> > the
> > > state of the cluster from--in other words it isn't like you need to
> > > reconfigure your clients every time you add some servers. This is
> exactly
> > > how zookeeper works too--if we used zookeeper you would need to give a
> > list
> > > of zk urls in case a particular zk server was down. Basically either
> way
> > > you need a few statically configured nodes to go to discover the full
> > state
> > > of the cluster. For people who don't like hard coding hosts you can
> use a
> > > VIP or dns or something instead.
> > In our configuration, the zookeeper quorum is actually one of the few
> > stable (in the sense of host names / ip addresses) pillars of the
> > complete ecosystem: every distributed service uses zookeeper to
> > coordinate the hosts that make up the service as a whole. Considering
> > that the kafka cluster will save the information needed for this
> > bootstrap to zookeeper anyhow, having clients (either producers or
> > consumers) retrieve this information at first use makes sense to me.
> >
> > We could create routine that retrieves a list of brokers from zookeeper
> > before initializing a Producer, but that feels more like a workaround
> > for a feature that in my humble opinion could well be part of the kafka
> > client library. That said, I realise that having two options for
> > connection bootstrapping (assuming that hardcoding a list of brokers is
> > here to stay) could be confusing for new users, but bypassing zookeeper
> > for this was rather confusing for me when I first came across it :)
> >
> > So, in short, I'd love it if the option to bootstrap the broker list
> > from zookeeper was there, rather than requiring to configure additional
> > (moving) virtual hostnames or fixed ip addresses for producers in our
> > cluster setup. I've been baffled a few times by this option not being
> > available for a distributed service that coordinates itself through
> > zookeeper.
> >
> > Just my two cents :)
> >
> > Mattijs
> >
>

Re: New Producer Public API

Posted by Jay Kreps <ja...@gmail.com>.
Hmmm, I would really strongly urge us to not introduce a zk dependency just
for discovery. People who want to implement this can certainly do so by
simply looking up urls and setting them in the consumer config, but our
experience with doing this at large scale was pretty bad. Hardcoding the
discovery broker URLS shouldn't be worse than hardcoding the zk urls, and
folks who want to avoid that can use DNS or a vip. I think this will be
better in every way for 99% of people.

-Jay

On Tue, Jan 28, 2014 at 10:09 AM, Neha Narkhede <ne...@gmail.com>wrote:

> >> The producer since 0.8 is actually zookeeper free, so this is not new to
> this client it is true for the current client as well. Our experience was
> that direct zookeeper connections from zillions of producers wasn't a good
> idea for a number of reasons.
>
> The problem with several thousand connections to zookeeper is mainly the
> long lived sessions causing overhead on zookeeper.
> This further degrades zookeeper performance causing it to be flaky and
> expire sessions/disconnect clients and so on. That being said,
> I don't see why we can't use zookeeper *just* for the bootstrap on client
> startup and close the connection right after the bootstrap is done.
> IMO, this is more intuitive and convenient as it will allow users to the
> same "bootstrap config" across producers, consumers and brokers and
> will not cause any performance/operational issues on zookeeper. This is
> assuming that all the zillion clients don't bootstrap at the same time,
> which is rare in practice.
>
> Thanks,
> Neha
>
>
> On Tue, Jan 28, 2014 at 8:02 AM, Mattijs Ugen (DT) <mattijs@holmes.nl
> >wrote:
>
> > Sorry to tune in a bit late, but here goes.
> >
> > > 1. The producer since 0.8 is actually zookeeper free, so this is not
> new
> > to
> > > this client it is true for the current client as well. Our experience
> was
> > > that direct zookeeper connections from zillions of producers wasn't a
> > good
> > > idea for a number of reasons. Our intention is to remove this
> dependency
> > > from the consumer as well. The configuration in the producer doesn't
> need
> > > the full set of brokers, though, just one or two machines to bootstrap
> > the
> > > state of the cluster from--in other words it isn't like you need to
> > > reconfigure your clients every time you add some servers. This is
> exactly
> > > how zookeeper works too--if we used zookeeper you would need to give a
> > list
> > > of zk urls in case a particular zk server was down. Basically either
> way
> > > you need a few statically configured nodes to go to discover the full
> > state
> > > of the cluster. For people who don't like hard coding hosts you can
> use a
> > > VIP or dns or something instead.
> > In our configuration, the zookeeper quorum is actually one of the few
> > stable (in the sense of host names / ip addresses) pillars of the
> > complete ecosystem: every distributed service uses zookeeper to
> > coordinate the hosts that make up the service as a whole. Considering
> > that the kafka cluster will save the information needed for this
> > bootstrap to zookeeper anyhow, having clients (either producers or
> > consumers) retrieve this information at first use makes sense to me.
> >
> > We could create routine that retrieves a list of brokers from zookeeper
> > before initializing a Producer, but that feels more like a workaround
> > for a feature that in my humble opinion could well be part of the kafka
> > client library. That said, I realise that having two options for
> > connection bootstrapping (assuming that hardcoding a list of brokers is
> > here to stay) could be confusing for new users, but bypassing zookeeper
> > for this was rather confusing for me when I first came across it :)
> >
> > So, in short, I'd love it if the option to bootstrap the broker list
> > from zookeeper was there, rather than requiring to configure additional
> > (moving) virtual hostnames or fixed ip addresses for producers in our
> > cluster setup. I've been baffled a few times by this option not being
> > available for a distributed service that coordinates itself through
> > zookeeper.
> >
> > Just my two cents :)
> >
> > Mattijs
> >
>

Re: New Producer Public API

Posted by Neha Narkhede <ne...@gmail.com>.
>> The producer since 0.8 is actually zookeeper free, so this is not new to
this client it is true for the current client as well. Our experience was
that direct zookeeper connections from zillions of producers wasn't a good
idea for a number of reasons.

The problem with several thousand connections to zookeeper is mainly the
long lived sessions causing overhead on zookeeper.
This further degrades zookeeper performance causing it to be flaky and
expire sessions/disconnect clients and so on. That being said,
I don't see why we can't use zookeeper *just* for the bootstrap on client
startup and close the connection right after the bootstrap is done.
IMO, this is more intuitive and convenient as it will allow users to the
same "bootstrap config" across producers, consumers and brokers and
will not cause any performance/operational issues on zookeeper. This is
assuming that all the zillion clients don't bootstrap at the same time,
which is rare in practice.

Thanks,
Neha


On Tue, Jan 28, 2014 at 8:02 AM, Mattijs Ugen (DT) <ma...@holmes.nl>wrote:

> Sorry to tune in a bit late, but here goes.
>
> > 1. The producer since 0.8 is actually zookeeper free, so this is not new
> to
> > this client it is true for the current client as well. Our experience was
> > that direct zookeeper connections from zillions of producers wasn't a
> good
> > idea for a number of reasons. Our intention is to remove this dependency
> > from the consumer as well. The configuration in the producer doesn't need
> > the full set of brokers, though, just one or two machines to bootstrap
> the
> > state of the cluster from--in other words it isn't like you need to
> > reconfigure your clients every time you add some servers. This is exactly
> > how zookeeper works too--if we used zookeeper you would need to give a
> list
> > of zk urls in case a particular zk server was down. Basically either way
> > you need a few statically configured nodes to go to discover the full
> state
> > of the cluster. For people who don't like hard coding hosts you can use a
> > VIP or dns or something instead.
> In our configuration, the zookeeper quorum is actually one of the few
> stable (in the sense of host names / ip addresses) pillars of the
> complete ecosystem: every distributed service uses zookeeper to
> coordinate the hosts that make up the service as a whole. Considering
> that the kafka cluster will save the information needed for this
> bootstrap to zookeeper anyhow, having clients (either producers or
> consumers) retrieve this information at first use makes sense to me.
>
> We could create routine that retrieves a list of brokers from zookeeper
> before initializing a Producer, but that feels more like a workaround
> for a feature that in my humble opinion could well be part of the kafka
> client library. That said, I realise that having two options for
> connection bootstrapping (assuming that hardcoding a list of brokers is
> here to stay) could be confusing for new users, but bypassing zookeeper
> for this was rather confusing for me when I first came across it :)
>
> So, in short, I'd love it if the option to bootstrap the broker list
> from zookeeper was there, rather than requiring to configure additional
> (moving) virtual hostnames or fixed ip addresses for producers in our
> cluster setup. I've been baffled a few times by this option not being
> available for a distributed service that coordinates itself through
> zookeeper.
>
> Just my two cents :)
>
> Mattijs
>

Re: New Producer Public API

Posted by "Mattijs Ugen (DT)" <ma...@holmes.nl>.
Sorry to tune in a bit late, but here goes.

> 1. The producer since 0.8 is actually zookeeper free, so this is not new to
> this client it is true for the current client as well. Our experience was
> that direct zookeeper connections from zillions of producers wasn't a good
> idea for a number of reasons. Our intention is to remove this dependency
> from the consumer as well. The configuration in the producer doesn't need
> the full set of brokers, though, just one or two machines to bootstrap the
> state of the cluster from--in other words it isn't like you need to
> reconfigure your clients every time you add some servers. This is exactly
> how zookeeper works too--if we used zookeeper you would need to give a list
> of zk urls in case a particular zk server was down. Basically either way
> you need a few statically configured nodes to go to discover the full state
> of the cluster. For people who don't like hard coding hosts you can use a
> VIP or dns or something instead.
In our configuration, the zookeeper quorum is actually one of the few
stable (in the sense of host names / ip addresses) pillars of the
complete ecosystem: every distributed service uses zookeeper to
coordinate the hosts that make up the service as a whole. Considering
that the kafka cluster will save the information needed for this
bootstrap to zookeeper anyhow, having clients (either producers or
consumers) retrieve this information at first use makes sense to me.

We could create routine that retrieves a list of brokers from zookeeper
before initializing a Producer, but that feels more like a workaround
for a feature that in my humble opinion could well be part of the kafka
client library. That said, I realise that having two options for
connection bootstrapping (assuming that hardcoding a list of brokers is
here to stay) could be confusing for new users, but bypassing zookeeper
for this was rather confusing for me when I first came across it :)

So, in short, I'd love it if the option to bootstrap the broker list
from zookeeper was there, rather than requiring to configure additional
(moving) virtual hostnames or fixed ip addresses for producers in our
cluster setup. I've been baffled a few times by this option not being
available for a distributed service that coordinates itself through
zookeeper.

Just my two cents :)

Mattijs

Re: New Producer Public API

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

These are good questions.

1. The producer since 0.8 is actually zookeeper free, so this is not new to
this client it is true for the current client as well. Our experience was
that direct zookeeper connections from zillions of producers wasn't a good
idea for a number of reasons. Our intention is to remove this dependency
from the consumer as well. The configuration in the producer doesn't need
the full set of brokers, though, just one or two machines to bootstrap the
state of the cluster from--in other words it isn't like you need to
reconfigure your clients every time you add some servers. This is exactly
how zookeeper works too--if we used zookeeper you would need to give a list
of zk urls in case a particular zk server was down. Basically either way
you need a few statically configured nodes to go to discover the full state
of the cluster. For people who don't like hard coding hosts you can use a
VIP or dns or something instead.

2. Yes this is a good point and was a concern I had too--the current
behavior is that with bad urls the client would start normally and then
hang trying to fetch metadata when the first message is sent and finally
give up and throw an exception. This is not ideal.

The challenge is this: we use the bootstrap urls to fetch metadata for
particular topics but we don't know which until we start getting messages
for them. We have the option of fetching metadata for all topics but the
problem is that for a cluster hosting tens of thousands of topics that is
actually a ton of data.

An alternative that this just made me think of is that we could proactively
connect to bootstrap urls sequentially until one succeeds when the producer
is first created and fail fast if we can't establish a connection. This
would not be wasted work as we could use the connection for the metadata
request when the first message is sent. I like this solution and will
implement it. So thanks for asking!

-Jay



On Fri, Jan 24, 2014 at 2:46 PM, Roger Hoover <ro...@gmail.com>wrote:

> A couple comments:
>
> 1) Why does the config use a broker list instead of discovering the brokers
> in ZooKeeper?  It doesn't match the HighLevelConsumer API.
>
> 2) It looks like broker connections are created on demand.  I'm wondering
> if sometimes you might want to flush out config or network connectivity
> issues before pushing the first message through.
>
> Should there also be a KafkaProducer.connect() or .open() method or
> connectAll()?  I guess it would try to connect to all brokers in the
> BROKER_LIST_CONFIG
>
> HTH,
>
> Roger
>
>
> On Fri, Jan 24, 2014 at 11:54 AM, Jay Kreps <ja...@gmail.com> wrote:
>
> > As mentioned in a previous email we are working on a re-implementation of
> > the producer. I would like to use this email thread to discuss the
> details
> > of the public API and the configuration. I would love for us to be
> > incredibly picky about this public api now so it is as good as possible
> and
> > we don't need to break it in the future.
> >
> > The best way to get a feel for the API is actually to take a look at the
> > javadoc, my hope is to get the api docs good enough so that it is
> > self-explanatory:
> >
> >
> http://empathybox.com/kafka-javadoc/index.html?kafka/clients/producer/KafkaProducer.html
> >
> > Please take a look at this API and give me any thoughts you may have!
> >
> > It may also be reasonable to take a look at the configs:
> >
> >
> http://empathybox.com/kafka-javadoc/kafka/clients/producer/ProducerConfig.html
> >
> > The actual code is posted here:
> > https://issues.apache.org/jira/browse/KAFKA-1227
> >
> > A few questions or comments to kick things off:
> > 1. We need to make a decision on whether serialization of the user's key
> > and value should be done by the user (with our api just taking byte[]) or
> > if we should take an object and allow the user to configure a Serializer
> > class which we instantiate via reflection. We take the later approach in
> > the current producer, and I have carried this through to this prototype.
> > The tradeoff I see is this: taking byte[] is actually simpler, the user
> can
> > directly do whatever serialization they like. The complication is
> actually
> > partitioning. Currently partitioning is done by a similar plug-in api
> > (Partitioner) which the user can implement and configure to override how
> > partitions are assigned. If we take byte[] as input then we have no
> access
> > to the original object and partitioning MUST be done on the byte[]. This
> is
> > fine for hash partitioning. However for various types of semantic
> > partitioning (range partitioning, or whatever) you would want access to
> the
> > original object. In the current approach a producer who wishes to send
> > byte[] they have serialized in their own code can configure the
> > BytesSerialization we supply which is just a "no op" serialization.
> > 2. We should obsess over naming and make sure each of the class names are
> > good.
> > 3. Jun has already pointed out that we need to include the topic and
> > partition in the response, which is absolutely right. I haven't done that
> > yet but that definitely needs to be there.
> > 4. Currently RecordSend.await will throw an exception if the request
> > failed. The intention here is that producer.send(message).await() exactly
> > simulates a synchronous call. Guozhang has noted that this is a little
> > annoying since the user must then catch exceptions. However if we remove
> > this then if the user doesn't check for errors they won't know one has
> > occurred, which I predict will be a common mistake.
> > 5. Perhaps there is more we could do to make the async callbacks and
> future
> > we give back intuitive and easy to program against?
> >
> > Some background info on implementation:
> >
> > At a high level the primary difference in this producer is that it
> removes
> > the distinction between the "sync" and "async" producer. Effectively all
> > requests are sent asynchronously but always return a future response
> object
> > that gives the offset as well as any error that may have occurred when
> the
> > request is complete. The batching that is done in the async producer only
> > today is done whenever possible now. This means that the sync producer,
> > under load, can get performance as good as the async producer
> (preliminary
> > results show the producer getting 1m messages/sec). This works similar to
> > group commit in databases but with respect to the actual network
> > transmission--any messages that arrive while a send is in progress are
> > batched together. It is also possible to encourage batching even under
> low
> > load to save server resources by introducing a delay on the send to allow
> > more messages to accumulate; this is done using the linger.ms config
> (this
> > is similar to Nagle's algorithm in TCP).
> >
> > This producer does all network communication asynchronously and in
> parallel
> > to all servers so the performance penalty for acks=-1 and waiting on
> > replication should be much reduced. I haven't done much benchmarking on
> > this yet, though.
> >
> > The high level design is described a little here, though this is now a
> > little out of date:
> > https://cwiki.apache.org/confluence/display/KAFKA/Client+Rewrite
> >
> > -Jay
> >
>

Re: New Producer Public API

Posted by Jay Kreps <ja...@gmail.com>.
I thought a bit about it and I think the getCluster() thing was overly
simplistic because we try to only maintain metadata about the current set
of topics the producer cares about so the cluster might not have the
partitions for the topic the user cares about. I think actually what we
need is a new method on producer:
  List<PartitionInfo> partitionsFor(String...topics)
The intended usage of this method would be:
  int partition = myPartitionFunction(key, producer.partitionsFor(topic));
  producer.send(new ProducerRecord(topic, partition, key, value))
That is, the producer would re-fetch the current set of partitions every
time and the partitions would refresh at whatever schedule the producer
metadata refresh was set with.

So in the case of partition change producers would pick this up as their
natural metadata updates occurred.

This actually solves an important problem in the existing api, which is
that the send method will block on the first message send if we don't yet
have metadata for the destination topic. It blocks until metadata is
fetched. This is a little weird as this occurs even in non-blocking mode.
This allows an escape hatch. Someone who wants to avoid that small block on
the first send can initialize their producer and call
producer.partitionsFor(topics) to force metadata initialization.

-Jay



On Thu, Jan 30, 2014 at 4:34 PM, Jun Rao <ju...@gmail.com> wrote:

> With option 1A, if we increase # partitions on a topic, how will the
> producer find out newly created partitions? Do we expect the producer to
> periodically call getCluster()?
>
> As for ZK dependency, one of the goals of client rewrite is to reduce
> dependencies so that one can implement the client in languages other than
> java. ZK client is only available in a small number of languages.
>
> Thanks,
>
> Jun
>
>
> On Fri, Jan 24, 2014 at 5:17 PM, Jay Kreps <ja...@gmail.com> wrote:
>
> > Clark and all,
> >
> > I thought a little bit about the serialization question. Here are the
> > options I see and the pros and cons I can think of. I'd love to hear
> > people's preferences if you have a strong one.
> >
> > One important consideration is that however the producer works will also
> > need to be how the new consumer works (which we hope to write next). That
> > is if you put objects in, you should get objects out. So we need to think
> > through both sides.
> >
> > Options:
> >
> > Option 0: What is in the existing scala code and the java code I
> > posted--Serializer and Partitioner plugin provided by the user via
> config.
> > Partitioner has a sane default, but Serializer needs to be specified in
> > config.
> >
> > Pros: How it works today in the scala code.
> > Cons: You have to adapt your serialization library of choice to our
> > interfaces. The reflective class loading means typo in the serializer
> name
> > give odd errors. Likewise there is little type safety--the ProducerRecord
> > takes Object and any type errors between the object provided and the
> > serializer give occurs at runtime.
> >
> > Option 1: No plugins
> >
> > This would mean byte[] key, byte[] value, and partitioning done by client
> > by passing in a partition *number* directly.
> >
> > The problem with this is that it is tricky to compute the partition
> > correctly and probably most people won't. We could add a getCluster()
> > method to return the Cluster instance you should use for partitioning.
> But
> > I suspect people would be lazy and not use that and instead hard-code
> > partitions which would break if partitions were added or they hard coded
> it
> > wrong. In my experience 3 partitioning strategies cover like 99% of cases
> > so not having a default implementation for this makes the common case
> > harder. Left to their own devices people will use bad hash functions and
> > get weird results.
> >
> > Option 1A: Alternatively we could partition by the key using the existing
> > default partitioning strategy which only uses the byte[] anyway but
> instead
> > of having a partitionKey we could have a numerical partition override and
> > add the getCluster() method to get the cluster metadata. That would make
> > custom partitioning possible but handle the common case simply.
> >
> > Option 2: Partitioner plugin remains, serializers go.
> >
> > The problem here is that the partitioner might lose access to the
> > deserialized key which would occasionally be useful for semantic
> > partitioning schemes. The Partitioner could deserialize the key but that
> > would be inefficient and weird.
> >
> > This problem could be fixed by having key and value be byte[] but
> retaining
> > partitionKey as an Object and passing it to the partitioner as is. Then
> if
> > you have a partitioner which requires the deserialized key you would need
> > to use this partition key. One weird side effect is that if you want to
> > have a custom partition key BUT want to partition by the bytes of that
> key
> > rather than the object value you must write a customer partitioner and
> > serialize it yourself.
> >
> > Of these I think I prefer 1A but could be convinced of 0 since that is
> how
> > it works now.
> >
> > Thoughts?
> >
> > -Jay
> >
> >
> > On Fri, Jan 24, 2014 at 3:30 PM, Clark Breyman <cl...@breyman.com>
> wrote:
> >
> > > Jay - Thanks for the call for comments. Here's some initial input:
> > >
> > > - Make message serialization a client responsibility (making all
> messages
> > > byte[]). Reflection-based loading makes it harder to use generic codecs
> > > (e.g.  Envelope<PREFIX, DATA, SUFFIX>) or build up codec
> > programmatically.
> > > Non-default partitioning should require an explicit partition key.
> > >
> > > - I really like the fact that it will be native Java. Please consider
> > using
> > > native maven and not sbt, gradle, ivy, etc as they don't reliably play
> > nice
> > > in the maven ecosystem. A jar without a well-formed pom doesn't feel
> > like a
> > > real artifact. The pom's generated by sbt et al. are not well formed.
> > Using
> > > maven will make builds and IDE integration much smoother.
> > >
> > > - Look at Nick Telford's dropwizard-extras package in which he defines
> > some
> > > Jackson-compatible POJO's for loading configuration. Seems like your
> > client
> > > migration is similar. The config objects should have constructors or
> > > factories that accept Map<String, String> and Properties for ease of
> > > migration.
> > >
> > > - Would you consider using the org.apache.kafka package for the new API
> > > (quibble)
> > >
> > > - Why create your own futures rather than use
> > > java.util.concurrent.Future<Long> or similar? Standard futures will
> play
> > > nice with other reactive libs and things like J8's ComposableFuture.
> > >
> > > Thanks again,
> > > C
> > >
> > >
> > >
> > > On Fri, Jan 24, 2014 at 2:46 PM, Roger Hoover <roger.hoover@gmail.com
> > > >wrote:
> > >
> > > > A couple comments:
> > > >
> > > > 1) Why does the config use a broker list instead of discovering the
> > > brokers
> > > > in ZooKeeper?  It doesn't match the HighLevelConsumer API.
> > > >
> > > > 2) It looks like broker connections are created on demand.  I'm
> > wondering
> > > > if sometimes you might want to flush out config or network
> connectivity
> > > > issues before pushing the first message through.
> > > >
> > > > Should there also be a KafkaProducer.connect() or .open() method or
> > > > connectAll()?  I guess it would try to connect to all brokers in the
> > > > BROKER_LIST_CONFIG
> > > >
> > > > HTH,
> > > >
> > > > Roger
> > > >
> > > >
> > > > On Fri, Jan 24, 2014 at 11:54 AM, Jay Kreps <ja...@gmail.com>
> > wrote:
> > > >
> > > > > As mentioned in a previous email we are working on a
> > re-implementation
> > > of
> > > > > the producer. I would like to use this email thread to discuss the
> > > > details
> > > > > of the public API and the configuration. I would love for us to be
> > > > > incredibly picky about this public api now so it is as good as
> > possible
> > > > and
> > > > > we don't need to break it in the future.
> > > > >
> > > > > The best way to get a feel for the API is actually to take a look
> at
> > > the
> > > > > javadoc, my hope is to get the api docs good enough so that it is
> > > > > self-explanatory:
> > > > >
> > > > >
> > > >
> > >
> >
> http://empathybox.com/kafka-javadoc/index.html?kafka/clients/producer/KafkaProducer.html
> > > > >
> > > > > Please take a look at this API and give me any thoughts you may
> have!
> > > > >
> > > > > It may also be reasonable to take a look at the configs:
> > > > >
> > > > >
> > > >
> > >
> >
> http://empathybox.com/kafka-javadoc/kafka/clients/producer/ProducerConfig.html
> > > > >
> > > > > The actual code is posted here:
> > > > > https://issues.apache.org/jira/browse/KAFKA-1227
> > > > >
> > > > > A few questions or comments to kick things off:
> > > > > 1. We need to make a decision on whether serialization of the
> user's
> > > key
> > > > > and value should be done by the user (with our api just taking
> > byte[])
> > > or
> > > > > if we should take an object and allow the user to configure a
> > > Serializer
> > > > > class which we instantiate via reflection. We take the later
> approach
> > > in
> > > > > the current producer, and I have carried this through to this
> > > prototype.
> > > > > The tradeoff I see is this: taking byte[] is actually simpler, the
> > user
> > > > can
> > > > > directly do whatever serialization they like. The complication is
> > > > actually
> > > > > partitioning. Currently partitioning is done by a similar plug-in
> api
> > > > > (Partitioner) which the user can implement and configure to
> override
> > > how
> > > > > partitions are assigned. If we take byte[] as input then we have no
> > > > access
> > > > > to the original object and partitioning MUST be done on the byte[].
> > > This
> > > > is
> > > > > fine for hash partitioning. However for various types of semantic
> > > > > partitioning (range partitioning, or whatever) you would want
> access
> > to
> > > > the
> > > > > original object. In the current approach a producer who wishes to
> > send
> > > > > byte[] they have serialized in their own code can configure the
> > > > > BytesSerialization we supply which is just a "no op" serialization.
> > > > > 2. We should obsess over naming and make sure each of the class
> names
> > > are
> > > > > good.
> > > > > 3. Jun has already pointed out that we need to include the topic
> and
> > > > > partition in the response, which is absolutely right. I haven't
> done
> > > that
> > > > > yet but that definitely needs to be there.
> > > > > 4. Currently RecordSend.await will throw an exception if the
> request
> > > > > failed. The intention here is that producer.send(message).await()
> > > exactly
> > > > > simulates a synchronous call. Guozhang has noted that this is a
> > little
> > > > > annoying since the user must then catch exceptions. However if we
> > > remove
> > > > > this then if the user doesn't check for errors they won't know one
> > has
> > > > > occurred, which I predict will be a common mistake.
> > > > > 5. Perhaps there is more we could do to make the async callbacks
> and
> > > > future
> > > > > we give back intuitive and easy to program against?
> > > > >
> > > > > Some background info on implementation:
> > > > >
> > > > > At a high level the primary difference in this producer is that it
> > > > removes
> > > > > the distinction between the "sync" and "async" producer.
> Effectively
> > > all
> > > > > requests are sent asynchronously but always return a future
> response
> > > > object
> > > > > that gives the offset as well as any error that may have occurred
> > when
> > > > the
> > > > > request is complete. The batching that is done in the async
> producer
> > > only
> > > > > today is done whenever possible now. This means that the sync
> > producer,
> > > > > under load, can get performance as good as the async producer
> > > > (preliminary
> > > > > results show the producer getting 1m messages/sec). This works
> > similar
> > > to
> > > > > group commit in databases but with respect to the actual network
> > > > > transmission--any messages that arrive while a send is in progress
> > are
> > > > > batched together. It is also possible to encourage batching even
> > under
> > > > low
> > > > > load to save server resources by introducing a delay on the send to
> > > allow
> > > > > more messages to accumulate; this is done using the linger.msconfig
> > > > (this
> > > > > is similar to Nagle's algorithm in TCP).
> > > > >
> > > > > This producer does all network communication asynchronously and in
> > > > parallel
> > > > > to all servers so the performance penalty for acks=-1 and waiting
> on
> > > > > replication should be much reduced. I haven't done much
> benchmarking
> > on
> > > > > this yet, though.
> > > > >
> > > > > The high level design is described a little here, though this is
> now
> > a
> > > > > little out of date:
> > > > > https://cwiki.apache.org/confluence/display/KAFKA/Client+Rewrite
> > > > >
> > > > > -Jay
> > > > >
> > > >
> > >
> >
>

Re: New Producer Public API

Posted by David Arthur <mu...@gmail.com>.
On 1/30/14 8:18 PM, Joel Koshy wrote:
> That's a good point about 1A - does seem that we would need to have
> some kind of TTL for each topic's metadata.
>
> Also, WRT ZK dependency I don't think that decision (for the Java
> client) affects other clients. i.e., other client implementations can
> use whatever discovery mechanism it chooses. That said, I prefer not
> having a ZK dependency for the same reasons covered earlier in this
> thread.
FWIW, I think including ZK for broker discovery is a nice feature. Users 
of kafka-python are constantly asking for something like this. If client 
dependencies are a concern, then we could abstract the bootstrap 
strategy into a simple pluggable interface so we could publish a 
separate artifact. I could also imagine some AWS-specific bootstrap 
strategy (e.g., get hosts from a particular security group, load 
balancer/auto-scaling group, etc).

Or, we could just include ZK

-David
>
> On Thu, Jan 30, 2014 at 4:34 PM, Jun Rao <ju...@gmail.com> wrote:
>> With option 1A, if we increase # partitions on a topic, how will the
>> producer find out newly created partitions? Do we expect the producer to
>> periodically call getCluster()?
>>
>> As for ZK dependency, one of the goals of client rewrite is to reduce
>> dependencies so that one can implement the client in languages other than
>> java. ZK client is only available in a small number of languages.
>>
>> Thanks,
>>
>> Jun
>>
>>
>> On Fri, Jan 24, 2014 at 5:17 PM, Jay Kreps <ja...@gmail.com> wrote:
>>
>>> Clark and all,
>>>
>>> I thought a little bit about the serialization question. Here are the
>>> options I see and the pros and cons I can think of. I'd love to hear
>>> people's preferences if you have a strong one.
>>>
>>> One important consideration is that however the producer works will also
>>> need to be how the new consumer works (which we hope to write next). That
>>> is if you put objects in, you should get objects out. So we need to think
>>> through both sides.
>>>
>>> Options:
>>>
>>> Option 0: What is in the existing scala code and the java code I
>>> posted--Serializer and Partitioner plugin provided by the user via config.
>>> Partitioner has a sane default, but Serializer needs to be specified in
>>> config.
>>>
>>> Pros: How it works today in the scala code.
>>> Cons: You have to adapt your serialization library of choice to our
>>> interfaces. The reflective class loading means typo in the serializer name
>>> give odd errors. Likewise there is little type safety--the ProducerRecord
>>> takes Object and any type errors between the object provided and the
>>> serializer give occurs at runtime.
>>>
>>> Option 1: No plugins
>>>
>>> This would mean byte[] key, byte[] value, and partitioning done by client
>>> by passing in a partition *number* directly.
>>>
>>> The problem with this is that it is tricky to compute the partition
>>> correctly and probably most people won't. We could add a getCluster()
>>> method to return the Cluster instance you should use for partitioning. But
>>> I suspect people would be lazy and not use that and instead hard-code
>>> partitions which would break if partitions were added or they hard coded it
>>> wrong. In my experience 3 partitioning strategies cover like 99% of cases
>>> so not having a default implementation for this makes the common case
>>> harder. Left to their own devices people will use bad hash functions and
>>> get weird results.
>>>
>>> Option 1A: Alternatively we could partition by the key using the existing
>>> default partitioning strategy which only uses the byte[] anyway but instead
>>> of having a partitionKey we could have a numerical partition override and
>>> add the getCluster() method to get the cluster metadata. That would make
>>> custom partitioning possible but handle the common case simply.
>>>
>>> Option 2: Partitioner plugin remains, serializers go.
>>>
>>> The problem here is that the partitioner might lose access to the
>>> deserialized key which would occasionally be useful for semantic
>>> partitioning schemes. The Partitioner could deserialize the key but that
>>> would be inefficient and weird.
>>>
>>> This problem could be fixed by having key and value be byte[] but retaining
>>> partitionKey as an Object and passing it to the partitioner as is. Then if
>>> you have a partitioner which requires the deserialized key you would need
>>> to use this partition key. One weird side effect is that if you want to
>>> have a custom partition key BUT want to partition by the bytes of that key
>>> rather than the object value you must write a customer partitioner and
>>> serialize it yourself.
>>>
>>> Of these I think I prefer 1A but could be convinced of 0 since that is how
>>> it works now.
>>>
>>> Thoughts?
>>>
>>> -Jay
>>>
>>>
>>> On Fri, Jan 24, 2014 at 3:30 PM, Clark Breyman <cl...@breyman.com> wrote:
>>>
>>>> Jay - Thanks for the call for comments. Here's some initial input:
>>>>
>>>> - Make message serialization a client responsibility (making all messages
>>>> byte[]). Reflection-based loading makes it harder to use generic codecs
>>>> (e.g.  Envelope<PREFIX, DATA, SUFFIX>) or build up codec
>>> programmatically.
>>>> Non-default partitioning should require an explicit partition key.
>>>>
>>>> - I really like the fact that it will be native Java. Please consider
>>> using
>>>> native maven and not sbt, gradle, ivy, etc as they don't reliably play
>>> nice
>>>> in the maven ecosystem. A jar without a well-formed pom doesn't feel
>>> like a
>>>> real artifact. The pom's generated by sbt et al. are not well formed.
>>> Using
>>>> maven will make builds and IDE integration much smoother.
>>>>
>>>> - Look at Nick Telford's dropwizard-extras package in which he defines
>>> some
>>>> Jackson-compatible POJO's for loading configuration. Seems like your
>>> client
>>>> migration is similar. The config objects should have constructors or
>>>> factories that accept Map<String, String> and Properties for ease of
>>>> migration.
>>>>
>>>> - Would you consider using the org.apache.kafka package for the new API
>>>> (quibble)
>>>>
>>>> - Why create your own futures rather than use
>>>> java.util.concurrent.Future<Long> or similar? Standard futures will play
>>>> nice with other reactive libs and things like J8's ComposableFuture.
>>>>
>>>> Thanks again,
>>>> C
>>>>
>>>>
>>>>
>>>> On Fri, Jan 24, 2014 at 2:46 PM, Roger Hoover <roger.hoover@gmail.com
>>>>> wrote:
>>>>> A couple comments:
>>>>>
>>>>> 1) Why does the config use a broker list instead of discovering the
>>>> brokers
>>>>> in ZooKeeper?  It doesn't match the HighLevelConsumer API.
>>>>>
>>>>> 2) It looks like broker connections are created on demand.  I'm
>>> wondering
>>>>> if sometimes you might want to flush out config or network connectivity
>>>>> issues before pushing the first message through.
>>>>>
>>>>> Should there also be a KafkaProducer.connect() or .open() method or
>>>>> connectAll()?  I guess it would try to connect to all brokers in the
>>>>> BROKER_LIST_CONFIG
>>>>>
>>>>> HTH,
>>>>>
>>>>> Roger
>>>>>
>>>>>
>>>>> On Fri, Jan 24, 2014 at 11:54 AM, Jay Kreps <ja...@gmail.com>
>>> wrote:
>>>>>> As mentioned in a previous email we are working on a
>>> re-implementation
>>>> of
>>>>>> the producer. I would like to use this email thread to discuss the
>>>>> details
>>>>>> of the public API and the configuration. I would love for us to be
>>>>>> incredibly picky about this public api now so it is as good as
>>> possible
>>>>> and
>>>>>> we don't need to break it in the future.
>>>>>>
>>>>>> The best way to get a feel for the API is actually to take a look at
>>>> the
>>>>>> javadoc, my hope is to get the api docs good enough so that it is
>>>>>> self-explanatory:
>>>>>>
>>>>>>
>>> http://empathybox.com/kafka-javadoc/index.html?kafka/clients/producer/KafkaProducer.html
>>>>>> Please take a look at this API and give me any thoughts you may have!
>>>>>>
>>>>>> It may also be reasonable to take a look at the configs:
>>>>>>
>>>>>>
>>> http://empathybox.com/kafka-javadoc/kafka/clients/producer/ProducerConfig.html
>>>>>> The actual code is posted here:
>>>>>> https://issues.apache.org/jira/browse/KAFKA-1227
>>>>>>
>>>>>> A few questions or comments to kick things off:
>>>>>> 1. We need to make a decision on whether serialization of the user's
>>>> key
>>>>>> and value should be done by the user (with our api just taking
>>> byte[])
>>>> or
>>>>>> if we should take an object and allow the user to configure a
>>>> Serializer
>>>>>> class which we instantiate via reflection. We take the later approach
>>>> in
>>>>>> the current producer, and I have carried this through to this
>>>> prototype.
>>>>>> The tradeoff I see is this: taking byte[] is actually simpler, the
>>> user
>>>>> can
>>>>>> directly do whatever serialization they like. The complication is
>>>>> actually
>>>>>> partitioning. Currently partitioning is done by a similar plug-in api
>>>>>> (Partitioner) which the user can implement and configure to override
>>>> how
>>>>>> partitions are assigned. If we take byte[] as input then we have no
>>>>> access
>>>>>> to the original object and partitioning MUST be done on the byte[].
>>>> This
>>>>> is
>>>>>> fine for hash partitioning. However for various types of semantic
>>>>>> partitioning (range partitioning, or whatever) you would want access
>>> to
>>>>> the
>>>>>> original object. In the current approach a producer who wishes to
>>> send
>>>>>> byte[] they have serialized in their own code can configure the
>>>>>> BytesSerialization we supply which is just a "no op" serialization.
>>>>>> 2. We should obsess over naming and make sure each of the class names
>>>> are
>>>>>> good.
>>>>>> 3. Jun has already pointed out that we need to include the topic and
>>>>>> partition in the response, which is absolutely right. I haven't done
>>>> that
>>>>>> yet but that definitely needs to be there.
>>>>>> 4. Currently RecordSend.await will throw an exception if the request
>>>>>> failed. The intention here is that producer.send(message).await()
>>>> exactly
>>>>>> simulates a synchronous call. Guozhang has noted that this is a
>>> little
>>>>>> annoying since the user must then catch exceptions. However if we
>>>> remove
>>>>>> this then if the user doesn't check for errors they won't know one
>>> has
>>>>>> occurred, which I predict will be a common mistake.
>>>>>> 5. Perhaps there is more we could do to make the async callbacks and
>>>>> future
>>>>>> we give back intuitive and easy to program against?
>>>>>>
>>>>>> Some background info on implementation:
>>>>>>
>>>>>> At a high level the primary difference in this producer is that it
>>>>> removes
>>>>>> the distinction between the "sync" and "async" producer. Effectively
>>>> all
>>>>>> requests are sent asynchronously but always return a future response
>>>>> object
>>>>>> that gives the offset as well as any error that may have occurred
>>> when
>>>>> the
>>>>>> request is complete. The batching that is done in the async producer
>>>> only
>>>>>> today is done whenever possible now. This means that the sync
>>> producer,
>>>>>> under load, can get performance as good as the async producer
>>>>> (preliminary
>>>>>> results show the producer getting 1m messages/sec). This works
>>> similar
>>>> to
>>>>>> group commit in databases but with respect to the actual network
>>>>>> transmission--any messages that arrive while a send is in progress
>>> are
>>>>>> batched together. It is also possible to encourage batching even
>>> under
>>>>> low
>>>>>> load to save server resources by introducing a delay on the send to
>>>> allow
>>>>>> more messages to accumulate; this is done using the linger.ms config
>>>>> (this
>>>>>> is similar to Nagle's algorithm in TCP).
>>>>>>
>>>>>> This producer does all network communication asynchronously and in
>>>>> parallel
>>>>>> to all servers so the performance penalty for acks=-1 and waiting on
>>>>>> replication should be much reduced. I haven't done much benchmarking
>>> on
>>>>>> this yet, though.
>>>>>>
>>>>>> The high level design is described a little here, though this is now
>>> a
>>>>>> little out of date:
>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/Client+Rewrite
>>>>>>
>>>>>> -Jay
>>>>>>


Re: New Producer Public API

Posted by Joel Koshy <jj...@gmail.com>.
That's a good point about 1A - does seem that we would need to have
some kind of TTL for each topic's metadata.

Also, WRT ZK dependency I don't think that decision (for the Java
client) affects other clients. i.e., other client implementations can
use whatever discovery mechanism it chooses. That said, I prefer not
having a ZK dependency for the same reasons covered earlier in this
thread.

On Thu, Jan 30, 2014 at 4:34 PM, Jun Rao <ju...@gmail.com> wrote:
> With option 1A, if we increase # partitions on a topic, how will the
> producer find out newly created partitions? Do we expect the producer to
> periodically call getCluster()?
>
> As for ZK dependency, one of the goals of client rewrite is to reduce
> dependencies so that one can implement the client in languages other than
> java. ZK client is only available in a small number of languages.
>
> Thanks,
>
> Jun
>
>
> On Fri, Jan 24, 2014 at 5:17 PM, Jay Kreps <ja...@gmail.com> wrote:
>
>> Clark and all,
>>
>> I thought a little bit about the serialization question. Here are the
>> options I see and the pros and cons I can think of. I'd love to hear
>> people's preferences if you have a strong one.
>>
>> One important consideration is that however the producer works will also
>> need to be how the new consumer works (which we hope to write next). That
>> is if you put objects in, you should get objects out. So we need to think
>> through both sides.
>>
>> Options:
>>
>> Option 0: What is in the existing scala code and the java code I
>> posted--Serializer and Partitioner plugin provided by the user via config.
>> Partitioner has a sane default, but Serializer needs to be specified in
>> config.
>>
>> Pros: How it works today in the scala code.
>> Cons: You have to adapt your serialization library of choice to our
>> interfaces. The reflective class loading means typo in the serializer name
>> give odd errors. Likewise there is little type safety--the ProducerRecord
>> takes Object and any type errors between the object provided and the
>> serializer give occurs at runtime.
>>
>> Option 1: No plugins
>>
>> This would mean byte[] key, byte[] value, and partitioning done by client
>> by passing in a partition *number* directly.
>>
>> The problem with this is that it is tricky to compute the partition
>> correctly and probably most people won't. We could add a getCluster()
>> method to return the Cluster instance you should use for partitioning. But
>> I suspect people would be lazy and not use that and instead hard-code
>> partitions which would break if partitions were added or they hard coded it
>> wrong. In my experience 3 partitioning strategies cover like 99% of cases
>> so not having a default implementation for this makes the common case
>> harder. Left to their own devices people will use bad hash functions and
>> get weird results.
>>
>> Option 1A: Alternatively we could partition by the key using the existing
>> default partitioning strategy which only uses the byte[] anyway but instead
>> of having a partitionKey we could have a numerical partition override and
>> add the getCluster() method to get the cluster metadata. That would make
>> custom partitioning possible but handle the common case simply.
>>
>> Option 2: Partitioner plugin remains, serializers go.
>>
>> The problem here is that the partitioner might lose access to the
>> deserialized key which would occasionally be useful for semantic
>> partitioning schemes. The Partitioner could deserialize the key but that
>> would be inefficient and weird.
>>
>> This problem could be fixed by having key and value be byte[] but retaining
>> partitionKey as an Object and passing it to the partitioner as is. Then if
>> you have a partitioner which requires the deserialized key you would need
>> to use this partition key. One weird side effect is that if you want to
>> have a custom partition key BUT want to partition by the bytes of that key
>> rather than the object value you must write a customer partitioner and
>> serialize it yourself.
>>
>> Of these I think I prefer 1A but could be convinced of 0 since that is how
>> it works now.
>>
>> Thoughts?
>>
>> -Jay
>>
>>
>> On Fri, Jan 24, 2014 at 3:30 PM, Clark Breyman <cl...@breyman.com> wrote:
>>
>> > Jay - Thanks for the call for comments. Here's some initial input:
>> >
>> > - Make message serialization a client responsibility (making all messages
>> > byte[]). Reflection-based loading makes it harder to use generic codecs
>> > (e.g.  Envelope<PREFIX, DATA, SUFFIX>) or build up codec
>> programmatically.
>> > Non-default partitioning should require an explicit partition key.
>> >
>> > - I really like the fact that it will be native Java. Please consider
>> using
>> > native maven and not sbt, gradle, ivy, etc as they don't reliably play
>> nice
>> > in the maven ecosystem. A jar without a well-formed pom doesn't feel
>> like a
>> > real artifact. The pom's generated by sbt et al. are not well formed.
>> Using
>> > maven will make builds and IDE integration much smoother.
>> >
>> > - Look at Nick Telford's dropwizard-extras package in which he defines
>> some
>> > Jackson-compatible POJO's for loading configuration. Seems like your
>> client
>> > migration is similar. The config objects should have constructors or
>> > factories that accept Map<String, String> and Properties for ease of
>> > migration.
>> >
>> > - Would you consider using the org.apache.kafka package for the new API
>> > (quibble)
>> >
>> > - Why create your own futures rather than use
>> > java.util.concurrent.Future<Long> or similar? Standard futures will play
>> > nice with other reactive libs and things like J8's ComposableFuture.
>> >
>> > Thanks again,
>> > C
>> >
>> >
>> >
>> > On Fri, Jan 24, 2014 at 2:46 PM, Roger Hoover <roger.hoover@gmail.com
>> > >wrote:
>> >
>> > > A couple comments:
>> > >
>> > > 1) Why does the config use a broker list instead of discovering the
>> > brokers
>> > > in ZooKeeper?  It doesn't match the HighLevelConsumer API.
>> > >
>> > > 2) It looks like broker connections are created on demand.  I'm
>> wondering
>> > > if sometimes you might want to flush out config or network connectivity
>> > > issues before pushing the first message through.
>> > >
>> > > Should there also be a KafkaProducer.connect() or .open() method or
>> > > connectAll()?  I guess it would try to connect to all brokers in the
>> > > BROKER_LIST_CONFIG
>> > >
>> > > HTH,
>> > >
>> > > Roger
>> > >
>> > >
>> > > On Fri, Jan 24, 2014 at 11:54 AM, Jay Kreps <ja...@gmail.com>
>> wrote:
>> > >
>> > > > As mentioned in a previous email we are working on a
>> re-implementation
>> > of
>> > > > the producer. I would like to use this email thread to discuss the
>> > > details
>> > > > of the public API and the configuration. I would love for us to be
>> > > > incredibly picky about this public api now so it is as good as
>> possible
>> > > and
>> > > > we don't need to break it in the future.
>> > > >
>> > > > The best way to get a feel for the API is actually to take a look at
>> > the
>> > > > javadoc, my hope is to get the api docs good enough so that it is
>> > > > self-explanatory:
>> > > >
>> > > >
>> > >
>> >
>> http://empathybox.com/kafka-javadoc/index.html?kafka/clients/producer/KafkaProducer.html
>> > > >
>> > > > Please take a look at this API and give me any thoughts you may have!
>> > > >
>> > > > It may also be reasonable to take a look at the configs:
>> > > >
>> > > >
>> > >
>> >
>> http://empathybox.com/kafka-javadoc/kafka/clients/producer/ProducerConfig.html
>> > > >
>> > > > The actual code is posted here:
>> > > > https://issues.apache.org/jira/browse/KAFKA-1227
>> > > >
>> > > > A few questions or comments to kick things off:
>> > > > 1. We need to make a decision on whether serialization of the user's
>> > key
>> > > > and value should be done by the user (with our api just taking
>> byte[])
>> > or
>> > > > if we should take an object and allow the user to configure a
>> > Serializer
>> > > > class which we instantiate via reflection. We take the later approach
>> > in
>> > > > the current producer, and I have carried this through to this
>> > prototype.
>> > > > The tradeoff I see is this: taking byte[] is actually simpler, the
>> user
>> > > can
>> > > > directly do whatever serialization they like. The complication is
>> > > actually
>> > > > partitioning. Currently partitioning is done by a similar plug-in api
>> > > > (Partitioner) which the user can implement and configure to override
>> > how
>> > > > partitions are assigned. If we take byte[] as input then we have no
>> > > access
>> > > > to the original object and partitioning MUST be done on the byte[].
>> > This
>> > > is
>> > > > fine for hash partitioning. However for various types of semantic
>> > > > partitioning (range partitioning, or whatever) you would want access
>> to
>> > > the
>> > > > original object. In the current approach a producer who wishes to
>> send
>> > > > byte[] they have serialized in their own code can configure the
>> > > > BytesSerialization we supply which is just a "no op" serialization.
>> > > > 2. We should obsess over naming and make sure each of the class names
>> > are
>> > > > good.
>> > > > 3. Jun has already pointed out that we need to include the topic and
>> > > > partition in the response, which is absolutely right. I haven't done
>> > that
>> > > > yet but that definitely needs to be there.
>> > > > 4. Currently RecordSend.await will throw an exception if the request
>> > > > failed. The intention here is that producer.send(message).await()
>> > exactly
>> > > > simulates a synchronous call. Guozhang has noted that this is a
>> little
>> > > > annoying since the user must then catch exceptions. However if we
>> > remove
>> > > > this then if the user doesn't check for errors they won't know one
>> has
>> > > > occurred, which I predict will be a common mistake.
>> > > > 5. Perhaps there is more we could do to make the async callbacks and
>> > > future
>> > > > we give back intuitive and easy to program against?
>> > > >
>> > > > Some background info on implementation:
>> > > >
>> > > > At a high level the primary difference in this producer is that it
>> > > removes
>> > > > the distinction between the "sync" and "async" producer. Effectively
>> > all
>> > > > requests are sent asynchronously but always return a future response
>> > > object
>> > > > that gives the offset as well as any error that may have occurred
>> when
>> > > the
>> > > > request is complete. The batching that is done in the async producer
>> > only
>> > > > today is done whenever possible now. This means that the sync
>> producer,
>> > > > under load, can get performance as good as the async producer
>> > > (preliminary
>> > > > results show the producer getting 1m messages/sec). This works
>> similar
>> > to
>> > > > group commit in databases but with respect to the actual network
>> > > > transmission--any messages that arrive while a send is in progress
>> are
>> > > > batched together. It is also possible to encourage batching even
>> under
>> > > low
>> > > > load to save server resources by introducing a delay on the send to
>> > allow
>> > > > more messages to accumulate; this is done using the linger.ms config
>> > > (this
>> > > > is similar to Nagle's algorithm in TCP).
>> > > >
>> > > > This producer does all network communication asynchronously and in
>> > > parallel
>> > > > to all servers so the performance penalty for acks=-1 and waiting on
>> > > > replication should be much reduced. I haven't done much benchmarking
>> on
>> > > > this yet, though.
>> > > >
>> > > > The high level design is described a little here, though this is now
>> a
>> > > > little out of date:
>> > > > https://cwiki.apache.org/confluence/display/KAFKA/Client+Rewrite
>> > > >
>> > > > -Jay
>> > > >
>> > >
>> >
>>

Re: New Producer Public API

Posted by Jay Kreps <ja...@gmail.com>.
I thought a bit about it and I think the getCluster() thing was overly
simplistic because we try to only maintain metadata about the current set
of topics the producer cares about so the cluster might not have the
partitions for the topic the user cares about. I think actually what we
need is a new method on producer:
  List<PartitionInfo> partitionsFor(String...topics)
The intended usage of this method would be:
  int partition = myPartitionFunction(key, producer.partitionsFor(topic));
  producer.send(new ProducerRecord(topic, partition, key, value))
That is, the producer would re-fetch the current set of partitions every
time and the partitions would refresh at whatever schedule the producer
metadata refresh was set with.

So in the case of partition change producers would pick this up as their
natural metadata updates occurred.

This actually solves an important problem in the existing api, which is
that the send method will block on the first message send if we don't yet
have metadata for the destination topic. It blocks until metadata is
fetched. This is a little weird as this occurs even in non-blocking mode.
This allows an escape hatch. Someone who wants to avoid that small block on
the first send can initialize their producer and call
producer.partitionsFor(topics) to force metadata initialization.

-Jay



On Thu, Jan 30, 2014 at 4:34 PM, Jun Rao <ju...@gmail.com> wrote:

> With option 1A, if we increase # partitions on a topic, how will the
> producer find out newly created partitions? Do we expect the producer to
> periodically call getCluster()?
>
> As for ZK dependency, one of the goals of client rewrite is to reduce
> dependencies so that one can implement the client in languages other than
> java. ZK client is only available in a small number of languages.
>
> Thanks,
>
> Jun
>
>
> On Fri, Jan 24, 2014 at 5:17 PM, Jay Kreps <ja...@gmail.com> wrote:
>
> > Clark and all,
> >
> > I thought a little bit about the serialization question. Here are the
> > options I see and the pros and cons I can think of. I'd love to hear
> > people's preferences if you have a strong one.
> >
> > One important consideration is that however the producer works will also
> > need to be how the new consumer works (which we hope to write next). That
> > is if you put objects in, you should get objects out. So we need to think
> > through both sides.
> >
> > Options:
> >
> > Option 0: What is in the existing scala code and the java code I
> > posted--Serializer and Partitioner plugin provided by the user via
> config.
> > Partitioner has a sane default, but Serializer needs to be specified in
> > config.
> >
> > Pros: How it works today in the scala code.
> > Cons: You have to adapt your serialization library of choice to our
> > interfaces. The reflective class loading means typo in the serializer
> name
> > give odd errors. Likewise there is little type safety--the ProducerRecord
> > takes Object and any type errors between the object provided and the
> > serializer give occurs at runtime.
> >
> > Option 1: No plugins
> >
> > This would mean byte[] key, byte[] value, and partitioning done by client
> > by passing in a partition *number* directly.
> >
> > The problem with this is that it is tricky to compute the partition
> > correctly and probably most people won't. We could add a getCluster()
> > method to return the Cluster instance you should use for partitioning.
> But
> > I suspect people would be lazy and not use that and instead hard-code
> > partitions which would break if partitions were added or they hard coded
> it
> > wrong. In my experience 3 partitioning strategies cover like 99% of cases
> > so not having a default implementation for this makes the common case
> > harder. Left to their own devices people will use bad hash functions and
> > get weird results.
> >
> > Option 1A: Alternatively we could partition by the key using the existing
> > default partitioning strategy which only uses the byte[] anyway but
> instead
> > of having a partitionKey we could have a numerical partition override and
> > add the getCluster() method to get the cluster metadata. That would make
> > custom partitioning possible but handle the common case simply.
> >
> > Option 2: Partitioner plugin remains, serializers go.
> >
> > The problem here is that the partitioner might lose access to the
> > deserialized key which would occasionally be useful for semantic
> > partitioning schemes. The Partitioner could deserialize the key but that
> > would be inefficient and weird.
> >
> > This problem could be fixed by having key and value be byte[] but
> retaining
> > partitionKey as an Object and passing it to the partitioner as is. Then
> if
> > you have a partitioner which requires the deserialized key you would need
> > to use this partition key. One weird side effect is that if you want to
> > have a custom partition key BUT want to partition by the bytes of that
> key
> > rather than the object value you must write a customer partitioner and
> > serialize it yourself.
> >
> > Of these I think I prefer 1A but could be convinced of 0 since that is
> how
> > it works now.
> >
> > Thoughts?
> >
> > -Jay
> >
> >
> > On Fri, Jan 24, 2014 at 3:30 PM, Clark Breyman <cl...@breyman.com>
> wrote:
> >
> > > Jay - Thanks for the call for comments. Here's some initial input:
> > >
> > > - Make message serialization a client responsibility (making all
> messages
> > > byte[]). Reflection-based loading makes it harder to use generic codecs
> > > (e.g.  Envelope<PREFIX, DATA, SUFFIX>) or build up codec
> > programmatically.
> > > Non-default partitioning should require an explicit partition key.
> > >
> > > - I really like the fact that it will be native Java. Please consider
> > using
> > > native maven and not sbt, gradle, ivy, etc as they don't reliably play
> > nice
> > > in the maven ecosystem. A jar without a well-formed pom doesn't feel
> > like a
> > > real artifact. The pom's generated by sbt et al. are not well formed.
> > Using
> > > maven will make builds and IDE integration much smoother.
> > >
> > > - Look at Nick Telford's dropwizard-extras package in which he defines
> > some
> > > Jackson-compatible POJO's for loading configuration. Seems like your
> > client
> > > migration is similar. The config objects should have constructors or
> > > factories that accept Map<String, String> and Properties for ease of
> > > migration.
> > >
> > > - Would you consider using the org.apache.kafka package for the new API
> > > (quibble)
> > >
> > > - Why create your own futures rather than use
> > > java.util.concurrent.Future<Long> or similar? Standard futures will
> play
> > > nice with other reactive libs and things like J8's ComposableFuture.
> > >
> > > Thanks again,
> > > C
> > >
> > >
> > >
> > > On Fri, Jan 24, 2014 at 2:46 PM, Roger Hoover <roger.hoover@gmail.com
> > > >wrote:
> > >
> > > > A couple comments:
> > > >
> > > > 1) Why does the config use a broker list instead of discovering the
> > > brokers
> > > > in ZooKeeper?  It doesn't match the HighLevelConsumer API.
> > > >
> > > > 2) It looks like broker connections are created on demand.  I'm
> > wondering
> > > > if sometimes you might want to flush out config or network
> connectivity
> > > > issues before pushing the first message through.
> > > >
> > > > Should there also be a KafkaProducer.connect() or .open() method or
> > > > connectAll()?  I guess it would try to connect to all brokers in the
> > > > BROKER_LIST_CONFIG
> > > >
> > > > HTH,
> > > >
> > > > Roger
> > > >
> > > >
> > > > On Fri, Jan 24, 2014 at 11:54 AM, Jay Kreps <ja...@gmail.com>
> > wrote:
> > > >
> > > > > As mentioned in a previous email we are working on a
> > re-implementation
> > > of
> > > > > the producer. I would like to use this email thread to discuss the
> > > > details
> > > > > of the public API and the configuration. I would love for us to be
> > > > > incredibly picky about this public api now so it is as good as
> > possible
> > > > and
> > > > > we don't need to break it in the future.
> > > > >
> > > > > The best way to get a feel for the API is actually to take a look
> at
> > > the
> > > > > javadoc, my hope is to get the api docs good enough so that it is
> > > > > self-explanatory:
> > > > >
> > > > >
> > > >
> > >
> >
> http://empathybox.com/kafka-javadoc/index.html?kafka/clients/producer/KafkaProducer.html
> > > > >
> > > > > Please take a look at this API and give me any thoughts you may
> have!
> > > > >
> > > > > It may also be reasonable to take a look at the configs:
> > > > >
> > > > >
> > > >
> > >
> >
> http://empathybox.com/kafka-javadoc/kafka/clients/producer/ProducerConfig.html
> > > > >
> > > > > The actual code is posted here:
> > > > > https://issues.apache.org/jira/browse/KAFKA-1227
> > > > >
> > > > > A few questions or comments to kick things off:
> > > > > 1. We need to make a decision on whether serialization of the
> user's
> > > key
> > > > > and value should be done by the user (with our api just taking
> > byte[])
> > > or
> > > > > if we should take an object and allow the user to configure a
> > > Serializer
> > > > > class which we instantiate via reflection. We take the later
> approach
> > > in
> > > > > the current producer, and I have carried this through to this
> > > prototype.
> > > > > The tradeoff I see is this: taking byte[] is actually simpler, the
> > user
> > > > can
> > > > > directly do whatever serialization they like. The complication is
> > > > actually
> > > > > partitioning. Currently partitioning is done by a similar plug-in
> api
> > > > > (Partitioner) which the user can implement and configure to
> override
> > > how
> > > > > partitions are assigned. If we take byte[] as input then we have no
> > > > access
> > > > > to the original object and partitioning MUST be done on the byte[].
> > > This
> > > > is
> > > > > fine for hash partitioning. However for various types of semantic
> > > > > partitioning (range partitioning, or whatever) you would want
> access
> > to
> > > > the
> > > > > original object. In the current approach a producer who wishes to
> > send
> > > > > byte[] they have serialized in their own code can configure the
> > > > > BytesSerialization we supply which is just a "no op" serialization.
> > > > > 2. We should obsess over naming and make sure each of the class
> names
> > > are
> > > > > good.
> > > > > 3. Jun has already pointed out that we need to include the topic
> and
> > > > > partition in the response, which is absolutely right. I haven't
> done
> > > that
> > > > > yet but that definitely needs to be there.
> > > > > 4. Currently RecordSend.await will throw an exception if the
> request
> > > > > failed. The intention here is that producer.send(message).await()
> > > exactly
> > > > > simulates a synchronous call. Guozhang has noted that this is a
> > little
> > > > > annoying since the user must then catch exceptions. However if we
> > > remove
> > > > > this then if the user doesn't check for errors they won't know one
> > has
> > > > > occurred, which I predict will be a common mistake.
> > > > > 5. Perhaps there is more we could do to make the async callbacks
> and
> > > > future
> > > > > we give back intuitive and easy to program against?
> > > > >
> > > > > Some background info on implementation:
> > > > >
> > > > > At a high level the primary difference in this producer is that it
> > > > removes
> > > > > the distinction between the "sync" and "async" producer.
> Effectively
> > > all
> > > > > requests are sent asynchronously but always return a future
> response
> > > > object
> > > > > that gives the offset as well as any error that may have occurred
> > when
> > > > the
> > > > > request is complete. The batching that is done in the async
> producer
> > > only
> > > > > today is done whenever possible now. This means that the sync
> > producer,
> > > > > under load, can get performance as good as the async producer
> > > > (preliminary
> > > > > results show the producer getting 1m messages/sec). This works
> > similar
> > > to
> > > > > group commit in databases but with respect to the actual network
> > > > > transmission--any messages that arrive while a send is in progress
> > are
> > > > > batched together. It is also possible to encourage batching even
> > under
> > > > low
> > > > > load to save server resources by introducing a delay on the send to
> > > allow
> > > > > more messages to accumulate; this is done using the linger.msconfig
> > > > (this
> > > > > is similar to Nagle's algorithm in TCP).
> > > > >
> > > > > This producer does all network communication asynchronously and in
> > > > parallel
> > > > > to all servers so the performance penalty for acks=-1 and waiting
> on
> > > > > replication should be much reduced. I haven't done much
> benchmarking
> > on
> > > > > this yet, though.
> > > > >
> > > > > The high level design is described a little here, though this is
> now
> > a
> > > > > little out of date:
> > > > > https://cwiki.apache.org/confluence/display/KAFKA/Client+Rewrite
> > > > >
> > > > > -Jay
> > > > >
> > > >
> > >
> >
>

Re: New Producer Public API

Posted by Jun Rao <ju...@gmail.com>.
With option 1A, if we increase # partitions on a topic, how will the
producer find out newly created partitions? Do we expect the producer to
periodically call getCluster()?

As for ZK dependency, one of the goals of client rewrite is to reduce
dependencies so that one can implement the client in languages other than
java. ZK client is only available in a small number of languages.

Thanks,

Jun


On Fri, Jan 24, 2014 at 5:17 PM, Jay Kreps <ja...@gmail.com> wrote:

> Clark and all,
>
> I thought a little bit about the serialization question. Here are the
> options I see and the pros and cons I can think of. I'd love to hear
> people's preferences if you have a strong one.
>
> One important consideration is that however the producer works will also
> need to be how the new consumer works (which we hope to write next). That
> is if you put objects in, you should get objects out. So we need to think
> through both sides.
>
> Options:
>
> Option 0: What is in the existing scala code and the java code I
> posted--Serializer and Partitioner plugin provided by the user via config.
> Partitioner has a sane default, but Serializer needs to be specified in
> config.
>
> Pros: How it works today in the scala code.
> Cons: You have to adapt your serialization library of choice to our
> interfaces. The reflective class loading means typo in the serializer name
> give odd errors. Likewise there is little type safety--the ProducerRecord
> takes Object and any type errors between the object provided and the
> serializer give occurs at runtime.
>
> Option 1: No plugins
>
> This would mean byte[] key, byte[] value, and partitioning done by client
> by passing in a partition *number* directly.
>
> The problem with this is that it is tricky to compute the partition
> correctly and probably most people won't. We could add a getCluster()
> method to return the Cluster instance you should use for partitioning. But
> I suspect people would be lazy and not use that and instead hard-code
> partitions which would break if partitions were added or they hard coded it
> wrong. In my experience 3 partitioning strategies cover like 99% of cases
> so not having a default implementation for this makes the common case
> harder. Left to their own devices people will use bad hash functions and
> get weird results.
>
> Option 1A: Alternatively we could partition by the key using the existing
> default partitioning strategy which only uses the byte[] anyway but instead
> of having a partitionKey we could have a numerical partition override and
> add the getCluster() method to get the cluster metadata. That would make
> custom partitioning possible but handle the common case simply.
>
> Option 2: Partitioner plugin remains, serializers go.
>
> The problem here is that the partitioner might lose access to the
> deserialized key which would occasionally be useful for semantic
> partitioning schemes. The Partitioner could deserialize the key but that
> would be inefficient and weird.
>
> This problem could be fixed by having key and value be byte[] but retaining
> partitionKey as an Object and passing it to the partitioner as is. Then if
> you have a partitioner which requires the deserialized key you would need
> to use this partition key. One weird side effect is that if you want to
> have a custom partition key BUT want to partition by the bytes of that key
> rather than the object value you must write a customer partitioner and
> serialize it yourself.
>
> Of these I think I prefer 1A but could be convinced of 0 since that is how
> it works now.
>
> Thoughts?
>
> -Jay
>
>
> On Fri, Jan 24, 2014 at 3:30 PM, Clark Breyman <cl...@breyman.com> wrote:
>
> > Jay - Thanks for the call for comments. Here's some initial input:
> >
> > - Make message serialization a client responsibility (making all messages
> > byte[]). Reflection-based loading makes it harder to use generic codecs
> > (e.g.  Envelope<PREFIX, DATA, SUFFIX>) or build up codec
> programmatically.
> > Non-default partitioning should require an explicit partition key.
> >
> > - I really like the fact that it will be native Java. Please consider
> using
> > native maven and not sbt, gradle, ivy, etc as they don't reliably play
> nice
> > in the maven ecosystem. A jar without a well-formed pom doesn't feel
> like a
> > real artifact. The pom's generated by sbt et al. are not well formed.
> Using
> > maven will make builds and IDE integration much smoother.
> >
> > - Look at Nick Telford's dropwizard-extras package in which he defines
> some
> > Jackson-compatible POJO's for loading configuration. Seems like your
> client
> > migration is similar. The config objects should have constructors or
> > factories that accept Map<String, String> and Properties for ease of
> > migration.
> >
> > - Would you consider using the org.apache.kafka package for the new API
> > (quibble)
> >
> > - Why create your own futures rather than use
> > java.util.concurrent.Future<Long> or similar? Standard futures will play
> > nice with other reactive libs and things like J8's ComposableFuture.
> >
> > Thanks again,
> > C
> >
> >
> >
> > On Fri, Jan 24, 2014 at 2:46 PM, Roger Hoover <roger.hoover@gmail.com
> > >wrote:
> >
> > > A couple comments:
> > >
> > > 1) Why does the config use a broker list instead of discovering the
> > brokers
> > > in ZooKeeper?  It doesn't match the HighLevelConsumer API.
> > >
> > > 2) It looks like broker connections are created on demand.  I'm
> wondering
> > > if sometimes you might want to flush out config or network connectivity
> > > issues before pushing the first message through.
> > >
> > > Should there also be a KafkaProducer.connect() or .open() method or
> > > connectAll()?  I guess it would try to connect to all brokers in the
> > > BROKER_LIST_CONFIG
> > >
> > > HTH,
> > >
> > > Roger
> > >
> > >
> > > On Fri, Jan 24, 2014 at 11:54 AM, Jay Kreps <ja...@gmail.com>
> wrote:
> > >
> > > > As mentioned in a previous email we are working on a
> re-implementation
> > of
> > > > the producer. I would like to use this email thread to discuss the
> > > details
> > > > of the public API and the configuration. I would love for us to be
> > > > incredibly picky about this public api now so it is as good as
> possible
> > > and
> > > > we don't need to break it in the future.
> > > >
> > > > The best way to get a feel for the API is actually to take a look at
> > the
> > > > javadoc, my hope is to get the api docs good enough so that it is
> > > > self-explanatory:
> > > >
> > > >
> > >
> >
> http://empathybox.com/kafka-javadoc/index.html?kafka/clients/producer/KafkaProducer.html
> > > >
> > > > Please take a look at this API and give me any thoughts you may have!
> > > >
> > > > It may also be reasonable to take a look at the configs:
> > > >
> > > >
> > >
> >
> http://empathybox.com/kafka-javadoc/kafka/clients/producer/ProducerConfig.html
> > > >
> > > > The actual code is posted here:
> > > > https://issues.apache.org/jira/browse/KAFKA-1227
> > > >
> > > > A few questions or comments to kick things off:
> > > > 1. We need to make a decision on whether serialization of the user's
> > key
> > > > and value should be done by the user (with our api just taking
> byte[])
> > or
> > > > if we should take an object and allow the user to configure a
> > Serializer
> > > > class which we instantiate via reflection. We take the later approach
> > in
> > > > the current producer, and I have carried this through to this
> > prototype.
> > > > The tradeoff I see is this: taking byte[] is actually simpler, the
> user
> > > can
> > > > directly do whatever serialization they like. The complication is
> > > actually
> > > > partitioning. Currently partitioning is done by a similar plug-in api
> > > > (Partitioner) which the user can implement and configure to override
> > how
> > > > partitions are assigned. If we take byte[] as input then we have no
> > > access
> > > > to the original object and partitioning MUST be done on the byte[].
> > This
> > > is
> > > > fine for hash partitioning. However for various types of semantic
> > > > partitioning (range partitioning, or whatever) you would want access
> to
> > > the
> > > > original object. In the current approach a producer who wishes to
> send
> > > > byte[] they have serialized in their own code can configure the
> > > > BytesSerialization we supply which is just a "no op" serialization.
> > > > 2. We should obsess over naming and make sure each of the class names
> > are
> > > > good.
> > > > 3. Jun has already pointed out that we need to include the topic and
> > > > partition in the response, which is absolutely right. I haven't done
> > that
> > > > yet but that definitely needs to be there.
> > > > 4. Currently RecordSend.await will throw an exception if the request
> > > > failed. The intention here is that producer.send(message).await()
> > exactly
> > > > simulates a synchronous call. Guozhang has noted that this is a
> little
> > > > annoying since the user must then catch exceptions. However if we
> > remove
> > > > this then if the user doesn't check for errors they won't know one
> has
> > > > occurred, which I predict will be a common mistake.
> > > > 5. Perhaps there is more we could do to make the async callbacks and
> > > future
> > > > we give back intuitive and easy to program against?
> > > >
> > > > Some background info on implementation:
> > > >
> > > > At a high level the primary difference in this producer is that it
> > > removes
> > > > the distinction between the "sync" and "async" producer. Effectively
> > all
> > > > requests are sent asynchronously but always return a future response
> > > object
> > > > that gives the offset as well as any error that may have occurred
> when
> > > the
> > > > request is complete. The batching that is done in the async producer
> > only
> > > > today is done whenever possible now. This means that the sync
> producer,
> > > > under load, can get performance as good as the async producer
> > > (preliminary
> > > > results show the producer getting 1m messages/sec). This works
> similar
> > to
> > > > group commit in databases but with respect to the actual network
> > > > transmission--any messages that arrive while a send is in progress
> are
> > > > batched together. It is also possible to encourage batching even
> under
> > > low
> > > > load to save server resources by introducing a delay on the send to
> > allow
> > > > more messages to accumulate; this is done using the linger.ms config
> > > (this
> > > > is similar to Nagle's algorithm in TCP).
> > > >
> > > > This producer does all network communication asynchronously and in
> > > parallel
> > > > to all servers so the performance penalty for acks=-1 and waiting on
> > > > replication should be much reduced. I haven't done much benchmarking
> on
> > > > this yet, though.
> > > >
> > > > The high level design is described a little here, though this is now
> a
> > > > little out of date:
> > > > https://cwiki.apache.org/confluence/display/KAFKA/Client+Rewrite
> > > >
> > > > -Jay
> > > >
> > >
> >
>

Re: New Producer Public API

Posted by Joel Koshy <jj...@gmail.com>.
> The trouble with callbacks, IMHO, is determining the thread in which they
> will be executed. Since the IO thread is usually the thread that knows when
> the operation is complete, it's easiest to execute that callback within the
> IO thread. This can lead the IO thread to spend all its time on callbacks
> and not IO.
> 
> Is there an executor service associated with the client that actually
> executes the callback, or is the callback executed by the IO thread?

I think in the current design/prototype the callback is called from
the IO thread - which is it needs to be fast. Since the callback is
provided by the client, it could submit a longer-running task to an
executor service (or signal some other thread that a RecordSend is
complete). However, the task submission would still happen from the IO
thread.

> 
> 
> On Fri, Jan 31, 2014 at 10:33 AM, Oliver Dain <od...@3cinteractive.com>wrote:
> 
> > Hey all,
> >
> > I¹m excited about having a new Producer API, and I really like the idea of
> > removing the distinction between a synchronous and asynchronous producer.
> > The one comment I have about the current API is that it¹s hard to write
> > truly asynchronous code with the type of future returned by the send
> > method. The issue is that send returns a RecordSend and there¹s no way to
> > register a callback with that object. It is therefore necessary to poll
> > the object periodically to see if the send has completed. So if you have n
> > send calls outstanding you have to check n RecordSend objects which is
> > slow. In general this tends to lead to people using one thread per send
> > call and then calling RecordSend#await which removes much of the benefit
> > of an async API.
> >
> > I think it¹s much easier to write truly asynchronous code if the returned
> > future allows you to register a callback. That way, instead of polling you
> > can simply wait for the callback to be called. A good example of the kind
> > of thing I¹m thinking is the ListenableFuture class in the Guava libraries:
> >
> > https://code.google.com/p/guava-libraries/wiki/ListenableFutureExplained
> >
> >
> > HTH,
> > Oliver
> >
> >

-- 
Joel

Re: New Producer Public API

Posted by Tom Brown <to...@gmail.com>.
The trouble with callbacks, IMHO, is determining the thread in which they
will be executed. Since the IO thread is usually the thread that knows when
the operation is complete, it's easiest to execute that callback within the
IO thread. This can lead the IO thread to spend all its time on callbacks
and not IO.

Is there an executor service associated with the client that actually
executes the callback, or is the callback executed by the IO thread?

--Tom


On Fri, Jan 31, 2014 at 10:33 AM, Oliver Dain <od...@3cinteractive.com>wrote:

> Hey all,
>
> I¹m excited about having a new Producer API, and I really like the idea of
> removing the distinction between a synchronous and asynchronous producer.
> The one comment I have about the current API is that it¹s hard to write
> truly asynchronous code with the type of future returned by the send
> method. The issue is that send returns a RecordSend and there¹s no way to
> register a callback with that object. It is therefore necessary to poll
> the object periodically to see if the send has completed. So if you have n
> send calls outstanding you have to check n RecordSend objects which is
> slow. In general this tends to lead to people using one thread per send
> call and then calling RecordSend#await which removes much of the benefit
> of an async API.
>
> I think it¹s much easier to write truly asynchronous code if the returned
> future allows you to register a callback. That way, instead of polling you
> can simply wait for the callback to be called. A good example of the kind
> of thing I¹m thinking is the ListenableFuture class in the Guava libraries:
>
> https://code.google.com/p/guava-libraries/wiki/ListenableFutureExplained
>
>
> HTH,
> Oliver
>
>

Re: New Producer Public API

Posted by Joel Koshy <jj...@gmail.com>.
+ dev

(this thread has become a bit unwieldy)


On Thu, Jan 30, 2014 at 5:15 PM, Joel Koshy <jj...@gmail.com> wrote:
> Does it preclude those various implementations? i.e., it could become
> a producer config:
> default.partitioner.strategy="minimize-connections"/"roundrobin" - and
> so on; and implement those partitioners internally in the producer.
> Not as clear as a .class config, but it accomplishes the same effect
> no?
>
> On Thu, Jan 30, 2014 at 4:14 PM, Jay Kreps <ja...@gmail.com> wrote:
>> One downside to the 1A proposal is that without a Partitioner interface we
>> can't really package up and provide common partitioner implementations.
>> Example of these would be
>> 1. HashPartitioner - The default hash partitioning
>> 2. RoundRobinPartitioner - Just round-robins over partitions
>> 3. ConnectionMinimizingPartitioner - Choose partitions to minimize the
>> number of nodes you need to connect maintain TCP connections to.
>> 4. RangePartitioner - User provides break points that align partitions to
>> key ranges
>> 5. LocalityPartitioner - Prefer nodes on the same rack. This would be nice
>> for stream-processing use cases that read from one topic and write to
>> another. We would have to include rack information in our metadata.
>>
>> Having this kind of functionality included is actually kind of nice.
>>
>> -Jay
>>
>>
>> On Fri, Jan 24, 2014 at 5:17 PM, Jay Kreps <ja...@gmail.com> wrote:
>>
>>> Clark and all,
>>>
>>> I thought a little bit about the serialization question. Here are the
>>> options I see and the pros and cons I can think of. I'd love to hear
>>> people's preferences if you have a strong one.
>>>
>>> One important consideration is that however the producer works will also
>>> need to be how the new consumer works (which we hope to write next). That
>>> is if you put objects in, you should get objects out. So we need to think
>>> through both sides.
>>>
>>> Options:
>>>
>>> Option 0: What is in the existing scala code and the java code I
>>> posted--Serializer and Partitioner plugin provided by the user via config.
>>> Partitioner has a sane default, but Serializer needs to be specified in
>>> config.
>>>
>>> Pros: How it works today in the scala code.
>>> Cons: You have to adapt your serialization library of choice to our
>>> interfaces. The reflective class loading means typo in the serializer name
>>> give odd errors. Likewise there is little type safety--the ProducerRecord
>>> takes Object and any type errors between the object provided and the
>>> serializer give occurs at runtime.
>>>
>>> Option 1: No plugins
>>>
>>> This would mean byte[] key, byte[] value, and partitioning done by client
>>> by passing in a partition *number* directly.
>>>
>>> The problem with this is that it is tricky to compute the partition
>>> correctly and probably most people won't. We could add a getCluster()
>>> method to return the Cluster instance you should use for partitioning. But
>>> I suspect people would be lazy and not use that and instead hard-code
>>> partitions which would break if partitions were added or they hard coded it
>>> wrong. In my experience 3 partitioning strategies cover like 99% of cases
>>> so not having a default implementation for this makes the common case
>>> harder. Left to their own devices people will use bad hash functions and
>>> get weird results.
>>>
>>> Option 1A: Alternatively we could partition by the key using the existing
>>> default partitioning strategy which only uses the byte[] anyway but instead
>>> of having a partitionKey we could have a numerical partition override and
>>> add the getCluster() method to get the cluster metadata. That would make
>>> custom partitioning possible but handle the common case simply.
>>>
>>> Option 2: Partitioner plugin remains, serializers go.
>>>
>>> The problem here is that the partitioner might lose access to the
>>> deserialized key which would occasionally be useful for semantic
>>> partitioning schemes. The Partitioner could deserialize the key but that
>>> would be inefficient and weird.
>>>
>>> This problem could be fixed by having key and value be byte[] but
>>> retaining partitionKey as an Object and passing it to the partitioner as
>>> is. Then if you have a partitioner which requires the deserialized key you
>>> would need to use this partition key. One weird side effect is that if you
>>> want to have a custom partition key BUT want to partition by the bytes of
>>> that key rather than the object value you must write a customer partitioner
>>> and serialize it yourself.
>>>
>>> Of these I think I prefer 1A but could be convinced of 0 since that is how
>>> it works now.
>>>
>>> Thoughts?
>>>
>>> -Jay
>>>
>>>
>>> On Fri, Jan 24, 2014 at 3:30 PM, Clark Breyman <cl...@breyman.com> wrote:
>>>
>>>> Jay - Thanks for the call for comments. Here's some initial input:
>>>>
>>>> - Make message serialization a client responsibility (making all messages
>>>> byte[]). Reflection-based loading makes it harder to use generic codecs
>>>> (e.g.  Envelope<PREFIX, DATA, SUFFIX>) or build up codec programmatically.
>>>> Non-default partitioning should require an explicit partition key.
>>>>
>>>> - I really like the fact that it will be native Java. Please consider
>>>> using
>>>> native maven and not sbt, gradle, ivy, etc as they don't reliably play
>>>> nice
>>>> in the maven ecosystem. A jar without a well-formed pom doesn't feel like
>>>> a
>>>> real artifact. The pom's generated by sbt et al. are not well formed.
>>>> Using
>>>> maven will make builds and IDE integration much smoother.
>>>>
>>>> - Look at Nick Telford's dropwizard-extras package in which he defines
>>>> some
>>>> Jackson-compatible POJO's for loading configuration. Seems like your
>>>> client
>>>> migration is similar. The config objects should have constructors or
>>>> factories that accept Map<String, String> and Properties for ease of
>>>> migration.
>>>>
>>>> - Would you consider using the org.apache.kafka package for the new API
>>>> (quibble)
>>>>
>>>> - Why create your own futures rather than use
>>>> java.util.concurrent.Future<Long> or similar? Standard futures will play
>>>> nice with other reactive libs and things like J8's ComposableFuture.
>>>>
>>>> Thanks again,
>>>> C
>>>>
>>>>
>>>>
>>>> On Fri, Jan 24, 2014 at 2:46 PM, Roger Hoover <roger.hoover@gmail.com
>>>> >wrote:
>>>>
>>>> > A couple comments:
>>>> >
>>>> > 1) Why does the config use a broker list instead of discovering the
>>>> brokers
>>>> > in ZooKeeper?  It doesn't match the HighLevelConsumer API.
>>>> >
>>>> > 2) It looks like broker connections are created on demand.  I'm
>>>> wondering
>>>> > if sometimes you might want to flush out config or network connectivity
>>>> > issues before pushing the first message through.
>>>> >
>>>> > Should there also be a KafkaProducer.connect() or .open() method or
>>>> > connectAll()?  I guess it would try to connect to all brokers in the
>>>> > BROKER_LIST_CONFIG
>>>> >
>>>> > HTH,
>>>> >
>>>> > Roger
>>>> >
>>>> >
>>>> > On Fri, Jan 24, 2014 at 11:54 AM, Jay Kreps <ja...@gmail.com>
>>>> wrote:
>>>> >
>>>> > > As mentioned in a previous email we are working on a
>>>> re-implementation of
>>>> > > the producer. I would like to use this email thread to discuss the
>>>> > details
>>>> > > of the public API and the configuration. I would love for us to be
>>>> > > incredibly picky about this public api now so it is as good as
>>>> possible
>>>> > and
>>>> > > we don't need to break it in the future.
>>>> > >
>>>> > > The best way to get a feel for the API is actually to take a look at
>>>> the
>>>> > > javadoc, my hope is to get the api docs good enough so that it is
>>>> > > self-explanatory:
>>>> > >
>>>> > >
>>>> >
>>>> http://empathybox.com/kafka-javadoc/index.html?kafka/clients/producer/KafkaProducer.html
>>>> > >
>>>> > > Please take a look at this API and give me any thoughts you may have!
>>>> > >
>>>> > > It may also be reasonable to take a look at the configs:
>>>> > >
>>>> > >
>>>> >
>>>> http://empathybox.com/kafka-javadoc/kafka/clients/producer/ProducerConfig.html
>>>> > >
>>>> > > The actual code is posted here:
>>>> > > https://issues.apache.org/jira/browse/KAFKA-1227
>>>> > >
>>>> > > A few questions or comments to kick things off:
>>>> > > 1. We need to make a decision on whether serialization of the user's
>>>> key
>>>> > > and value should be done by the user (with our api just taking
>>>> byte[]) or
>>>> > > if we should take an object and allow the user to configure a
>>>> Serializer
>>>> > > class which we instantiate via reflection. We take the later approach
>>>> in
>>>> > > the current producer, and I have carried this through to this
>>>> prototype.
>>>> > > The tradeoff I see is this: taking byte[] is actually simpler, the
>>>> user
>>>> > can
>>>> > > directly do whatever serialization they like. The complication is
>>>> > actually
>>>> > > partitioning. Currently partitioning is done by a similar plug-in api
>>>> > > (Partitioner) which the user can implement and configure to override
>>>> how
>>>> > > partitions are assigned. If we take byte[] as input then we have no
>>>> > access
>>>> > > to the original object and partitioning MUST be done on the byte[].
>>>> This
>>>> > is
>>>> > > fine for hash partitioning. However for various types of semantic
>>>> > > partitioning (range partitioning, or whatever) you would want access
>>>> to
>>>> > the
>>>> > > original object. In the current approach a producer who wishes to send
>>>> > > byte[] they have serialized in their own code can configure the
>>>> > > BytesSerialization we supply which is just a "no op" serialization.
>>>> > > 2. We should obsess over naming and make sure each of the class names
>>>> are
>>>> > > good.
>>>> > > 3. Jun has already pointed out that we need to include the topic and
>>>> > > partition in the response, which is absolutely right. I haven't done
>>>> that
>>>> > > yet but that definitely needs to be there.
>>>> > > 4. Currently RecordSend.await will throw an exception if the request
>>>> > > failed. The intention here is that producer.send(message).await()
>>>> exactly
>>>> > > simulates a synchronous call. Guozhang has noted that this is a little
>>>> > > annoying since the user must then catch exceptions. However if we
>>>> remove
>>>> > > this then if the user doesn't check for errors they won't know one has
>>>> > > occurred, which I predict will be a common mistake.
>>>> > > 5. Perhaps there is more we could do to make the async callbacks and
>>>> > future
>>>> > > we give back intuitive and easy to program against?
>>>> > >
>>>> > > Some background info on implementation:
>>>> > >
>>>> > > At a high level the primary difference in this producer is that it
>>>> > removes
>>>> > > the distinction between the "sync" and "async" producer. Effectively
>>>> all
>>>> > > requests are sent asynchronously but always return a future response
>>>> > object
>>>> > > that gives the offset as well as any error that may have occurred when
>>>> > the
>>>> > > request is complete. The batching that is done in the async producer
>>>> only
>>>> > > today is done whenever possible now. This means that the sync
>>>> producer,
>>>> > > under load, can get performance as good as the async producer
>>>> > (preliminary
>>>> > > results show the producer getting 1m messages/sec). This works
>>>> similar to
>>>> > > group commit in databases but with respect to the actual network
>>>> > > transmission--any messages that arrive while a send is in progress are
>>>> > > batched together. It is also possible to encourage batching even under
>>>> > low
>>>> > > load to save server resources by introducing a delay on the send to
>>>> allow
>>>> > > more messages to accumulate; this is done using the linger.ms config
>>>> > (this
>>>> > > is similar to Nagle's algorithm in TCP).
>>>> > >
>>>> > > This producer does all network communication asynchronously and in
>>>> > parallel
>>>> > > to all servers so the performance penalty for acks=-1 and waiting on
>>>> > > replication should be much reduced. I haven't done much benchmarking
>>>> on
>>>> > > this yet, though.
>>>> > >
>>>> > > The high level design is described a little here, though this is now a
>>>> > > little out of date:
>>>> > > https://cwiki.apache.org/confluence/display/KAFKA/Client+Rewrite
>>>> > >
>>>> > > -Jay
>>>> > >
>>>> >
>>>>
>>>
>>>

Re: New Producer Public API

Posted by Jay Kreps <ja...@gmail.com>.
Joel--

Yeah we could theoretically retain a neutered Partitioner interface that
only had access to the byte[] key not the original object (which we no
longer have). Ideologically most partitioning should really happen based on
the byte[] not the original object to retain multi-language compatibility,
but sometimes the object is useful.

I kind of think this is one of those things where doing either A or B is
better than doing A and B both just for clarity.

-Jay


On Thu, Jan 30, 2014 at 5:15 PM, Joel Koshy <jj...@gmail.com> wrote:

> Does it preclude those various implementations? i.e., it could become
> a producer config:
> default.partitioner.strategy="minimize-connections"/"roundrobin" - and
> so on; and implement those partitioners internally in the producer.
> Not as clear as a .class config, but it accomplishes the same effect
> no?
>
> On Thu, Jan 30, 2014 at 4:14 PM, Jay Kreps <ja...@gmail.com> wrote:
> > One downside to the 1A proposal is that without a Partitioner interface
> we
> > can't really package up and provide common partitioner implementations.
> > Example of these would be
> > 1. HashPartitioner - The default hash partitioning
> > 2. RoundRobinPartitioner - Just round-robins over partitions
> > 3. ConnectionMinimizingPartitioner - Choose partitions to minimize the
> > number of nodes you need to connect maintain TCP connections to.
> > 4. RangePartitioner - User provides break points that align partitions to
> > key ranges
> > 5. LocalityPartitioner - Prefer nodes on the same rack. This would be
> nice
> > for stream-processing use cases that read from one topic and write to
> > another. We would have to include rack information in our metadata.
> >
> > Having this kind of functionality included is actually kind of nice.
> >
> > -Jay
> >
> >
> > On Fri, Jan 24, 2014 at 5:17 PM, Jay Kreps <ja...@gmail.com> wrote:
> >
> >> Clark and all,
> >>
> >> I thought a little bit about the serialization question. Here are the
> >> options I see and the pros and cons I can think of. I'd love to hear
> >> people's preferences if you have a strong one.
> >>
> >> One important consideration is that however the producer works will also
> >> need to be how the new consumer works (which we hope to write next).
> That
> >> is if you put objects in, you should get objects out. So we need to
> think
> >> through both sides.
> >>
> >> Options:
> >>
> >> Option 0: What is in the existing scala code and the java code I
> >> posted--Serializer and Partitioner plugin provided by the user via
> config.
> >> Partitioner has a sane default, but Serializer needs to be specified in
> >> config.
> >>
> >> Pros: How it works today in the scala code.
> >> Cons: You have to adapt your serialization library of choice to our
> >> interfaces. The reflective class loading means typo in the serializer
> name
> >> give odd errors. Likewise there is little type safety--the
> ProducerRecord
> >> takes Object and any type errors between the object provided and the
> >> serializer give occurs at runtime.
> >>
> >> Option 1: No plugins
> >>
> >> This would mean byte[] key, byte[] value, and partitioning done by
> client
> >> by passing in a partition *number* directly.
> >>
> >> The problem with this is that it is tricky to compute the partition
> >> correctly and probably most people won't. We could add a getCluster()
> >> method to return the Cluster instance you should use for partitioning.
> But
> >> I suspect people would be lazy and not use that and instead hard-code
> >> partitions which would break if partitions were added or they hard
> coded it
> >> wrong. In my experience 3 partitioning strategies cover like 99% of
> cases
> >> so not having a default implementation for this makes the common case
> >> harder. Left to their own devices people will use bad hash functions and
> >> get weird results.
> >>
> >> Option 1A: Alternatively we could partition by the key using the
> existing
> >> default partitioning strategy which only uses the byte[] anyway but
> instead
> >> of having a partitionKey we could have a numerical partition override
> and
> >> add the getCluster() method to get the cluster metadata. That would make
> >> custom partitioning possible but handle the common case simply.
> >>
> >> Option 2: Partitioner plugin remains, serializers go.
> >>
> >> The problem here is that the partitioner might lose access to the
> >> deserialized key which would occasionally be useful for semantic
> >> partitioning schemes. The Partitioner could deserialize the key but that
> >> would be inefficient and weird.
> >>
> >> This problem could be fixed by having key and value be byte[] but
> >> retaining partitionKey as an Object and passing it to the partitioner as
> >> is. Then if you have a partitioner which requires the deserialized key
> you
> >> would need to use this partition key. One weird side effect is that if
> you
> >> want to have a custom partition key BUT want to partition by the bytes
> of
> >> that key rather than the object value you must write a customer
> partitioner
> >> and serialize it yourself.
> >>
> >> Of these I think I prefer 1A but could be convinced of 0 since that is
> how
> >> it works now.
> >>
> >> Thoughts?
> >>
> >> -Jay
> >>
> >>
> >> On Fri, Jan 24, 2014 at 3:30 PM, Clark Breyman <cl...@breyman.com>
> wrote:
> >>
> >>> Jay - Thanks for the call for comments. Here's some initial input:
> >>>
> >>> - Make message serialization a client responsibility (making all
> messages
> >>> byte[]). Reflection-based loading makes it harder to use generic codecs
> >>> (e.g.  Envelope<PREFIX, DATA, SUFFIX>) or build up codec
> programmatically.
> >>> Non-default partitioning should require an explicit partition key.
> >>>
> >>> - I really like the fact that it will be native Java. Please consider
> >>> using
> >>> native maven and not sbt, gradle, ivy, etc as they don't reliably play
> >>> nice
> >>> in the maven ecosystem. A jar without a well-formed pom doesn't feel
> like
> >>> a
> >>> real artifact. The pom's generated by sbt et al. are not well formed.
> >>> Using
> >>> maven will make builds and IDE integration much smoother.
> >>>
> >>> - Look at Nick Telford's dropwizard-extras package in which he defines
> >>> some
> >>> Jackson-compatible POJO's for loading configuration. Seems like your
> >>> client
> >>> migration is similar. The config objects should have constructors or
> >>> factories that accept Map<String, String> and Properties for ease of
> >>> migration.
> >>>
> >>> - Would you consider using the org.apache.kafka package for the new API
> >>> (quibble)
> >>>
> >>> - Why create your own futures rather than use
> >>> java.util.concurrent.Future<Long> or similar? Standard futures will
> play
> >>> nice with other reactive libs and things like J8's ComposableFuture.
> >>>
> >>> Thanks again,
> >>> C
> >>>
> >>>
> >>>
> >>> On Fri, Jan 24, 2014 at 2:46 PM, Roger Hoover <roger.hoover@gmail.com
> >>> >wrote:
> >>>
> >>> > A couple comments:
> >>> >
> >>> > 1) Why does the config use a broker list instead of discovering the
> >>> brokers
> >>> > in ZooKeeper?  It doesn't match the HighLevelConsumer API.
> >>> >
> >>> > 2) It looks like broker connections are created on demand.  I'm
> >>> wondering
> >>> > if sometimes you might want to flush out config or network
> connectivity
> >>> > issues before pushing the first message through.
> >>> >
> >>> > Should there also be a KafkaProducer.connect() or .open() method or
> >>> > connectAll()?  I guess it would try to connect to all brokers in the
> >>> > BROKER_LIST_CONFIG
> >>> >
> >>> > HTH,
> >>> >
> >>> > Roger
> >>> >
> >>> >
> >>> > On Fri, Jan 24, 2014 at 11:54 AM, Jay Kreps <ja...@gmail.com>
> >>> wrote:
> >>> >
> >>> > > As mentioned in a previous email we are working on a
> >>> re-implementation of
> >>> > > the producer. I would like to use this email thread to discuss the
> >>> > details
> >>> > > of the public API and the configuration. I would love for us to be
> >>> > > incredibly picky about this public api now so it is as good as
> >>> possible
> >>> > and
> >>> > > we don't need to break it in the future.
> >>> > >
> >>> > > The best way to get a feel for the API is actually to take a look
> at
> >>> the
> >>> > > javadoc, my hope is to get the api docs good enough so that it is
> >>> > > self-explanatory:
> >>> > >
> >>> > >
> >>> >
> >>>
> http://empathybox.com/kafka-javadoc/index.html?kafka/clients/producer/KafkaProducer.html
> >>> > >
> >>> > > Please take a look at this API and give me any thoughts you may
> have!
> >>> > >
> >>> > > It may also be reasonable to take a look at the configs:
> >>> > >
> >>> > >
> >>> >
> >>>
> http://empathybox.com/kafka-javadoc/kafka/clients/producer/ProducerConfig.html
> >>> > >
> >>> > > The actual code is posted here:
> >>> > > https://issues.apache.org/jira/browse/KAFKA-1227
> >>> > >
> >>> > > A few questions or comments to kick things off:
> >>> > > 1. We need to make a decision on whether serialization of the
> user's
> >>> key
> >>> > > and value should be done by the user (with our api just taking
> >>> byte[]) or
> >>> > > if we should take an object and allow the user to configure a
> >>> Serializer
> >>> > > class which we instantiate via reflection. We take the later
> approach
> >>> in
> >>> > > the current producer, and I have carried this through to this
> >>> prototype.
> >>> > > The tradeoff I see is this: taking byte[] is actually simpler, the
> >>> user
> >>> > can
> >>> > > directly do whatever serialization they like. The complication is
> >>> > actually
> >>> > > partitioning. Currently partitioning is done by a similar plug-in
> api
> >>> > > (Partitioner) which the user can implement and configure to
> override
> >>> how
> >>> > > partitions are assigned. If we take byte[] as input then we have no
> >>> > access
> >>> > > to the original object and partitioning MUST be done on the byte[].
> >>> This
> >>> > is
> >>> > > fine for hash partitioning. However for various types of semantic
> >>> > > partitioning (range partitioning, or whatever) you would want
> access
> >>> to
> >>> > the
> >>> > > original object. In the current approach a producer who wishes to
> send
> >>> > > byte[] they have serialized in their own code can configure the
> >>> > > BytesSerialization we supply which is just a "no op" serialization.
> >>> > > 2. We should obsess over naming and make sure each of the class
> names
> >>> are
> >>> > > good.
> >>> > > 3. Jun has already pointed out that we need to include the topic
> and
> >>> > > partition in the response, which is absolutely right. I haven't
> done
> >>> that
> >>> > > yet but that definitely needs to be there.
> >>> > > 4. Currently RecordSend.await will throw an exception if the
> request
> >>> > > failed. The intention here is that producer.send(message).await()
> >>> exactly
> >>> > > simulates a synchronous call. Guozhang has noted that this is a
> little
> >>> > > annoying since the user must then catch exceptions. However if we
> >>> remove
> >>> > > this then if the user doesn't check for errors they won't know one
> has
> >>> > > occurred, which I predict will be a common mistake.
> >>> > > 5. Perhaps there is more we could do to make the async callbacks
> and
> >>> > future
> >>> > > we give back intuitive and easy to program against?
> >>> > >
> >>> > > Some background info on implementation:
> >>> > >
> >>> > > At a high level the primary difference in this producer is that it
> >>> > removes
> >>> > > the distinction between the "sync" and "async" producer.
> Effectively
> >>> all
> >>> > > requests are sent asynchronously but always return a future
> response
> >>> > object
> >>> > > that gives the offset as well as any error that may have occurred
> when
> >>> > the
> >>> > > request is complete. The batching that is done in the async
> producer
> >>> only
> >>> > > today is done whenever possible now. This means that the sync
> >>> producer,
> >>> > > under load, can get performance as good as the async producer
> >>> > (preliminary
> >>> > > results show the producer getting 1m messages/sec). This works
> >>> similar to
> >>> > > group commit in databases but with respect to the actual network
> >>> > > transmission--any messages that arrive while a send is in progress
> are
> >>> > > batched together. It is also possible to encourage batching even
> under
> >>> > low
> >>> > > load to save server resources by introducing a delay on the send to
> >>> allow
> >>> > > more messages to accumulate; this is done using the linger.msconfig
> >>> > (this
> >>> > > is similar to Nagle's algorithm in TCP).
> >>> > >
> >>> > > This producer does all network communication asynchronously and in
> >>> > parallel
> >>> > > to all servers so the performance penalty for acks=-1 and waiting
> on
> >>> > > replication should be much reduced. I haven't done much
> benchmarking
> >>> on
> >>> > > this yet, though.
> >>> > >
> >>> > > The high level design is described a little here, though this is
> now a
> >>> > > little out of date:
> >>> > > https://cwiki.apache.org/confluence/display/KAFKA/Client+Rewrite
> >>> > >
> >>> > > -Jay
> >>> > >
> >>> >
> >>>
> >>
> >>
>

Re: New Producer Public API

Posted by Joel Koshy <jj...@gmail.com>.
+ dev

(this thread has become a bit unwieldy)


On Thu, Jan 30, 2014 at 5:15 PM, Joel Koshy <jj...@gmail.com> wrote:
> Does it preclude those various implementations? i.e., it could become
> a producer config:
> default.partitioner.strategy="minimize-connections"/"roundrobin" - and
> so on; and implement those partitioners internally in the producer.
> Not as clear as a .class config, but it accomplishes the same effect
> no?
>
> On Thu, Jan 30, 2014 at 4:14 PM, Jay Kreps <ja...@gmail.com> wrote:
>> One downside to the 1A proposal is that without a Partitioner interface we
>> can't really package up and provide common partitioner implementations.
>> Example of these would be
>> 1. HashPartitioner - The default hash partitioning
>> 2. RoundRobinPartitioner - Just round-robins over partitions
>> 3. ConnectionMinimizingPartitioner - Choose partitions to minimize the
>> number of nodes you need to connect maintain TCP connections to.
>> 4. RangePartitioner - User provides break points that align partitions to
>> key ranges
>> 5. LocalityPartitioner - Prefer nodes on the same rack. This would be nice
>> for stream-processing use cases that read from one topic and write to
>> another. We would have to include rack information in our metadata.
>>
>> Having this kind of functionality included is actually kind of nice.
>>
>> -Jay
>>
>>
>> On Fri, Jan 24, 2014 at 5:17 PM, Jay Kreps <ja...@gmail.com> wrote:
>>
>>> Clark and all,
>>>
>>> I thought a little bit about the serialization question. Here are the
>>> options I see and the pros and cons I can think of. I'd love to hear
>>> people's preferences if you have a strong one.
>>>
>>> One important consideration is that however the producer works will also
>>> need to be how the new consumer works (which we hope to write next). That
>>> is if you put objects in, you should get objects out. So we need to think
>>> through both sides.
>>>
>>> Options:
>>>
>>> Option 0: What is in the existing scala code and the java code I
>>> posted--Serializer and Partitioner plugin provided by the user via config.
>>> Partitioner has a sane default, but Serializer needs to be specified in
>>> config.
>>>
>>> Pros: How it works today in the scala code.
>>> Cons: You have to adapt your serialization library of choice to our
>>> interfaces. The reflective class loading means typo in the serializer name
>>> give odd errors. Likewise there is little type safety--the ProducerRecord
>>> takes Object and any type errors between the object provided and the
>>> serializer give occurs at runtime.
>>>
>>> Option 1: No plugins
>>>
>>> This would mean byte[] key, byte[] value, and partitioning done by client
>>> by passing in a partition *number* directly.
>>>
>>> The problem with this is that it is tricky to compute the partition
>>> correctly and probably most people won't. We could add a getCluster()
>>> method to return the Cluster instance you should use for partitioning. But
>>> I suspect people would be lazy and not use that and instead hard-code
>>> partitions which would break if partitions were added or they hard coded it
>>> wrong. In my experience 3 partitioning strategies cover like 99% of cases
>>> so not having a default implementation for this makes the common case
>>> harder. Left to their own devices people will use bad hash functions and
>>> get weird results.
>>>
>>> Option 1A: Alternatively we could partition by the key using the existing
>>> default partitioning strategy which only uses the byte[] anyway but instead
>>> of having a partitionKey we could have a numerical partition override and
>>> add the getCluster() method to get the cluster metadata. That would make
>>> custom partitioning possible but handle the common case simply.
>>>
>>> Option 2: Partitioner plugin remains, serializers go.
>>>
>>> The problem here is that the partitioner might lose access to the
>>> deserialized key which would occasionally be useful for semantic
>>> partitioning schemes. The Partitioner could deserialize the key but that
>>> would be inefficient and weird.
>>>
>>> This problem could be fixed by having key and value be byte[] but
>>> retaining partitionKey as an Object and passing it to the partitioner as
>>> is. Then if you have a partitioner which requires the deserialized key you
>>> would need to use this partition key. One weird side effect is that if you
>>> want to have a custom partition key BUT want to partition by the bytes of
>>> that key rather than the object value you must write a customer partitioner
>>> and serialize it yourself.
>>>
>>> Of these I think I prefer 1A but could be convinced of 0 since that is how
>>> it works now.
>>>
>>> Thoughts?
>>>
>>> -Jay
>>>
>>>
>>> On Fri, Jan 24, 2014 at 3:30 PM, Clark Breyman <cl...@breyman.com> wrote:
>>>
>>>> Jay - Thanks for the call for comments. Here's some initial input:
>>>>
>>>> - Make message serialization a client responsibility (making all messages
>>>> byte[]). Reflection-based loading makes it harder to use generic codecs
>>>> (e.g.  Envelope<PREFIX, DATA, SUFFIX>) or build up codec programmatically.
>>>> Non-default partitioning should require an explicit partition key.
>>>>
>>>> - I really like the fact that it will be native Java. Please consider
>>>> using
>>>> native maven and not sbt, gradle, ivy, etc as they don't reliably play
>>>> nice
>>>> in the maven ecosystem. A jar without a well-formed pom doesn't feel like
>>>> a
>>>> real artifact. The pom's generated by sbt et al. are not well formed.
>>>> Using
>>>> maven will make builds and IDE integration much smoother.
>>>>
>>>> - Look at Nick Telford's dropwizard-extras package in which he defines
>>>> some
>>>> Jackson-compatible POJO's for loading configuration. Seems like your
>>>> client
>>>> migration is similar. The config objects should have constructors or
>>>> factories that accept Map<String, String> and Properties for ease of
>>>> migration.
>>>>
>>>> - Would you consider using the org.apache.kafka package for the new API
>>>> (quibble)
>>>>
>>>> - Why create your own futures rather than use
>>>> java.util.concurrent.Future<Long> or similar? Standard futures will play
>>>> nice with other reactive libs and things like J8's ComposableFuture.
>>>>
>>>> Thanks again,
>>>> C
>>>>
>>>>
>>>>
>>>> On Fri, Jan 24, 2014 at 2:46 PM, Roger Hoover <roger.hoover@gmail.com
>>>> >wrote:
>>>>
>>>> > A couple comments:
>>>> >
>>>> > 1) Why does the config use a broker list instead of discovering the
>>>> brokers
>>>> > in ZooKeeper?  It doesn't match the HighLevelConsumer API.
>>>> >
>>>> > 2) It looks like broker connections are created on demand.  I'm
>>>> wondering
>>>> > if sometimes you might want to flush out config or network connectivity
>>>> > issues before pushing the first message through.
>>>> >
>>>> > Should there also be a KafkaProducer.connect() or .open() method or
>>>> > connectAll()?  I guess it would try to connect to all brokers in the
>>>> > BROKER_LIST_CONFIG
>>>> >
>>>> > HTH,
>>>> >
>>>> > Roger
>>>> >
>>>> >
>>>> > On Fri, Jan 24, 2014 at 11:54 AM, Jay Kreps <ja...@gmail.com>
>>>> wrote:
>>>> >
>>>> > > As mentioned in a previous email we are working on a
>>>> re-implementation of
>>>> > > the producer. I would like to use this email thread to discuss the
>>>> > details
>>>> > > of the public API and the configuration. I would love for us to be
>>>> > > incredibly picky about this public api now so it is as good as
>>>> possible
>>>> > and
>>>> > > we don't need to break it in the future.
>>>> > >
>>>> > > The best way to get a feel for the API is actually to take a look at
>>>> the
>>>> > > javadoc, my hope is to get the api docs good enough so that it is
>>>> > > self-explanatory:
>>>> > >
>>>> > >
>>>> >
>>>> http://empathybox.com/kafka-javadoc/index.html?kafka/clients/producer/KafkaProducer.html
>>>> > >
>>>> > > Please take a look at this API and give me any thoughts you may have!
>>>> > >
>>>> > > It may also be reasonable to take a look at the configs:
>>>> > >
>>>> > >
>>>> >
>>>> http://empathybox.com/kafka-javadoc/kafka/clients/producer/ProducerConfig.html
>>>> > >
>>>> > > The actual code is posted here:
>>>> > > https://issues.apache.org/jira/browse/KAFKA-1227
>>>> > >
>>>> > > A few questions or comments to kick things off:
>>>> > > 1. We need to make a decision on whether serialization of the user's
>>>> key
>>>> > > and value should be done by the user (with our api just taking
>>>> byte[]) or
>>>> > > if we should take an object and allow the user to configure a
>>>> Serializer
>>>> > > class which we instantiate via reflection. We take the later approach
>>>> in
>>>> > > the current producer, and I have carried this through to this
>>>> prototype.
>>>> > > The tradeoff I see is this: taking byte[] is actually simpler, the
>>>> user
>>>> > can
>>>> > > directly do whatever serialization they like. The complication is
>>>> > actually
>>>> > > partitioning. Currently partitioning is done by a similar plug-in api
>>>> > > (Partitioner) which the user can implement and configure to override
>>>> how
>>>> > > partitions are assigned. If we take byte[] as input then we have no
>>>> > access
>>>> > > to the original object and partitioning MUST be done on the byte[].
>>>> This
>>>> > is
>>>> > > fine for hash partitioning. However for various types of semantic
>>>> > > partitioning (range partitioning, or whatever) you would want access
>>>> to
>>>> > the
>>>> > > original object. In the current approach a producer who wishes to send
>>>> > > byte[] they have serialized in their own code can configure the
>>>> > > BytesSerialization we supply which is just a "no op" serialization.
>>>> > > 2. We should obsess over naming and make sure each of the class names
>>>> are
>>>> > > good.
>>>> > > 3. Jun has already pointed out that we need to include the topic and
>>>> > > partition in the response, which is absolutely right. I haven't done
>>>> that
>>>> > > yet but that definitely needs to be there.
>>>> > > 4. Currently RecordSend.await will throw an exception if the request
>>>> > > failed. The intention here is that producer.send(message).await()
>>>> exactly
>>>> > > simulates a synchronous call. Guozhang has noted that this is a little
>>>> > > annoying since the user must then catch exceptions. However if we
>>>> remove
>>>> > > this then if the user doesn't check for errors they won't know one has
>>>> > > occurred, which I predict will be a common mistake.
>>>> > > 5. Perhaps there is more we could do to make the async callbacks and
>>>> > future
>>>> > > we give back intuitive and easy to program against?
>>>> > >
>>>> > > Some background info on implementation:
>>>> > >
>>>> > > At a high level the primary difference in this producer is that it
>>>> > removes
>>>> > > the distinction between the "sync" and "async" producer. Effectively
>>>> all
>>>> > > requests are sent asynchronously but always return a future response
>>>> > object
>>>> > > that gives the offset as well as any error that may have occurred when
>>>> > the
>>>> > > request is complete. The batching that is done in the async producer
>>>> only
>>>> > > today is done whenever possible now. This means that the sync
>>>> producer,
>>>> > > under load, can get performance as good as the async producer
>>>> > (preliminary
>>>> > > results show the producer getting 1m messages/sec). This works
>>>> similar to
>>>> > > group commit in databases but with respect to the actual network
>>>> > > transmission--any messages that arrive while a send is in progress are
>>>> > > batched together. It is also possible to encourage batching even under
>>>> > low
>>>> > > load to save server resources by introducing a delay on the send to
>>>> allow
>>>> > > more messages to accumulate; this is done using the linger.ms config
>>>> > (this
>>>> > > is similar to Nagle's algorithm in TCP).
>>>> > >
>>>> > > This producer does all network communication asynchronously and in
>>>> > parallel
>>>> > > to all servers so the performance penalty for acks=-1 and waiting on
>>>> > > replication should be much reduced. I haven't done much benchmarking
>>>> on
>>>> > > this yet, though.
>>>> > >
>>>> > > The high level design is described a little here, though this is now a
>>>> > > little out of date:
>>>> > > https://cwiki.apache.org/confluence/display/KAFKA/Client+Rewrite
>>>> > >
>>>> > > -Jay
>>>> > >
>>>> >
>>>>
>>>
>>>

Re: New Producer Public API

Posted by Joel Koshy <jj...@gmail.com>.
Does it preclude those various implementations? i.e., it could become
a producer config:
default.partitioner.strategy="minimize-connections"/"roundrobin" - and
so on; and implement those partitioners internally in the producer.
Not as clear as a .class config, but it accomplishes the same effect
no?

On Thu, Jan 30, 2014 at 4:14 PM, Jay Kreps <ja...@gmail.com> wrote:
> One downside to the 1A proposal is that without a Partitioner interface we
> can't really package up and provide common partitioner implementations.
> Example of these would be
> 1. HashPartitioner - The default hash partitioning
> 2. RoundRobinPartitioner - Just round-robins over partitions
> 3. ConnectionMinimizingPartitioner - Choose partitions to minimize the
> number of nodes you need to connect maintain TCP connections to.
> 4. RangePartitioner - User provides break points that align partitions to
> key ranges
> 5. LocalityPartitioner - Prefer nodes on the same rack. This would be nice
> for stream-processing use cases that read from one topic and write to
> another. We would have to include rack information in our metadata.
>
> Having this kind of functionality included is actually kind of nice.
>
> -Jay
>
>
> On Fri, Jan 24, 2014 at 5:17 PM, Jay Kreps <ja...@gmail.com> wrote:
>
>> Clark and all,
>>
>> I thought a little bit about the serialization question. Here are the
>> options I see and the pros and cons I can think of. I'd love to hear
>> people's preferences if you have a strong one.
>>
>> One important consideration is that however the producer works will also
>> need to be how the new consumer works (which we hope to write next). That
>> is if you put objects in, you should get objects out. So we need to think
>> through both sides.
>>
>> Options:
>>
>> Option 0: What is in the existing scala code and the java code I
>> posted--Serializer and Partitioner plugin provided by the user via config.
>> Partitioner has a sane default, but Serializer needs to be specified in
>> config.
>>
>> Pros: How it works today in the scala code.
>> Cons: You have to adapt your serialization library of choice to our
>> interfaces. The reflective class loading means typo in the serializer name
>> give odd errors. Likewise there is little type safety--the ProducerRecord
>> takes Object and any type errors between the object provided and the
>> serializer give occurs at runtime.
>>
>> Option 1: No plugins
>>
>> This would mean byte[] key, byte[] value, and partitioning done by client
>> by passing in a partition *number* directly.
>>
>> The problem with this is that it is tricky to compute the partition
>> correctly and probably most people won't. We could add a getCluster()
>> method to return the Cluster instance you should use for partitioning. But
>> I suspect people would be lazy and not use that and instead hard-code
>> partitions which would break if partitions were added or they hard coded it
>> wrong. In my experience 3 partitioning strategies cover like 99% of cases
>> so not having a default implementation for this makes the common case
>> harder. Left to their own devices people will use bad hash functions and
>> get weird results.
>>
>> Option 1A: Alternatively we could partition by the key using the existing
>> default partitioning strategy which only uses the byte[] anyway but instead
>> of having a partitionKey we could have a numerical partition override and
>> add the getCluster() method to get the cluster metadata. That would make
>> custom partitioning possible but handle the common case simply.
>>
>> Option 2: Partitioner plugin remains, serializers go.
>>
>> The problem here is that the partitioner might lose access to the
>> deserialized key which would occasionally be useful for semantic
>> partitioning schemes. The Partitioner could deserialize the key but that
>> would be inefficient and weird.
>>
>> This problem could be fixed by having key and value be byte[] but
>> retaining partitionKey as an Object and passing it to the partitioner as
>> is. Then if you have a partitioner which requires the deserialized key you
>> would need to use this partition key. One weird side effect is that if you
>> want to have a custom partition key BUT want to partition by the bytes of
>> that key rather than the object value you must write a customer partitioner
>> and serialize it yourself.
>>
>> Of these I think I prefer 1A but could be convinced of 0 since that is how
>> it works now.
>>
>> Thoughts?
>>
>> -Jay
>>
>>
>> On Fri, Jan 24, 2014 at 3:30 PM, Clark Breyman <cl...@breyman.com> wrote:
>>
>>> Jay - Thanks for the call for comments. Here's some initial input:
>>>
>>> - Make message serialization a client responsibility (making all messages
>>> byte[]). Reflection-based loading makes it harder to use generic codecs
>>> (e.g.  Envelope<PREFIX, DATA, SUFFIX>) or build up codec programmatically.
>>> Non-default partitioning should require an explicit partition key.
>>>
>>> - I really like the fact that it will be native Java. Please consider
>>> using
>>> native maven and not sbt, gradle, ivy, etc as they don't reliably play
>>> nice
>>> in the maven ecosystem. A jar without a well-formed pom doesn't feel like
>>> a
>>> real artifact. The pom's generated by sbt et al. are not well formed.
>>> Using
>>> maven will make builds and IDE integration much smoother.
>>>
>>> - Look at Nick Telford's dropwizard-extras package in which he defines
>>> some
>>> Jackson-compatible POJO's for loading configuration. Seems like your
>>> client
>>> migration is similar. The config objects should have constructors or
>>> factories that accept Map<String, String> and Properties for ease of
>>> migration.
>>>
>>> - Would you consider using the org.apache.kafka package for the new API
>>> (quibble)
>>>
>>> - Why create your own futures rather than use
>>> java.util.concurrent.Future<Long> or similar? Standard futures will play
>>> nice with other reactive libs and things like J8's ComposableFuture.
>>>
>>> Thanks again,
>>> C
>>>
>>>
>>>
>>> On Fri, Jan 24, 2014 at 2:46 PM, Roger Hoover <roger.hoover@gmail.com
>>> >wrote:
>>>
>>> > A couple comments:
>>> >
>>> > 1) Why does the config use a broker list instead of discovering the
>>> brokers
>>> > in ZooKeeper?  It doesn't match the HighLevelConsumer API.
>>> >
>>> > 2) It looks like broker connections are created on demand.  I'm
>>> wondering
>>> > if sometimes you might want to flush out config or network connectivity
>>> > issues before pushing the first message through.
>>> >
>>> > Should there also be a KafkaProducer.connect() or .open() method or
>>> > connectAll()?  I guess it would try to connect to all brokers in the
>>> > BROKER_LIST_CONFIG
>>> >
>>> > HTH,
>>> >
>>> > Roger
>>> >
>>> >
>>> > On Fri, Jan 24, 2014 at 11:54 AM, Jay Kreps <ja...@gmail.com>
>>> wrote:
>>> >
>>> > > As mentioned in a previous email we are working on a
>>> re-implementation of
>>> > > the producer. I would like to use this email thread to discuss the
>>> > details
>>> > > of the public API and the configuration. I would love for us to be
>>> > > incredibly picky about this public api now so it is as good as
>>> possible
>>> > and
>>> > > we don't need to break it in the future.
>>> > >
>>> > > The best way to get a feel for the API is actually to take a look at
>>> the
>>> > > javadoc, my hope is to get the api docs good enough so that it is
>>> > > self-explanatory:
>>> > >
>>> > >
>>> >
>>> http://empathybox.com/kafka-javadoc/index.html?kafka/clients/producer/KafkaProducer.html
>>> > >
>>> > > Please take a look at this API and give me any thoughts you may have!
>>> > >
>>> > > It may also be reasonable to take a look at the configs:
>>> > >
>>> > >
>>> >
>>> http://empathybox.com/kafka-javadoc/kafka/clients/producer/ProducerConfig.html
>>> > >
>>> > > The actual code is posted here:
>>> > > https://issues.apache.org/jira/browse/KAFKA-1227
>>> > >
>>> > > A few questions or comments to kick things off:
>>> > > 1. We need to make a decision on whether serialization of the user's
>>> key
>>> > > and value should be done by the user (with our api just taking
>>> byte[]) or
>>> > > if we should take an object and allow the user to configure a
>>> Serializer
>>> > > class which we instantiate via reflection. We take the later approach
>>> in
>>> > > the current producer, and I have carried this through to this
>>> prototype.
>>> > > The tradeoff I see is this: taking byte[] is actually simpler, the
>>> user
>>> > can
>>> > > directly do whatever serialization they like. The complication is
>>> > actually
>>> > > partitioning. Currently partitioning is done by a similar plug-in api
>>> > > (Partitioner) which the user can implement and configure to override
>>> how
>>> > > partitions are assigned. If we take byte[] as input then we have no
>>> > access
>>> > > to the original object and partitioning MUST be done on the byte[].
>>> This
>>> > is
>>> > > fine for hash partitioning. However for various types of semantic
>>> > > partitioning (range partitioning, or whatever) you would want access
>>> to
>>> > the
>>> > > original object. In the current approach a producer who wishes to send
>>> > > byte[] they have serialized in their own code can configure the
>>> > > BytesSerialization we supply which is just a "no op" serialization.
>>> > > 2. We should obsess over naming and make sure each of the class names
>>> are
>>> > > good.
>>> > > 3. Jun has already pointed out that we need to include the topic and
>>> > > partition in the response, which is absolutely right. I haven't done
>>> that
>>> > > yet but that definitely needs to be there.
>>> > > 4. Currently RecordSend.await will throw an exception if the request
>>> > > failed. The intention here is that producer.send(message).await()
>>> exactly
>>> > > simulates a synchronous call. Guozhang has noted that this is a little
>>> > > annoying since the user must then catch exceptions. However if we
>>> remove
>>> > > this then if the user doesn't check for errors they won't know one has
>>> > > occurred, which I predict will be a common mistake.
>>> > > 5. Perhaps there is more we could do to make the async callbacks and
>>> > future
>>> > > we give back intuitive and easy to program against?
>>> > >
>>> > > Some background info on implementation:
>>> > >
>>> > > At a high level the primary difference in this producer is that it
>>> > removes
>>> > > the distinction between the "sync" and "async" producer. Effectively
>>> all
>>> > > requests are sent asynchronously but always return a future response
>>> > object
>>> > > that gives the offset as well as any error that may have occurred when
>>> > the
>>> > > request is complete. The batching that is done in the async producer
>>> only
>>> > > today is done whenever possible now. This means that the sync
>>> producer,
>>> > > under load, can get performance as good as the async producer
>>> > (preliminary
>>> > > results show the producer getting 1m messages/sec). This works
>>> similar to
>>> > > group commit in databases but with respect to the actual network
>>> > > transmission--any messages that arrive while a send is in progress are
>>> > > batched together. It is also possible to encourage batching even under
>>> > low
>>> > > load to save server resources by introducing a delay on the send to
>>> allow
>>> > > more messages to accumulate; this is done using the linger.ms config
>>> > (this
>>> > > is similar to Nagle's algorithm in TCP).
>>> > >
>>> > > This producer does all network communication asynchronously and in
>>> > parallel
>>> > > to all servers so the performance penalty for acks=-1 and waiting on
>>> > > replication should be much reduced. I haven't done much benchmarking
>>> on
>>> > > this yet, though.
>>> > >
>>> > > The high level design is described a little here, though this is now a
>>> > > little out of date:
>>> > > https://cwiki.apache.org/confluence/display/KAFKA/Client+Rewrite
>>> > >
>>> > > -Jay
>>> > >
>>> >
>>>
>>
>>

Re: New Producer Public API

Posted by Jun Rao <ju...@gmail.com>.
For RangePartitioner, it seems that we will need the key object.
Range-partitioning on the serialized key bytes is probably confusing.

Thanks,

Jun


On Thu, Jan 30, 2014 at 4:14 PM, Jay Kreps <ja...@gmail.com> wrote:

> One downside to the 1A proposal is that without a Partitioner interface we
> can't really package up and provide common partitioner implementations.
> Example of these would be
> 1. HashPartitioner - The default hash partitioning
> 2. RoundRobinPartitioner - Just round-robins over partitions
> 3. ConnectionMinimizingPartitioner - Choose partitions to minimize the
> number of nodes you need to connect maintain TCP connections to.
> 4. RangePartitioner - User provides break points that align partitions to
> key ranges
> 5. LocalityPartitioner - Prefer nodes on the same rack. This would be nice
> for stream-processing use cases that read from one topic and write to
> another. We would have to include rack information in our metadata.
>
> Having this kind of functionality included is actually kind of nice.
>
> -Jay
>
>
> On Fri, Jan 24, 2014 at 5:17 PM, Jay Kreps <ja...@gmail.com> wrote:
>
> > Clark and all,
> >
> > I thought a little bit about the serialization question. Here are the
> > options I see and the pros and cons I can think of. I'd love to hear
> > people's preferences if you have a strong one.
> >
> > One important consideration is that however the producer works will also
> > need to be how the new consumer works (which we hope to write next). That
> > is if you put objects in, you should get objects out. So we need to think
> > through both sides.
> >
> > Options:
> >
> > Option 0: What is in the existing scala code and the java code I
> > posted--Serializer and Partitioner plugin provided by the user via
> config.
> > Partitioner has a sane default, but Serializer needs to be specified in
> > config.
> >
> > Pros: How it works today in the scala code.
> > Cons: You have to adapt your serialization library of choice to our
> > interfaces. The reflective class loading means typo in the serializer
> name
> > give odd errors. Likewise there is little type safety--the ProducerRecord
> > takes Object and any type errors between the object provided and the
> > serializer give occurs at runtime.
> >
> > Option 1: No plugins
> >
> > This would mean byte[] key, byte[] value, and partitioning done by client
> > by passing in a partition *number* directly.
> >
> > The problem with this is that it is tricky to compute the partition
> > correctly and probably most people won't. We could add a getCluster()
> > method to return the Cluster instance you should use for partitioning.
> But
> > I suspect people would be lazy and not use that and instead hard-code
> > partitions which would break if partitions were added or they hard coded
> it
> > wrong. In my experience 3 partitioning strategies cover like 99% of cases
> > so not having a default implementation for this makes the common case
> > harder. Left to their own devices people will use bad hash functions and
> > get weird results.
> >
> > Option 1A: Alternatively we could partition by the key using the existing
> > default partitioning strategy which only uses the byte[] anyway but
> instead
> > of having a partitionKey we could have a numerical partition override and
> > add the getCluster() method to get the cluster metadata. That would make
> > custom partitioning possible but handle the common case simply.
> >
> > Option 2: Partitioner plugin remains, serializers go.
> >
> > The problem here is that the partitioner might lose access to the
> > deserialized key which would occasionally be useful for semantic
> > partitioning schemes. The Partitioner could deserialize the key but that
> > would be inefficient and weird.
> >
> > This problem could be fixed by having key and value be byte[] but
> > retaining partitionKey as an Object and passing it to the partitioner as
> > is. Then if you have a partitioner which requires the deserialized key
> you
> > would need to use this partition key. One weird side effect is that if
> you
> > want to have a custom partition key BUT want to partition by the bytes of
> > that key rather than the object value you must write a customer
> partitioner
> > and serialize it yourself.
> >
> > Of these I think I prefer 1A but could be convinced of 0 since that is
> how
> > it works now.
> >
> > Thoughts?
> >
> > -Jay
> >
> >
> > On Fri, Jan 24, 2014 at 3:30 PM, Clark Breyman <cl...@breyman.com>
> wrote:
> >
> >> Jay - Thanks for the call for comments. Here's some initial input:
> >>
> >> - Make message serialization a client responsibility (making all
> messages
> >> byte[]). Reflection-based loading makes it harder to use generic codecs
> >> (e.g.  Envelope<PREFIX, DATA, SUFFIX>) or build up codec
> programmatically.
> >> Non-default partitioning should require an explicit partition key.
> >>
> >> - I really like the fact that it will be native Java. Please consider
> >> using
> >> native maven and not sbt, gradle, ivy, etc as they don't reliably play
> >> nice
> >> in the maven ecosystem. A jar without a well-formed pom doesn't feel
> like
> >> a
> >> real artifact. The pom's generated by sbt et al. are not well formed.
> >> Using
> >> maven will make builds and IDE integration much smoother.
> >>
> >> - Look at Nick Telford's dropwizard-extras package in which he defines
> >> some
> >> Jackson-compatible POJO's for loading configuration. Seems like your
> >> client
> >> migration is similar. The config objects should have constructors or
> >> factories that accept Map<String, String> and Properties for ease of
> >> migration.
> >>
> >> - Would you consider using the org.apache.kafka package for the new API
> >> (quibble)
> >>
> >> - Why create your own futures rather than use
> >> java.util.concurrent.Future<Long> or similar? Standard futures will play
> >> nice with other reactive libs and things like J8's ComposableFuture.
> >>
> >> Thanks again,
> >> C
> >>
> >>
> >>
> >> On Fri, Jan 24, 2014 at 2:46 PM, Roger Hoover <roger.hoover@gmail.com
> >> >wrote:
> >>
> >> > A couple comments:
> >> >
> >> > 1) Why does the config use a broker list instead of discovering the
> >> brokers
> >> > in ZooKeeper?  It doesn't match the HighLevelConsumer API.
> >> >
> >> > 2) It looks like broker connections are created on demand.  I'm
> >> wondering
> >> > if sometimes you might want to flush out config or network
> connectivity
> >> > issues before pushing the first message through.
> >> >
> >> > Should there also be a KafkaProducer.connect() or .open() method or
> >> > connectAll()?  I guess it would try to connect to all brokers in the
> >> > BROKER_LIST_CONFIG
> >> >
> >> > HTH,
> >> >
> >> > Roger
> >> >
> >> >
> >> > On Fri, Jan 24, 2014 at 11:54 AM, Jay Kreps <ja...@gmail.com>
> >> wrote:
> >> >
> >> > > As mentioned in a previous email we are working on a
> >> re-implementation of
> >> > > the producer. I would like to use this email thread to discuss the
> >> > details
> >> > > of the public API and the configuration. I would love for us to be
> >> > > incredibly picky about this public api now so it is as good as
> >> possible
> >> > and
> >> > > we don't need to break it in the future.
> >> > >
> >> > > The best way to get a feel for the API is actually to take a look at
> >> the
> >> > > javadoc, my hope is to get the api docs good enough so that it is
> >> > > self-explanatory:
> >> > >
> >> > >
> >> >
> >>
> http://empathybox.com/kafka-javadoc/index.html?kafka/clients/producer/KafkaProducer.html
> >> > >
> >> > > Please take a look at this API and give me any thoughts you may
> have!
> >> > >
> >> > > It may also be reasonable to take a look at the configs:
> >> > >
> >> > >
> >> >
> >>
> http://empathybox.com/kafka-javadoc/kafka/clients/producer/ProducerConfig.html
> >> > >
> >> > > The actual code is posted here:
> >> > > https://issues.apache.org/jira/browse/KAFKA-1227
> >> > >
> >> > > A few questions or comments to kick things off:
> >> > > 1. We need to make a decision on whether serialization of the user's
> >> key
> >> > > and value should be done by the user (with our api just taking
> >> byte[]) or
> >> > > if we should take an object and allow the user to configure a
> >> Serializer
> >> > > class which we instantiate via reflection. We take the later
> approach
> >> in
> >> > > the current producer, and I have carried this through to this
> >> prototype.
> >> > > The tradeoff I see is this: taking byte[] is actually simpler, the
> >> user
> >> > can
> >> > > directly do whatever serialization they like. The complication is
> >> > actually
> >> > > partitioning. Currently partitioning is done by a similar plug-in
> api
> >> > > (Partitioner) which the user can implement and configure to override
> >> how
> >> > > partitions are assigned. If we take byte[] as input then we have no
> >> > access
> >> > > to the original object and partitioning MUST be done on the byte[].
> >> This
> >> > is
> >> > > fine for hash partitioning. However for various types of semantic
> >> > > partitioning (range partitioning, or whatever) you would want access
> >> to
> >> > the
> >> > > original object. In the current approach a producer who wishes to
> send
> >> > > byte[] they have serialized in their own code can configure the
> >> > > BytesSerialization we supply which is just a "no op" serialization.
> >> > > 2. We should obsess over naming and make sure each of the class
> names
> >> are
> >> > > good.
> >> > > 3. Jun has already pointed out that we need to include the topic and
> >> > > partition in the response, which is absolutely right. I haven't done
> >> that
> >> > > yet but that definitely needs to be there.
> >> > > 4. Currently RecordSend.await will throw an exception if the request
> >> > > failed. The intention here is that producer.send(message).await()
> >> exactly
> >> > > simulates a synchronous call. Guozhang has noted that this is a
> little
> >> > > annoying since the user must then catch exceptions. However if we
> >> remove
> >> > > this then if the user doesn't check for errors they won't know one
> has
> >> > > occurred, which I predict will be a common mistake.
> >> > > 5. Perhaps there is more we could do to make the async callbacks and
> >> > future
> >> > > we give back intuitive and easy to program against?
> >> > >
> >> > > Some background info on implementation:
> >> > >
> >> > > At a high level the primary difference in this producer is that it
> >> > removes
> >> > > the distinction between the "sync" and "async" producer. Effectively
> >> all
> >> > > requests are sent asynchronously but always return a future response
> >> > object
> >> > > that gives the offset as well as any error that may have occurred
> when
> >> > the
> >> > > request is complete. The batching that is done in the async producer
> >> only
> >> > > today is done whenever possible now. This means that the sync
> >> producer,
> >> > > under load, can get performance as good as the async producer
> >> > (preliminary
> >> > > results show the producer getting 1m messages/sec). This works
> >> similar to
> >> > > group commit in databases but with respect to the actual network
> >> > > transmission--any messages that arrive while a send is in progress
> are
> >> > > batched together. It is also possible to encourage batching even
> under
> >> > low
> >> > > load to save server resources by introducing a delay on the send to
> >> allow
> >> > > more messages to accumulate; this is done using the linger.msconfig
> >> > (this
> >> > > is similar to Nagle's algorithm in TCP).
> >> > >
> >> > > This producer does all network communication asynchronously and in
> >> > parallel
> >> > > to all servers so the performance penalty for acks=-1 and waiting on
> >> > > replication should be much reduced. I haven't done much benchmarking
> >> on
> >> > > this yet, though.
> >> > >
> >> > > The high level design is described a little here, though this is
> now a
> >> > > little out of date:
> >> > > https://cwiki.apache.org/confluence/display/KAFKA/Client+Rewrite
> >> > >
> >> > > -Jay
> >> > >
> >> >
> >>
> >
> >
>

Re: New Producer Public API

Posted by Jay Kreps <ja...@gmail.com>.
This is pretty hard to do with the architecture we've gone with as the
stored events are not objects, but tightly packed serialized bytes. This
approach is much better from a performance and memory management point of
view, though, so I'd be very hesitant to change it. So it is pretty hard to
provide a usable api. I think likely this is something that would be better
implemented in the application (e.g. use a blocking queue and batch into a
single message after the timeout).

-Jay


On Thu, Feb 6, 2014 at 1:16 PM, S Ahmed <sa...@gmail.com> wrote:

> How about the following use case:
>
> Just before the producer actually sends the payload to kakfa, could an
> event be exposed that would allow one to loop through the messages and
> potentially delete some of them?
>
> Example:
>
> Say you have 100 messages, but before you send these messages to kakfa, you
> can easily aggregate many of these messages to reduce the message count.
>  If there are messages that store counts, you could aggregate these into a
> single message and then send to kafka.
>
> Thoughts?
>
>
>
> On Wed, Feb 5, 2014 at 2:03 PM, Jay Kreps <ja...@gmail.com> wrote:
>
> > It might. I considered this but ended up going this way. Now that we have
> > changed partitionKey=>partition it almost works. The difference is the
> > consumer gets an offset too which the producer doesn't have.
> >
> > One thing I think this points to is the value of getting the consumer
> java
> > api worked out even in the absence of an implementation just so we can
> > write some fake code that uses both and kind of see how it feels.
> >
> > -Jay
> >
> >
> > On Wed, Feb 5, 2014 at 10:23 AM, Neha Narkhede <neha.narkhede@gmail.com
> > >wrote:
> >
> > > Currently, the user will send ProducerRecords using the new producer.
> The
> > > expectation will be that you get the same thing as output from the
> > > consumer. Since ProduceRecord is a holder for topic, partition, key and
> > > value, does it make sense to rename it to just Record? So, the
> > send/receive
> > > APIs would look like the following -
> > >
> > > producer.send(Record record);
> > > List<Record> poll();
> > >
> > > Thoughts?
> > >
> > >
> > > On Sun, Feb 2, 2014 at 4:12 PM, Guozhang Wang <wa...@gmail.com>
> > wrote:
> > >
> > > > I think the most common motivate of having a customized partitioner
> is
> > to
> > > > make sure some messages always go to the same partition, but people
> may
> > > > seldom want to know about which partition exactly they go to. If that
> > is
> > > > true, why not just assign the same byte array as partition key with
> the
> > > > default hash based partitioning in option 1.A? But again, that is
> based
> > > on
> > > > my presumption that very few users would want to really specify the
> > > > partition id.
> > > >
> > > >
> > > >
> > > > On Fri, Jan 31, 2014 at 2:44 PM, Jay Kreps <ja...@gmail.com>
> > wrote:
> > > >
> > > > > Hey Tom,
> > > > >
> > > > > Agreed, there is definitely nothing that prevents our including
> > > > partitioner
> > > > > implementations, but it does get a little less seamless.
> > > > >
> > > > > -Jay
> > > > >
> > > > >
> > > > > On Fri, Jan 31, 2014 at 2:35 PM, Tom Brown <to...@gmail.com>
> > > wrote:
> > > > >
> > > > > > Regarding partitioning APIs, I don't think there is not a common
> > > subset
> > > > > of
> > > > > > information that is required for all strategies. Instead of
> > modifying
> > > > the
> > > > > > core API to easily support all of the various partitioning
> > > strategies,
> > > > > > offer the most common ones as libraries they can build into their
> > own
> > > > > data
> > > > > > pipeline, just like serialization. The core API would simply
> > accept a
> > > > > > partition index. You could include one default strategy (random)
> > that
> > > > > only
> > > > > > applies if they set "-1" for the partition index.
> > > > > >
> > > > > > That way, each partitioning strategy could have its own API that
> > > makes
> > > > > > sense for it. For example, a round-robin partitioner only needs
> one
> > > > > method:
> > > > > > "nextPartition()", while a hash-based one needs
> > > > > "getPartitionFor(byte[])".
> > > > > >
> > > > > > For those who actually need a pluggable strategy, a superset of
> the
> > > API
> > > > > > could be codified into an interface (perhaps the existing
> > partitioner
> > > > > > interface), but it would still have to be used from outside of
> the
> > > core
> > > > > > API.
> > > > > >
> > > > > > This design would make the core API less confusing (when do I
> use a
> > > > > > partiton key instead of a partition index, does the key overwrite
> > the
> > > > > > index, can the key be null, etc...?) while still providing the
> > > > > flexibility
> > > > > > you want.
> > > > > >
> > > > > > --Tom
> > > > > >
> > > > > > On Fri, Jan 31, 2014 at 12:07 PM, Jay Kreps <jay.kreps@gmail.com
> >
> > > > wrote:
> > > > > >
> > > > > > > Oliver,
> > > > > > >
> > > > > > > Yeah that was my original plan--allow the registration of
> > multiple
> > > > > > > callbacks on the future. But there is some additional
> > > implementation
> > > > > > > complexity because then you need more synchronization variables
> > to
> > > > > ensure
> > > > > > > the callback gets executed even if the request has completed at
> > the
> > > > > time
> > > > > > > the callback is registered. This also makes it unpredictable
> the
> > > > order
> > > > > of
> > > > > > > callback execution--I want to be able to guarantee that for a
> > > > > particular
> > > > > > > partition callbacks for lower offset messages happen before
> > > callbacks
> > > > > for
> > > > > > > higher offset messages so that if you set a highwater mark or
> > > > something
> > > > > > it
> > > > > > > is easy to reason about. This has the added benefit that
> > callbacks
> > > > > > execute
> > > > > > > in the I/O thread ALWAYS instead of it being non-deterministic
> > > which
> > > > > is a
> > > > > > > little confusing.
> > > > > > >
> > > > > > > I thought a single callback is sufficient since you can always
> > > > include
> > > > > > > multiple actions in that callback, and I think that case is
> rare
> > > > > anyway.
> > > > > > >
> > > > > > > I did think about the possibility of adding a thread pool for
> > > > handling
> > > > > > the
> > > > > > > callbacks. But there are a lot of possible configurations for
> > such
> > > a
> > > > > > thread
> > > > > > > pool and a simplistic approach would no longer guarantee
> in-order
> > > > > > > processing of callbacks (you would need to hash execution over
> > > > threads
> > > > > by
> > > > > > > partition id). I think by just exposing the simple method that
> > > > executes
> > > > > > in
> > > > > > > the I/O thread you can easily implement the pooled execution
> > using
> > > > the
> > > > > > > therad pooling mechanism of your choice by just having the
> > callback
> > > > use
> > > > > > an
> > > > > > > executor to run the action (i.e. make an AsyncCallback that
> > takes a
> > > > > > > threadpool and a Runnable or something like that). This gives
> the
> > > > user
> > > > > > full
> > > > > > > control over the executor (there are lots of details around
> > thread
> > > > > re-use
> > > > > > > in executors, thread factories, etc and trying to expose
> configs
> > > for
> > > > > > every
> > > > > > > variation will be a pain). This also makes it totally
> transparent
> > > how
> > > > > it
> > > > > > > works; that is if we did expose all kinds of thread pool
> configs
> > > you
> > > > > > would
> > > > > > > still probably end up reading our code to figure out exactly
> what
> > > > they
> > > > > > all
> > > > > > > did.
> > > > > > >
> > > > > > > -Jay
> > > > > > >
> > > > > > >
> > > > > > > On Fri, Jan 31, 2014 at 9:39 AM, Oliver Dain <
> > > > odain@3cinteractive.com
> > > > > > > >wrote:
> > > > > > >
> > > > > > > > Hmmm.. I should read the docs more carefully before I open my
> > big
> > > > > > mouth:
> > > > > > > I
> > > > > > > > just noticed the KafkaProducer#send overload that takes a
> > > callback.
> > > > > > That
> > > > > > > > definitely helps address my concern though I think the API
> > would
> > > be
> > > > > > > > cleaner if there was only one variant that returned a future
> > and
> > > > you
> > > > > > > could
> > > > > > > > register the callback with the future. This is not nearly as
> > > > > important
> > > > > > as
> > > > > > > > I'd thought given the ability to register a callback - just a
> > > > > > preference.
> > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > > On 1/31/14, 9:33 AM, "Oliver Dain" <od...@3cinteractive.com>
> > > > wrote:
> > > > > > > >
> > > > > > > > >Hey all,
> > > > > > > > >
> > > > > > > > >I¹m excited about having a new Producer API, and I really
> like
> > > the
> > > > > > idea
> > > > > > > of
> > > > > > > > >removing the distinction between a synchronous and
> > asynchronous
> > > > > > > producer.
> > > > > > > > >The one comment I have about the current API is that it¹s
> hard
> > > to
> > > > > > write
> > > > > > > > >truly asynchronous code with the type of future returned by
> > the
> > > > send
> > > > > > > > >method. The issue is that send returns a RecordSend and
> > there¹s
> > > no
> > > > > way
> > > > > > > to
> > > > > > > > >register a callback with that object. It is therefore
> > necessary
> > > to
> > > > > > poll
> > > > > > > > >the object periodically to see if the send has completed. So
> > if
> > > > you
> > > > > > > have n
> > > > > > > > >send calls outstanding you have to check n RecordSend
> objects
> > > > which
> > > > > is
> > > > > > > > >slow. In general this tends to lead to people using one
> thread
> > > per
> > > > > > send
> > > > > > > > >call and then calling RecordSend#await which removes much of
> > the
> > > > > > benefit
> > > > > > > > >of an async API.
> > > > > > > > >
> > > > > > > > >I think it¹s much easier to write truly asynchronous code if
> > the
> > > > > > > returned
> > > > > > > > >future allows you to register a callback. That way, instead
> of
> > > > > polling
> > > > > > > you
> > > > > > > > >can simply wait for the callback to be called. A good
> example
> > of
> > > > the
> > > > > > > kind
> > > > > > > > >of thing I¹m thinking is the ListenableFuture class in the
> > Guava
> > > > > > > > >libraries:
> > > > > > > > >
> > > > > > > > >
> > > > > > >
> > > > >
> > >
> https://code.google.com/p/guava-libraries/wiki/ListenableFutureExplained
> > > > > > > > >
> > > > > > > > >
> > > > > > > > >HTH,
> > > > > > > > >Oliver
> > > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > > >
> > > >
> > > > --
> > > > -- Guozhang
> > > >
> > >
> >
>

Re: New Producer Public API

Posted by S Ahmed <sa...@gmail.com>.
How about the following use case:

Just before the producer actually sends the payload to kakfa, could an
event be exposed that would allow one to loop through the messages and
potentially delete some of them?

Example:

Say you have 100 messages, but before you send these messages to kakfa, you
can easily aggregate many of these messages to reduce the message count.
 If there are messages that store counts, you could aggregate these into a
single message and then send to kafka.

Thoughts?



On Wed, Feb 5, 2014 at 2:03 PM, Jay Kreps <ja...@gmail.com> wrote:

> It might. I considered this but ended up going this way. Now that we have
> changed partitionKey=>partition it almost works. The difference is the
> consumer gets an offset too which the producer doesn't have.
>
> One thing I think this points to is the value of getting the consumer java
> api worked out even in the absence of an implementation just so we can
> write some fake code that uses both and kind of see how it feels.
>
> -Jay
>
>
> On Wed, Feb 5, 2014 at 10:23 AM, Neha Narkhede <neha.narkhede@gmail.com
> >wrote:
>
> > Currently, the user will send ProducerRecords using the new producer. The
> > expectation will be that you get the same thing as output from the
> > consumer. Since ProduceRecord is a holder for topic, partition, key and
> > value, does it make sense to rename it to just Record? So, the
> send/receive
> > APIs would look like the following -
> >
> > producer.send(Record record);
> > List<Record> poll();
> >
> > Thoughts?
> >
> >
> > On Sun, Feb 2, 2014 at 4:12 PM, Guozhang Wang <wa...@gmail.com>
> wrote:
> >
> > > I think the most common motivate of having a customized partitioner is
> to
> > > make sure some messages always go to the same partition, but people may
> > > seldom want to know about which partition exactly they go to. If that
> is
> > > true, why not just assign the same byte array as partition key with the
> > > default hash based partitioning in option 1.A? But again, that is based
> > on
> > > my presumption that very few users would want to really specify the
> > > partition id.
> > >
> > >
> > >
> > > On Fri, Jan 31, 2014 at 2:44 PM, Jay Kreps <ja...@gmail.com>
> wrote:
> > >
> > > > Hey Tom,
> > > >
> > > > Agreed, there is definitely nothing that prevents our including
> > > partitioner
> > > > implementations, but it does get a little less seamless.
> > > >
> > > > -Jay
> > > >
> > > >
> > > > On Fri, Jan 31, 2014 at 2:35 PM, Tom Brown <to...@gmail.com>
> > wrote:
> > > >
> > > > > Regarding partitioning APIs, I don't think there is not a common
> > subset
> > > > of
> > > > > information that is required for all strategies. Instead of
> modifying
> > > the
> > > > > core API to easily support all of the various partitioning
> > strategies,
> > > > > offer the most common ones as libraries they can build into their
> own
> > > > data
> > > > > pipeline, just like serialization. The core API would simply
> accept a
> > > > > partition index. You could include one default strategy (random)
> that
> > > > only
> > > > > applies if they set "-1" for the partition index.
> > > > >
> > > > > That way, each partitioning strategy could have its own API that
> > makes
> > > > > sense for it. For example, a round-robin partitioner only needs one
> > > > method:
> > > > > "nextPartition()", while a hash-based one needs
> > > > "getPartitionFor(byte[])".
> > > > >
> > > > > For those who actually need a pluggable strategy, a superset of the
> > API
> > > > > could be codified into an interface (perhaps the existing
> partitioner
> > > > > interface), but it would still have to be used from outside of the
> > core
> > > > > API.
> > > > >
> > > > > This design would make the core API less confusing (when do I use a
> > > > > partiton key instead of a partition index, does the key overwrite
> the
> > > > > index, can the key be null, etc...?) while still providing the
> > > > flexibility
> > > > > you want.
> > > > >
> > > > > --Tom
> > > > >
> > > > > On Fri, Jan 31, 2014 at 12:07 PM, Jay Kreps <ja...@gmail.com>
> > > wrote:
> > > > >
> > > > > > Oliver,
> > > > > >
> > > > > > Yeah that was my original plan--allow the registration of
> multiple
> > > > > > callbacks on the future. But there is some additional
> > implementation
> > > > > > complexity because then you need more synchronization variables
> to
> > > > ensure
> > > > > > the callback gets executed even if the request has completed at
> the
> > > > time
> > > > > > the callback is registered. This also makes it unpredictable the
> > > order
> > > > of
> > > > > > callback execution--I want to be able to guarantee that for a
> > > > particular
> > > > > > partition callbacks for lower offset messages happen before
> > callbacks
> > > > for
> > > > > > higher offset messages so that if you set a highwater mark or
> > > something
> > > > > it
> > > > > > is easy to reason about. This has the added benefit that
> callbacks
> > > > > execute
> > > > > > in the I/O thread ALWAYS instead of it being non-deterministic
> > which
> > > > is a
> > > > > > little confusing.
> > > > > >
> > > > > > I thought a single callback is sufficient since you can always
> > > include
> > > > > > multiple actions in that callback, and I think that case is rare
> > > > anyway.
> > > > > >
> > > > > > I did think about the possibility of adding a thread pool for
> > > handling
> > > > > the
> > > > > > callbacks. But there are a lot of possible configurations for
> such
> > a
> > > > > thread
> > > > > > pool and a simplistic approach would no longer guarantee in-order
> > > > > > processing of callbacks (you would need to hash execution over
> > > threads
> > > > by
> > > > > > partition id). I think by just exposing the simple method that
> > > executes
> > > > > in
> > > > > > the I/O thread you can easily implement the pooled execution
> using
> > > the
> > > > > > therad pooling mechanism of your choice by just having the
> callback
> > > use
> > > > > an
> > > > > > executor to run the action (i.e. make an AsyncCallback that
> takes a
> > > > > > threadpool and a Runnable or something like that). This gives the
> > > user
> > > > > full
> > > > > > control over the executor (there are lots of details around
> thread
> > > > re-use
> > > > > > in executors, thread factories, etc and trying to expose configs
> > for
> > > > > every
> > > > > > variation will be a pain). This also makes it totally transparent
> > how
> > > > it
> > > > > > works; that is if we did expose all kinds of thread pool configs
> > you
> > > > > would
> > > > > > still probably end up reading our code to figure out exactly what
> > > they
> > > > > all
> > > > > > did.
> > > > > >
> > > > > > -Jay
> > > > > >
> > > > > >
> > > > > > On Fri, Jan 31, 2014 at 9:39 AM, Oliver Dain <
> > > odain@3cinteractive.com
> > > > > > >wrote:
> > > > > >
> > > > > > > Hmmm.. I should read the docs more carefully before I open my
> big
> > > > > mouth:
> > > > > > I
> > > > > > > just noticed the KafkaProducer#send overload that takes a
> > callback.
> > > > > That
> > > > > > > definitely helps address my concern though I think the API
> would
> > be
> > > > > > > cleaner if there was only one variant that returned a future
> and
> > > you
> > > > > > could
> > > > > > > register the callback with the future. This is not nearly as
> > > > important
> > > > > as
> > > > > > > I'd thought given the ability to register a callback - just a
> > > > > preference.
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > On 1/31/14, 9:33 AM, "Oliver Dain" <od...@3cinteractive.com>
> > > wrote:
> > > > > > >
> > > > > > > >Hey all,
> > > > > > > >
> > > > > > > >I¹m excited about having a new Producer API, and I really like
> > the
> > > > > idea
> > > > > > of
> > > > > > > >removing the distinction between a synchronous and
> asynchronous
> > > > > > producer.
> > > > > > > >The one comment I have about the current API is that it¹s hard
> > to
> > > > > write
> > > > > > > >truly asynchronous code with the type of future returned by
> the
> > > send
> > > > > > > >method. The issue is that send returns a RecordSend and
> there¹s
> > no
> > > > way
> > > > > > to
> > > > > > > >register a callback with that object. It is therefore
> necessary
> > to
> > > > > poll
> > > > > > > >the object periodically to see if the send has completed. So
> if
> > > you
> > > > > > have n
> > > > > > > >send calls outstanding you have to check n RecordSend objects
> > > which
> > > > is
> > > > > > > >slow. In general this tends to lead to people using one thread
> > per
> > > > > send
> > > > > > > >call and then calling RecordSend#await which removes much of
> the
> > > > > benefit
> > > > > > > >of an async API.
> > > > > > > >
> > > > > > > >I think it¹s much easier to write truly asynchronous code if
> the
> > > > > > returned
> > > > > > > >future allows you to register a callback. That way, instead of
> > > > polling
> > > > > > you
> > > > > > > >can simply wait for the callback to be called. A good example
> of
> > > the
> > > > > > kind
> > > > > > > >of thing I¹m thinking is the ListenableFuture class in the
> Guava
> > > > > > > >libraries:
> > > > > > > >
> > > > > > > >
> > > > > >
> > > >
> > https://code.google.com/p/guava-libraries/wiki/ListenableFutureExplained
> > > > > > > >
> > > > > > > >
> > > > > > > >HTH,
> > > > > > > >Oliver
> > > > > > > >
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> > >
> > >
> > > --
> > > -- Guozhang
> > >
> >
>

Re: New Producer Public API

Posted by Jay Kreps <ja...@gmail.com>.
It might. I considered this but ended up going this way. Now that we have
changed partitionKey=>partition it almost works. The difference is the
consumer gets an offset too which the producer doesn't have.

One thing I think this points to is the value of getting the consumer java
api worked out even in the absence of an implementation just so we can
write some fake code that uses both and kind of see how it feels.

-Jay


On Wed, Feb 5, 2014 at 10:23 AM, Neha Narkhede <ne...@gmail.com>wrote:

> Currently, the user will send ProducerRecords using the new producer. The
> expectation will be that you get the same thing as output from the
> consumer. Since ProduceRecord is a holder for topic, partition, key and
> value, does it make sense to rename it to just Record? So, the send/receive
> APIs would look like the following -
>
> producer.send(Record record);
> List<Record> poll();
>
> Thoughts?
>
>
> On Sun, Feb 2, 2014 at 4:12 PM, Guozhang Wang <wa...@gmail.com> wrote:
>
> > I think the most common motivate of having a customized partitioner is to
> > make sure some messages always go to the same partition, but people may
> > seldom want to know about which partition exactly they go to. If that is
> > true, why not just assign the same byte array as partition key with the
> > default hash based partitioning in option 1.A? But again, that is based
> on
> > my presumption that very few users would want to really specify the
> > partition id.
> >
> >
> >
> > On Fri, Jan 31, 2014 at 2:44 PM, Jay Kreps <ja...@gmail.com> wrote:
> >
> > > Hey Tom,
> > >
> > > Agreed, there is definitely nothing that prevents our including
> > partitioner
> > > implementations, but it does get a little less seamless.
> > >
> > > -Jay
> > >
> > >
> > > On Fri, Jan 31, 2014 at 2:35 PM, Tom Brown <to...@gmail.com>
> wrote:
> > >
> > > > Regarding partitioning APIs, I don't think there is not a common
> subset
> > > of
> > > > information that is required for all strategies. Instead of modifying
> > the
> > > > core API to easily support all of the various partitioning
> strategies,
> > > > offer the most common ones as libraries they can build into their own
> > > data
> > > > pipeline, just like serialization. The core API would simply accept a
> > > > partition index. You could include one default strategy (random) that
> > > only
> > > > applies if they set "-1" for the partition index.
> > > >
> > > > That way, each partitioning strategy could have its own API that
> makes
> > > > sense for it. For example, a round-robin partitioner only needs one
> > > method:
> > > > "nextPartition()", while a hash-based one needs
> > > "getPartitionFor(byte[])".
> > > >
> > > > For those who actually need a pluggable strategy, a superset of the
> API
> > > > could be codified into an interface (perhaps the existing partitioner
> > > > interface), but it would still have to be used from outside of the
> core
> > > > API.
> > > >
> > > > This design would make the core API less confusing (when do I use a
> > > > partiton key instead of a partition index, does the key overwrite the
> > > > index, can the key be null, etc...?) while still providing the
> > > flexibility
> > > > you want.
> > > >
> > > > --Tom
> > > >
> > > > On Fri, Jan 31, 2014 at 12:07 PM, Jay Kreps <ja...@gmail.com>
> > wrote:
> > > >
> > > > > Oliver,
> > > > >
> > > > > Yeah that was my original plan--allow the registration of multiple
> > > > > callbacks on the future. But there is some additional
> implementation
> > > > > complexity because then you need more synchronization variables to
> > > ensure
> > > > > the callback gets executed even if the request has completed at the
> > > time
> > > > > the callback is registered. This also makes it unpredictable the
> > order
> > > of
> > > > > callback execution--I want to be able to guarantee that for a
> > > particular
> > > > > partition callbacks for lower offset messages happen before
> callbacks
> > > for
> > > > > higher offset messages so that if you set a highwater mark or
> > something
> > > > it
> > > > > is easy to reason about. This has the added benefit that callbacks
> > > > execute
> > > > > in the I/O thread ALWAYS instead of it being non-deterministic
> which
> > > is a
> > > > > little confusing.
> > > > >
> > > > > I thought a single callback is sufficient since you can always
> > include
> > > > > multiple actions in that callback, and I think that case is rare
> > > anyway.
> > > > >
> > > > > I did think about the possibility of adding a thread pool for
> > handling
> > > > the
> > > > > callbacks. But there are a lot of possible configurations for such
> a
> > > > thread
> > > > > pool and a simplistic approach would no longer guarantee in-order
> > > > > processing of callbacks (you would need to hash execution over
> > threads
> > > by
> > > > > partition id). I think by just exposing the simple method that
> > executes
> > > > in
> > > > > the I/O thread you can easily implement the pooled execution using
> > the
> > > > > therad pooling mechanism of your choice by just having the callback
> > use
> > > > an
> > > > > executor to run the action (i.e. make an AsyncCallback that takes a
> > > > > threadpool and a Runnable or something like that). This gives the
> > user
> > > > full
> > > > > control over the executor (there are lots of details around thread
> > > re-use
> > > > > in executors, thread factories, etc and trying to expose configs
> for
> > > > every
> > > > > variation will be a pain). This also makes it totally transparent
> how
> > > it
> > > > > works; that is if we did expose all kinds of thread pool configs
> you
> > > > would
> > > > > still probably end up reading our code to figure out exactly what
> > they
> > > > all
> > > > > did.
> > > > >
> > > > > -Jay
> > > > >
> > > > >
> > > > > On Fri, Jan 31, 2014 at 9:39 AM, Oliver Dain <
> > odain@3cinteractive.com
> > > > > >wrote:
> > > > >
> > > > > > Hmmm.. I should read the docs more carefully before I open my big
> > > > mouth:
> > > > > I
> > > > > > just noticed the KafkaProducer#send overload that takes a
> callback.
> > > > That
> > > > > > definitely helps address my concern though I think the API would
> be
> > > > > > cleaner if there was only one variant that returned a future and
> > you
> > > > > could
> > > > > > register the callback with the future. This is not nearly as
> > > important
> > > > as
> > > > > > I'd thought given the ability to register a callback - just a
> > > > preference.
> > > > > >
> > > > > >
> > > > > >
> > > > > >
> > > > > >
> > > > > > On 1/31/14, 9:33 AM, "Oliver Dain" <od...@3cinteractive.com>
> > wrote:
> > > > > >
> > > > > > >Hey all,
> > > > > > >
> > > > > > >I¹m excited about having a new Producer API, and I really like
> the
> > > > idea
> > > > > of
> > > > > > >removing the distinction between a synchronous and asynchronous
> > > > > producer.
> > > > > > >The one comment I have about the current API is that it¹s hard
> to
> > > > write
> > > > > > >truly asynchronous code with the type of future returned by the
> > send
> > > > > > >method. The issue is that send returns a RecordSend and there¹s
> no
> > > way
> > > > > to
> > > > > > >register a callback with that object. It is therefore necessary
> to
> > > > poll
> > > > > > >the object periodically to see if the send has completed. So if
> > you
> > > > > have n
> > > > > > >send calls outstanding you have to check n RecordSend objects
> > which
> > > is
> > > > > > >slow. In general this tends to lead to people using one thread
> per
> > > > send
> > > > > > >call and then calling RecordSend#await which removes much of the
> > > > benefit
> > > > > > >of an async API.
> > > > > > >
> > > > > > >I think it¹s much easier to write truly asynchronous code if the
> > > > > returned
> > > > > > >future allows you to register a callback. That way, instead of
> > > polling
> > > > > you
> > > > > > >can simply wait for the callback to be called. A good example of
> > the
> > > > > kind
> > > > > > >of thing I¹m thinking is the ListenableFuture class in the Guava
> > > > > > >libraries:
> > > > > > >
> > > > > > >
> > > > >
> > >
> https://code.google.com/p/guava-libraries/wiki/ListenableFutureExplained
> > > > > > >
> > > > > > >
> > > > > > >HTH,
> > > > > > >Oliver
> > > > > > >
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> >
> >
> > --
> > -- Guozhang
> >
>

Re: New Producer Public API

Posted by Neha Narkhede <ne...@gmail.com>.
Currently, the user will send ProducerRecords using the new producer. The
expectation will be that you get the same thing as output from the
consumer. Since ProduceRecord is a holder for topic, partition, key and
value, does it make sense to rename it to just Record? So, the send/receive
APIs would look like the following -

producer.send(Record record);
List<Record> poll();

Thoughts?


On Sun, Feb 2, 2014 at 4:12 PM, Guozhang Wang <wa...@gmail.com> wrote:

> I think the most common motivate of having a customized partitioner is to
> make sure some messages always go to the same partition, but people may
> seldom want to know about which partition exactly they go to. If that is
> true, why not just assign the same byte array as partition key with the
> default hash based partitioning in option 1.A? But again, that is based on
> my presumption that very few users would want to really specify the
> partition id.
>
>
>
> On Fri, Jan 31, 2014 at 2:44 PM, Jay Kreps <ja...@gmail.com> wrote:
>
> > Hey Tom,
> >
> > Agreed, there is definitely nothing that prevents our including
> partitioner
> > implementations, but it does get a little less seamless.
> >
> > -Jay
> >
> >
> > On Fri, Jan 31, 2014 at 2:35 PM, Tom Brown <to...@gmail.com> wrote:
> >
> > > Regarding partitioning APIs, I don't think there is not a common subset
> > of
> > > information that is required for all strategies. Instead of modifying
> the
> > > core API to easily support all of the various partitioning strategies,
> > > offer the most common ones as libraries they can build into their own
> > data
> > > pipeline, just like serialization. The core API would simply accept a
> > > partition index. You could include one default strategy (random) that
> > only
> > > applies if they set "-1" for the partition index.
> > >
> > > That way, each partitioning strategy could have its own API that makes
> > > sense for it. For example, a round-robin partitioner only needs one
> > method:
> > > "nextPartition()", while a hash-based one needs
> > "getPartitionFor(byte[])".
> > >
> > > For those who actually need a pluggable strategy, a superset of the API
> > > could be codified into an interface (perhaps the existing partitioner
> > > interface), but it would still have to be used from outside of the core
> > > API.
> > >
> > > This design would make the core API less confusing (when do I use a
> > > partiton key instead of a partition index, does the key overwrite the
> > > index, can the key be null, etc...?) while still providing the
> > flexibility
> > > you want.
> > >
> > > --Tom
> > >
> > > On Fri, Jan 31, 2014 at 12:07 PM, Jay Kreps <ja...@gmail.com>
> wrote:
> > >
> > > > Oliver,
> > > >
> > > > Yeah that was my original plan--allow the registration of multiple
> > > > callbacks on the future. But there is some additional implementation
> > > > complexity because then you need more synchronization variables to
> > ensure
> > > > the callback gets executed even if the request has completed at the
> > time
> > > > the callback is registered. This also makes it unpredictable the
> order
> > of
> > > > callback execution--I want to be able to guarantee that for a
> > particular
> > > > partition callbacks for lower offset messages happen before callbacks
> > for
> > > > higher offset messages so that if you set a highwater mark or
> something
> > > it
> > > > is easy to reason about. This has the added benefit that callbacks
> > > execute
> > > > in the I/O thread ALWAYS instead of it being non-deterministic which
> > is a
> > > > little confusing.
> > > >
> > > > I thought a single callback is sufficient since you can always
> include
> > > > multiple actions in that callback, and I think that case is rare
> > anyway.
> > > >
> > > > I did think about the possibility of adding a thread pool for
> handling
> > > the
> > > > callbacks. But there are a lot of possible configurations for such a
> > > thread
> > > > pool and a simplistic approach would no longer guarantee in-order
> > > > processing of callbacks (you would need to hash execution over
> threads
> > by
> > > > partition id). I think by just exposing the simple method that
> executes
> > > in
> > > > the I/O thread you can easily implement the pooled execution using
> the
> > > > therad pooling mechanism of your choice by just having the callback
> use
> > > an
> > > > executor to run the action (i.e. make an AsyncCallback that takes a
> > > > threadpool and a Runnable or something like that). This gives the
> user
> > > full
> > > > control over the executor (there are lots of details around thread
> > re-use
> > > > in executors, thread factories, etc and trying to expose configs for
> > > every
> > > > variation will be a pain). This also makes it totally transparent how
> > it
> > > > works; that is if we did expose all kinds of thread pool configs you
> > > would
> > > > still probably end up reading our code to figure out exactly what
> they
> > > all
> > > > did.
> > > >
> > > > -Jay
> > > >
> > > >
> > > > On Fri, Jan 31, 2014 at 9:39 AM, Oliver Dain <
> odain@3cinteractive.com
> > > > >wrote:
> > > >
> > > > > Hmmm.. I should read the docs more carefully before I open my big
> > > mouth:
> > > > I
> > > > > just noticed the KafkaProducer#send overload that takes a callback.
> > > That
> > > > > definitely helps address my concern though I think the API would be
> > > > > cleaner if there was only one variant that returned a future and
> you
> > > > could
> > > > > register the callback with the future. This is not nearly as
> > important
> > > as
> > > > > I'd thought given the ability to register a callback - just a
> > > preference.
> > > > >
> > > > >
> > > > >
> > > > >
> > > > >
> > > > > On 1/31/14, 9:33 AM, "Oliver Dain" <od...@3cinteractive.com>
> wrote:
> > > > >
> > > > > >Hey all,
> > > > > >
> > > > > >I¹m excited about having a new Producer API, and I really like the
> > > idea
> > > > of
> > > > > >removing the distinction between a synchronous and asynchronous
> > > > producer.
> > > > > >The one comment I have about the current API is that it¹s hard to
> > > write
> > > > > >truly asynchronous code with the type of future returned by the
> send
> > > > > >method. The issue is that send returns a RecordSend and there¹s no
> > way
> > > > to
> > > > > >register a callback with that object. It is therefore necessary to
> > > poll
> > > > > >the object periodically to see if the send has completed. So if
> you
> > > > have n
> > > > > >send calls outstanding you have to check n RecordSend objects
> which
> > is
> > > > > >slow. In general this tends to lead to people using one thread per
> > > send
> > > > > >call and then calling RecordSend#await which removes much of the
> > > benefit
> > > > > >of an async API.
> > > > > >
> > > > > >I think it¹s much easier to write truly asynchronous code if the
> > > > returned
> > > > > >future allows you to register a callback. That way, instead of
> > polling
> > > > you
> > > > > >can simply wait for the callback to be called. A good example of
> the
> > > > kind
> > > > > >of thing I¹m thinking is the ListenableFuture class in the Guava
> > > > > >libraries:
> > > > > >
> > > > > >
> > > >
> > https://code.google.com/p/guava-libraries/wiki/ListenableFutureExplained
> > > > > >
> > > > > >
> > > > > >HTH,
> > > > > >Oliver
> > > > > >
> > > > >
> > > > >
> > > >
> > >
> >
>
>
>
> --
> -- Guozhang
>

Re: New Producer Public API

Posted by Guozhang Wang <wa...@gmail.com>.
I think the most common motivate of having a customized partitioner is to
make sure some messages always go to the same partition, but people may
seldom want to know about which partition exactly they go to. If that is
true, why not just assign the same byte array as partition key with the
default hash based partitioning in option 1.A? But again, that is based on
my presumption that very few users would want to really specify the
partition id.



On Fri, Jan 31, 2014 at 2:44 PM, Jay Kreps <ja...@gmail.com> wrote:

> Hey Tom,
>
> Agreed, there is definitely nothing that prevents our including partitioner
> implementations, but it does get a little less seamless.
>
> -Jay
>
>
> On Fri, Jan 31, 2014 at 2:35 PM, Tom Brown <to...@gmail.com> wrote:
>
> > Regarding partitioning APIs, I don't think there is not a common subset
> of
> > information that is required for all strategies. Instead of modifying the
> > core API to easily support all of the various partitioning strategies,
> > offer the most common ones as libraries they can build into their own
> data
> > pipeline, just like serialization. The core API would simply accept a
> > partition index. You could include one default strategy (random) that
> only
> > applies if they set "-1" for the partition index.
> >
> > That way, each partitioning strategy could have its own API that makes
> > sense for it. For example, a round-robin partitioner only needs one
> method:
> > "nextPartition()", while a hash-based one needs
> "getPartitionFor(byte[])".
> >
> > For those who actually need a pluggable strategy, a superset of the API
> > could be codified into an interface (perhaps the existing partitioner
> > interface), but it would still have to be used from outside of the core
> > API.
> >
> > This design would make the core API less confusing (when do I use a
> > partiton key instead of a partition index, does the key overwrite the
> > index, can the key be null, etc...?) while still providing the
> flexibility
> > you want.
> >
> > --Tom
> >
> > On Fri, Jan 31, 2014 at 12:07 PM, Jay Kreps <ja...@gmail.com> wrote:
> >
> > > Oliver,
> > >
> > > Yeah that was my original plan--allow the registration of multiple
> > > callbacks on the future. But there is some additional implementation
> > > complexity because then you need more synchronization variables to
> ensure
> > > the callback gets executed even if the request has completed at the
> time
> > > the callback is registered. This also makes it unpredictable the order
> of
> > > callback execution--I want to be able to guarantee that for a
> particular
> > > partition callbacks for lower offset messages happen before callbacks
> for
> > > higher offset messages so that if you set a highwater mark or something
> > it
> > > is easy to reason about. This has the added benefit that callbacks
> > execute
> > > in the I/O thread ALWAYS instead of it being non-deterministic which
> is a
> > > little confusing.
> > >
> > > I thought a single callback is sufficient since you can always include
> > > multiple actions in that callback, and I think that case is rare
> anyway.
> > >
> > > I did think about the possibility of adding a thread pool for handling
> > the
> > > callbacks. But there are a lot of possible configurations for such a
> > thread
> > > pool and a simplistic approach would no longer guarantee in-order
> > > processing of callbacks (you would need to hash execution over threads
> by
> > > partition id). I think by just exposing the simple method that executes
> > in
> > > the I/O thread you can easily implement the pooled execution using the
> > > therad pooling mechanism of your choice by just having the callback use
> > an
> > > executor to run the action (i.e. make an AsyncCallback that takes a
> > > threadpool and a Runnable or something like that). This gives the user
> > full
> > > control over the executor (there are lots of details around thread
> re-use
> > > in executors, thread factories, etc and trying to expose configs for
> > every
> > > variation will be a pain). This also makes it totally transparent how
> it
> > > works; that is if we did expose all kinds of thread pool configs you
> > would
> > > still probably end up reading our code to figure out exactly what they
> > all
> > > did.
> > >
> > > -Jay
> > >
> > >
> > > On Fri, Jan 31, 2014 at 9:39 AM, Oliver Dain <odain@3cinteractive.com
> > > >wrote:
> > >
> > > > Hmmm.. I should read the docs more carefully before I open my big
> > mouth:
> > > I
> > > > just noticed the KafkaProducer#send overload that takes a callback.
> > That
> > > > definitely helps address my concern though I think the API would be
> > > > cleaner if there was only one variant that returned a future and you
> > > could
> > > > register the callback with the future. This is not nearly as
> important
> > as
> > > > I'd thought given the ability to register a callback - just a
> > preference.
> > > >
> > > >
> > > >
> > > >
> > > >
> > > > On 1/31/14, 9:33 AM, "Oliver Dain" <od...@3cinteractive.com> wrote:
> > > >
> > > > >Hey all,
> > > > >
> > > > >I¹m excited about having a new Producer API, and I really like the
> > idea
> > > of
> > > > >removing the distinction between a synchronous and asynchronous
> > > producer.
> > > > >The one comment I have about the current API is that it¹s hard to
> > write
> > > > >truly asynchronous code with the type of future returned by the send
> > > > >method. The issue is that send returns a RecordSend and there¹s no
> way
> > > to
> > > > >register a callback with that object. It is therefore necessary to
> > poll
> > > > >the object periodically to see if the send has completed. So if you
> > > have n
> > > > >send calls outstanding you have to check n RecordSend objects which
> is
> > > > >slow. In general this tends to lead to people using one thread per
> > send
> > > > >call and then calling RecordSend#await which removes much of the
> > benefit
> > > > >of an async API.
> > > > >
> > > > >I think it¹s much easier to write truly asynchronous code if the
> > > returned
> > > > >future allows you to register a callback. That way, instead of
> polling
> > > you
> > > > >can simply wait for the callback to be called. A good example of the
> > > kind
> > > > >of thing I¹m thinking is the ListenableFuture class in the Guava
> > > > >libraries:
> > > > >
> > > > >
> > >
> https://code.google.com/p/guava-libraries/wiki/ListenableFutureExplained
> > > > >
> > > > >
> > > > >HTH,
> > > > >Oliver
> > > > >
> > > >
> > > >
> > >
> >
>



-- 
-- Guozhang

Re: New Producer Public API

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

Agreed, there is definitely nothing that prevents our including partitioner
implementations, but it does get a little less seamless.

-Jay


On Fri, Jan 31, 2014 at 2:35 PM, Tom Brown <to...@gmail.com> wrote:

> Regarding partitioning APIs, I don't think there is not a common subset of
> information that is required for all strategies. Instead of modifying the
> core API to easily support all of the various partitioning strategies,
> offer the most common ones as libraries they can build into their own data
> pipeline, just like serialization. The core API would simply accept a
> partition index. You could include one default strategy (random) that only
> applies if they set "-1" for the partition index.
>
> That way, each partitioning strategy could have its own API that makes
> sense for it. For example, a round-robin partitioner only needs one method:
> "nextPartition()", while a hash-based one needs "getPartitionFor(byte[])".
>
> For those who actually need a pluggable strategy, a superset of the API
> could be codified into an interface (perhaps the existing partitioner
> interface), but it would still have to be used from outside of the core
> API.
>
> This design would make the core API less confusing (when do I use a
> partiton key instead of a partition index, does the key overwrite the
> index, can the key be null, etc...?) while still providing the flexibility
> you want.
>
> --Tom
>
> On Fri, Jan 31, 2014 at 12:07 PM, Jay Kreps <ja...@gmail.com> wrote:
>
> > Oliver,
> >
> > Yeah that was my original plan--allow the registration of multiple
> > callbacks on the future. But there is some additional implementation
> > complexity because then you need more synchronization variables to ensure
> > the callback gets executed even if the request has completed at the time
> > the callback is registered. This also makes it unpredictable the order of
> > callback execution--I want to be able to guarantee that for a particular
> > partition callbacks for lower offset messages happen before callbacks for
> > higher offset messages so that if you set a highwater mark or something
> it
> > is easy to reason about. This has the added benefit that callbacks
> execute
> > in the I/O thread ALWAYS instead of it being non-deterministic which is a
> > little confusing.
> >
> > I thought a single callback is sufficient since you can always include
> > multiple actions in that callback, and I think that case is rare anyway.
> >
> > I did think about the possibility of adding a thread pool for handling
> the
> > callbacks. But there are a lot of possible configurations for such a
> thread
> > pool and a simplistic approach would no longer guarantee in-order
> > processing of callbacks (you would need to hash execution over threads by
> > partition id). I think by just exposing the simple method that executes
> in
> > the I/O thread you can easily implement the pooled execution using the
> > therad pooling mechanism of your choice by just having the callback use
> an
> > executor to run the action (i.e. make an AsyncCallback that takes a
> > threadpool and a Runnable or something like that). This gives the user
> full
> > control over the executor (there are lots of details around thread re-use
> > in executors, thread factories, etc and trying to expose configs for
> every
> > variation will be a pain). This also makes it totally transparent how it
> > works; that is if we did expose all kinds of thread pool configs you
> would
> > still probably end up reading our code to figure out exactly what they
> all
> > did.
> >
> > -Jay
> >
> >
> > On Fri, Jan 31, 2014 at 9:39 AM, Oliver Dain <odain@3cinteractive.com
> > >wrote:
> >
> > > Hmmm.. I should read the docs more carefully before I open my big
> mouth:
> > I
> > > just noticed the KafkaProducer#send overload that takes a callback.
> That
> > > definitely helps address my concern though I think the API would be
> > > cleaner if there was only one variant that returned a future and you
> > could
> > > register the callback with the future. This is not nearly as important
> as
> > > I'd thought given the ability to register a callback - just a
> preference.
> > >
> > >
> > >
> > >
> > >
> > > On 1/31/14, 9:33 AM, "Oliver Dain" <od...@3cinteractive.com> wrote:
> > >
> > > >Hey all,
> > > >
> > > >I¹m excited about having a new Producer API, and I really like the
> idea
> > of
> > > >removing the distinction between a synchronous and asynchronous
> > producer.
> > > >The one comment I have about the current API is that it¹s hard to
> write
> > > >truly asynchronous code with the type of future returned by the send
> > > >method. The issue is that send returns a RecordSend and there¹s no way
> > to
> > > >register a callback with that object. It is therefore necessary to
> poll
> > > >the object periodically to see if the send has completed. So if you
> > have n
> > > >send calls outstanding you have to check n RecordSend objects which is
> > > >slow. In general this tends to lead to people using one thread per
> send
> > > >call and then calling RecordSend#await which removes much of the
> benefit
> > > >of an async API.
> > > >
> > > >I think it¹s much easier to write truly asynchronous code if the
> > returned
> > > >future allows you to register a callback. That way, instead of polling
> > you
> > > >can simply wait for the callback to be called. A good example of the
> > kind
> > > >of thing I¹m thinking is the ListenableFuture class in the Guava
> > > >libraries:
> > > >
> > > >
> > https://code.google.com/p/guava-libraries/wiki/ListenableFutureExplained
> > > >
> > > >
> > > >HTH,
> > > >Oliver
> > > >
> > >
> > >
> >
>

Re: New Producer Public API

Posted by Tom Brown <to...@gmail.com>.
Regarding partitioning APIs, I don't think there is not a common subset of
information that is required for all strategies. Instead of modifying the
core API to easily support all of the various partitioning strategies,
offer the most common ones as libraries they can build into their own data
pipeline, just like serialization. The core API would simply accept a
partition index. You could include one default strategy (random) that only
applies if they set "-1" for the partition index.

That way, each partitioning strategy could have its own API that makes
sense for it. For example, a round-robin partitioner only needs one method:
"nextPartition()", while a hash-based one needs "getPartitionFor(byte[])".

For those who actually need a pluggable strategy, a superset of the API
could be codified into an interface (perhaps the existing partitioner
interface), but it would still have to be used from outside of the core API.

This design would make the core API less confusing (when do I use a
partiton key instead of a partition index, does the key overwrite the
index, can the key be null, etc...?) while still providing the flexibility
you want.

--Tom

On Fri, Jan 31, 2014 at 12:07 PM, Jay Kreps <ja...@gmail.com> wrote:

> Oliver,
>
> Yeah that was my original plan--allow the registration of multiple
> callbacks on the future. But there is some additional implementation
> complexity because then you need more synchronization variables to ensure
> the callback gets executed even if the request has completed at the time
> the callback is registered. This also makes it unpredictable the order of
> callback execution--I want to be able to guarantee that for a particular
> partition callbacks for lower offset messages happen before callbacks for
> higher offset messages so that if you set a highwater mark or something it
> is easy to reason about. This has the added benefit that callbacks execute
> in the I/O thread ALWAYS instead of it being non-deterministic which is a
> little confusing.
>
> I thought a single callback is sufficient since you can always include
> multiple actions in that callback, and I think that case is rare anyway.
>
> I did think about the possibility of adding a thread pool for handling the
> callbacks. But there are a lot of possible configurations for such a thread
> pool and a simplistic approach would no longer guarantee in-order
> processing of callbacks (you would need to hash execution over threads by
> partition id). I think by just exposing the simple method that executes in
> the I/O thread you can easily implement the pooled execution using the
> therad pooling mechanism of your choice by just having the callback use an
> executor to run the action (i.e. make an AsyncCallback that takes a
> threadpool and a Runnable or something like that). This gives the user full
> control over the executor (there are lots of details around thread re-use
> in executors, thread factories, etc and trying to expose configs for every
> variation will be a pain). This also makes it totally transparent how it
> works; that is if we did expose all kinds of thread pool configs you would
> still probably end up reading our code to figure out exactly what they all
> did.
>
> -Jay
>
>
> On Fri, Jan 31, 2014 at 9:39 AM, Oliver Dain <odain@3cinteractive.com
> >wrote:
>
> > Hmmm.. I should read the docs more carefully before I open my big mouth:
> I
> > just noticed the KafkaProducer#send overload that takes a callback. That
> > definitely helps address my concern though I think the API would be
> > cleaner if there was only one variant that returned a future and you
> could
> > register the callback with the future. This is not nearly as important as
> > I'd thought given the ability to register a callback - just a preference.
> >
> >
> >
> >
> >
> > On 1/31/14, 9:33 AM, "Oliver Dain" <od...@3cinteractive.com> wrote:
> >
> > >Hey all,
> > >
> > >I¹m excited about having a new Producer API, and I really like the idea
> of
> > >removing the distinction between a synchronous and asynchronous
> producer.
> > >The one comment I have about the current API is that it¹s hard to write
> > >truly asynchronous code with the type of future returned by the send
> > >method. The issue is that send returns a RecordSend and there¹s no way
> to
> > >register a callback with that object. It is therefore necessary to poll
> > >the object periodically to see if the send has completed. So if you
> have n
> > >send calls outstanding you have to check n RecordSend objects which is
> > >slow. In general this tends to lead to people using one thread per send
> > >call and then calling RecordSend#await which removes much of the benefit
> > >of an async API.
> > >
> > >I think it¹s much easier to write truly asynchronous code if the
> returned
> > >future allows you to register a callback. That way, instead of polling
> you
> > >can simply wait for the callback to be called. A good example of the
> kind
> > >of thing I¹m thinking is the ListenableFuture class in the Guava
> > >libraries:
> > >
> > >
> https://code.google.com/p/guava-libraries/wiki/ListenableFutureExplained
> > >
> > >
> > >HTH,
> > >Oliver
> > >
> >
> >
>

Re: New Producer Public API

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

Yeah that was my original plan--allow the registration of multiple
callbacks on the future. But there is some additional implementation
complexity because then you need more synchronization variables to ensure
the callback gets executed even if the request has completed at the time
the callback is registered. This also makes it unpredictable the order of
callback execution--I want to be able to guarantee that for a particular
partition callbacks for lower offset messages happen before callbacks for
higher offset messages so that if you set a highwater mark or something it
is easy to reason about. This has the added benefit that callbacks execute
in the I/O thread ALWAYS instead of it being non-deterministic which is a
little confusing.

I thought a single callback is sufficient since you can always include
multiple actions in that callback, and I think that case is rare anyway.

I did think about the possibility of adding a thread pool for handling the
callbacks. But there are a lot of possible configurations for such a thread
pool and a simplistic approach would no longer guarantee in-order
processing of callbacks (you would need to hash execution over threads by
partition id). I think by just exposing the simple method that executes in
the I/O thread you can easily implement the pooled execution using the
therad pooling mechanism of your choice by just having the callback use an
executor to run the action (i.e. make an AsyncCallback that takes a
threadpool and a Runnable or something like that). This gives the user full
control over the executor (there are lots of details around thread re-use
in executors, thread factories, etc and trying to expose configs for every
variation will be a pain). This also makes it totally transparent how it
works; that is if we did expose all kinds of thread pool configs you would
still probably end up reading our code to figure out exactly what they all
did.

-Jay


On Fri, Jan 31, 2014 at 9:39 AM, Oliver Dain <od...@3cinteractive.com>wrote:

> Hmmm.. I should read the docs more carefully before I open my big mouth: I
> just noticed the KafkaProducer#send overload that takes a callback. That
> definitely helps address my concern though I think the API would be
> cleaner if there was only one variant that returned a future and you could
> register the callback with the future. This is not nearly as important as
> I'd thought given the ability to register a callback - just a preference.
>
>
>
>
>
> On 1/31/14, 9:33 AM, "Oliver Dain" <od...@3cinteractive.com> wrote:
>
> >Hey all,
> >
> >I¹m excited about having a new Producer API, and I really like the idea of
> >removing the distinction between a synchronous and asynchronous producer.
> >The one comment I have about the current API is that it¹s hard to write
> >truly asynchronous code with the type of future returned by the send
> >method. The issue is that send returns a RecordSend and there¹s no way to
> >register a callback with that object. It is therefore necessary to poll
> >the object periodically to see if the send has completed. So if you have n
> >send calls outstanding you have to check n RecordSend objects which is
> >slow. In general this tends to lead to people using one thread per send
> >call and then calling RecordSend#await which removes much of the benefit
> >of an async API.
> >
> >I think it¹s much easier to write truly asynchronous code if the returned
> >future allows you to register a callback. That way, instead of polling you
> >can simply wait for the callback to be called. A good example of the kind
> >of thing I¹m thinking is the ListenableFuture class in the Guava
> >libraries:
> >
> >https://code.google.com/p/guava-libraries/wiki/ListenableFutureExplained
> >
> >
> >HTH,
> >Oliver
> >
>
>

Re: New Producer Public API

Posted by Oliver Dain <od...@3cinteractive.com>.
Hmmm.. I should read the docs more carefully before I open my big mouth: I
just noticed the KafkaProducer#send overload that takes a callback. That
definitely helps address my concern though I think the API would be
cleaner if there was only one variant that returned a future and you could
register the callback with the future. This is not nearly as important as
I’d thought given the ability to register a callback - just a preference.





On 1/31/14, 9:33 AM, "Oliver Dain" <od...@3cinteractive.com> wrote:

>Hey all,
>
>I¹m excited about having a new Producer API, and I really like the idea of
>removing the distinction between a synchronous and asynchronous producer.
>The one comment I have about the current API is that it¹s hard to write
>truly asynchronous code with the type of future returned by the send
>method. The issue is that send returns a RecordSend and there¹s no way to
>register a callback with that object. It is therefore necessary to poll
>the object periodically to see if the send has completed. So if you have n
>send calls outstanding you have to check n RecordSend objects which is
>slow. In general this tends to lead to people using one thread per send
>call and then calling RecordSend#await which removes much of the benefit
>of an async API.
>
>I think it¹s much easier to write truly asynchronous code if the returned
>future allows you to register a callback. That way, instead of polling you
>can simply wait for the callback to be called. A good example of the kind
>of thing I¹m thinking is the ListenableFuture class in the Guava
>libraries:
>
>https://code.google.com/p/guava-libraries/wiki/ListenableFutureExplained
>
>
>HTH,
>Oliver
>


Re: New Producer Public API

Posted by Oliver Dain <od...@3cinteractive.com>.
Hey all,

I¹m excited about having a new Producer API, and I really like the idea of
removing the distinction between a synchronous and asynchronous producer.
The one comment I have about the current API is that it¹s hard to write
truly asynchronous code with the type of future returned by the send
method. The issue is that send returns a RecordSend and there¹s no way to
register a callback with that object. It is therefore necessary to poll
the object periodically to see if the send has completed. So if you have n
send calls outstanding you have to check n RecordSend objects which is
slow. In general this tends to lead to people using one thread per send
call and then calling RecordSend#await which removes much of the benefit
of an async API.

I think it¹s much easier to write truly asynchronous code if the returned
future allows you to register a callback. That way, instead of polling you
can simply wait for the callback to be called. A good example of the kind
of thing I¹m thinking is the ListenableFuture class in the Guava libraries:

https://code.google.com/p/guava-libraries/wiki/ListenableFutureExplained


HTH,
Oliver


Re: New Producer Public API

Posted by Oliver Dain <od...@3cinteractive.com>.
I wanted to suggest an alternative to the serialization issue. As I
understand it, the concern is that if the user is responsible for
serialization it becomes difficult for them to compute the partition as
the plugin that computes the partition would be called with byte[] forcing
the user to de-serialize the object in order to determine the correct
partition. My suggestion is to do away with the plugin and instead provide
a method like

RecordSend send(byte[] data, int partition)

That way, if the user wants to compute the partition themselves they can
do that with the regular object before they serialize it.


Re: New Producer Public API

Posted by Jay Kreps <ja...@gmail.com>.
One downside to the 1A proposal is that without a Partitioner interface we
can't really package up and provide common partitioner implementations.
Example of these would be
1. HashPartitioner - The default hash partitioning
2. RoundRobinPartitioner - Just round-robins over partitions
3. ConnectionMinimizingPartitioner - Choose partitions to minimize the
number of nodes you need to connect maintain TCP connections to.
4. RangePartitioner - User provides break points that align partitions to
key ranges
5. LocalityPartitioner - Prefer nodes on the same rack. This would be nice
for stream-processing use cases that read from one topic and write to
another. We would have to include rack information in our metadata.

Having this kind of functionality included is actually kind of nice.

-Jay


On Fri, Jan 24, 2014 at 5:17 PM, Jay Kreps <ja...@gmail.com> wrote:

> Clark and all,
>
> I thought a little bit about the serialization question. Here are the
> options I see and the pros and cons I can think of. I'd love to hear
> people's preferences if you have a strong one.
>
> One important consideration is that however the producer works will also
> need to be how the new consumer works (which we hope to write next). That
> is if you put objects in, you should get objects out. So we need to think
> through both sides.
>
> Options:
>
> Option 0: What is in the existing scala code and the java code I
> posted--Serializer and Partitioner plugin provided by the user via config.
> Partitioner has a sane default, but Serializer needs to be specified in
> config.
>
> Pros: How it works today in the scala code.
> Cons: You have to adapt your serialization library of choice to our
> interfaces. The reflective class loading means typo in the serializer name
> give odd errors. Likewise there is little type safety--the ProducerRecord
> takes Object and any type errors between the object provided and the
> serializer give occurs at runtime.
>
> Option 1: No plugins
>
> This would mean byte[] key, byte[] value, and partitioning done by client
> by passing in a partition *number* directly.
>
> The problem with this is that it is tricky to compute the partition
> correctly and probably most people won't. We could add a getCluster()
> method to return the Cluster instance you should use for partitioning. But
> I suspect people would be lazy and not use that and instead hard-code
> partitions which would break if partitions were added or they hard coded it
> wrong. In my experience 3 partitioning strategies cover like 99% of cases
> so not having a default implementation for this makes the common case
> harder. Left to their own devices people will use bad hash functions and
> get weird results.
>
> Option 1A: Alternatively we could partition by the key using the existing
> default partitioning strategy which only uses the byte[] anyway but instead
> of having a partitionKey we could have a numerical partition override and
> add the getCluster() method to get the cluster metadata. That would make
> custom partitioning possible but handle the common case simply.
>
> Option 2: Partitioner plugin remains, serializers go.
>
> The problem here is that the partitioner might lose access to the
> deserialized key which would occasionally be useful for semantic
> partitioning schemes. The Partitioner could deserialize the key but that
> would be inefficient and weird.
>
> This problem could be fixed by having key and value be byte[] but
> retaining partitionKey as an Object and passing it to the partitioner as
> is. Then if you have a partitioner which requires the deserialized key you
> would need to use this partition key. One weird side effect is that if you
> want to have a custom partition key BUT want to partition by the bytes of
> that key rather than the object value you must write a customer partitioner
> and serialize it yourself.
>
> Of these I think I prefer 1A but could be convinced of 0 since that is how
> it works now.
>
> Thoughts?
>
> -Jay
>
>
> On Fri, Jan 24, 2014 at 3:30 PM, Clark Breyman <cl...@breyman.com> wrote:
>
>> Jay - Thanks for the call for comments. Here's some initial input:
>>
>> - Make message serialization a client responsibility (making all messages
>> byte[]). Reflection-based loading makes it harder to use generic codecs
>> (e.g.  Envelope<PREFIX, DATA, SUFFIX>) or build up codec programmatically.
>> Non-default partitioning should require an explicit partition key.
>>
>> - I really like the fact that it will be native Java. Please consider
>> using
>> native maven and not sbt, gradle, ivy, etc as they don't reliably play
>> nice
>> in the maven ecosystem. A jar without a well-formed pom doesn't feel like
>> a
>> real artifact. The pom's generated by sbt et al. are not well formed.
>> Using
>> maven will make builds and IDE integration much smoother.
>>
>> - Look at Nick Telford's dropwizard-extras package in which he defines
>> some
>> Jackson-compatible POJO's for loading configuration. Seems like your
>> client
>> migration is similar. The config objects should have constructors or
>> factories that accept Map<String, String> and Properties for ease of
>> migration.
>>
>> - Would you consider using the org.apache.kafka package for the new API
>> (quibble)
>>
>> - Why create your own futures rather than use
>> java.util.concurrent.Future<Long> or similar? Standard futures will play
>> nice with other reactive libs and things like J8's ComposableFuture.
>>
>> Thanks again,
>> C
>>
>>
>>
>> On Fri, Jan 24, 2014 at 2:46 PM, Roger Hoover <roger.hoover@gmail.com
>> >wrote:
>>
>> > A couple comments:
>> >
>> > 1) Why does the config use a broker list instead of discovering the
>> brokers
>> > in ZooKeeper?  It doesn't match the HighLevelConsumer API.
>> >
>> > 2) It looks like broker connections are created on demand.  I'm
>> wondering
>> > if sometimes you might want to flush out config or network connectivity
>> > issues before pushing the first message through.
>> >
>> > Should there also be a KafkaProducer.connect() or .open() method or
>> > connectAll()?  I guess it would try to connect to all brokers in the
>> > BROKER_LIST_CONFIG
>> >
>> > HTH,
>> >
>> > Roger
>> >
>> >
>> > On Fri, Jan 24, 2014 at 11:54 AM, Jay Kreps <ja...@gmail.com>
>> wrote:
>> >
>> > > As mentioned in a previous email we are working on a
>> re-implementation of
>> > > the producer. I would like to use this email thread to discuss the
>> > details
>> > > of the public API and the configuration. I would love for us to be
>> > > incredibly picky about this public api now so it is as good as
>> possible
>> > and
>> > > we don't need to break it in the future.
>> > >
>> > > The best way to get a feel for the API is actually to take a look at
>> the
>> > > javadoc, my hope is to get the api docs good enough so that it is
>> > > self-explanatory:
>> > >
>> > >
>> >
>> http://empathybox.com/kafka-javadoc/index.html?kafka/clients/producer/KafkaProducer.html
>> > >
>> > > Please take a look at this API and give me any thoughts you may have!
>> > >
>> > > It may also be reasonable to take a look at the configs:
>> > >
>> > >
>> >
>> http://empathybox.com/kafka-javadoc/kafka/clients/producer/ProducerConfig.html
>> > >
>> > > The actual code is posted here:
>> > > https://issues.apache.org/jira/browse/KAFKA-1227
>> > >
>> > > A few questions or comments to kick things off:
>> > > 1. We need to make a decision on whether serialization of the user's
>> key
>> > > and value should be done by the user (with our api just taking
>> byte[]) or
>> > > if we should take an object and allow the user to configure a
>> Serializer
>> > > class which we instantiate via reflection. We take the later approach
>> in
>> > > the current producer, and I have carried this through to this
>> prototype.
>> > > The tradeoff I see is this: taking byte[] is actually simpler, the
>> user
>> > can
>> > > directly do whatever serialization they like. The complication is
>> > actually
>> > > partitioning. Currently partitioning is done by a similar plug-in api
>> > > (Partitioner) which the user can implement and configure to override
>> how
>> > > partitions are assigned. If we take byte[] as input then we have no
>> > access
>> > > to the original object and partitioning MUST be done on the byte[].
>> This
>> > is
>> > > fine for hash partitioning. However for various types of semantic
>> > > partitioning (range partitioning, or whatever) you would want access
>> to
>> > the
>> > > original object. In the current approach a producer who wishes to send
>> > > byte[] they have serialized in their own code can configure the
>> > > BytesSerialization we supply which is just a "no op" serialization.
>> > > 2. We should obsess over naming and make sure each of the class names
>> are
>> > > good.
>> > > 3. Jun has already pointed out that we need to include the topic and
>> > > partition in the response, which is absolutely right. I haven't done
>> that
>> > > yet but that definitely needs to be there.
>> > > 4. Currently RecordSend.await will throw an exception if the request
>> > > failed. The intention here is that producer.send(message).await()
>> exactly
>> > > simulates a synchronous call. Guozhang has noted that this is a little
>> > > annoying since the user must then catch exceptions. However if we
>> remove
>> > > this then if the user doesn't check for errors they won't know one has
>> > > occurred, which I predict will be a common mistake.
>> > > 5. Perhaps there is more we could do to make the async callbacks and
>> > future
>> > > we give back intuitive and easy to program against?
>> > >
>> > > Some background info on implementation:
>> > >
>> > > At a high level the primary difference in this producer is that it
>> > removes
>> > > the distinction between the "sync" and "async" producer. Effectively
>> all
>> > > requests are sent asynchronously but always return a future response
>> > object
>> > > that gives the offset as well as any error that may have occurred when
>> > the
>> > > request is complete. The batching that is done in the async producer
>> only
>> > > today is done whenever possible now. This means that the sync
>> producer,
>> > > under load, can get performance as good as the async producer
>> > (preliminary
>> > > results show the producer getting 1m messages/sec). This works
>> similar to
>> > > group commit in databases but with respect to the actual network
>> > > transmission--any messages that arrive while a send is in progress are
>> > > batched together. It is also possible to encourage batching even under
>> > low
>> > > load to save server resources by introducing a delay on the send to
>> allow
>> > > more messages to accumulate; this is done using the linger.ms config
>> > (this
>> > > is similar to Nagle's algorithm in TCP).
>> > >
>> > > This producer does all network communication asynchronously and in
>> > parallel
>> > > to all servers so the performance penalty for acks=-1 and waiting on
>> > > replication should be much reduced. I haven't done much benchmarking
>> on
>> > > this yet, though.
>> > >
>> > > The high level design is described a little here, though this is now a
>> > > little out of date:
>> > > https://cwiki.apache.org/confluence/display/KAFKA/Client+Rewrite
>> > >
>> > > -Jay
>> > >
>> >
>>
>
>

Re: New Producer Public API

Posted by Jun Rao <ju...@gmail.com>.
With option 1A, if we increase # partitions on a topic, how will the
producer find out newly created partitions? Do we expect the producer to
periodically call getCluster()?

As for ZK dependency, one of the goals of client rewrite is to reduce
dependencies so that one can implement the client in languages other than
java. ZK client is only available in a small number of languages.

Thanks,

Jun


On Fri, Jan 24, 2014 at 5:17 PM, Jay Kreps <ja...@gmail.com> wrote:

> Clark and all,
>
> I thought a little bit about the serialization question. Here are the
> options I see and the pros and cons I can think of. I'd love to hear
> people's preferences if you have a strong one.
>
> One important consideration is that however the producer works will also
> need to be how the new consumer works (which we hope to write next). That
> is if you put objects in, you should get objects out. So we need to think
> through both sides.
>
> Options:
>
> Option 0: What is in the existing scala code and the java code I
> posted--Serializer and Partitioner plugin provided by the user via config.
> Partitioner has a sane default, but Serializer needs to be specified in
> config.
>
> Pros: How it works today in the scala code.
> Cons: You have to adapt your serialization library of choice to our
> interfaces. The reflective class loading means typo in the serializer name
> give odd errors. Likewise there is little type safety--the ProducerRecord
> takes Object and any type errors between the object provided and the
> serializer give occurs at runtime.
>
> Option 1: No plugins
>
> This would mean byte[] key, byte[] value, and partitioning done by client
> by passing in a partition *number* directly.
>
> The problem with this is that it is tricky to compute the partition
> correctly and probably most people won't. We could add a getCluster()
> method to return the Cluster instance you should use for partitioning. But
> I suspect people would be lazy and not use that and instead hard-code
> partitions which would break if partitions were added or they hard coded it
> wrong. In my experience 3 partitioning strategies cover like 99% of cases
> so not having a default implementation for this makes the common case
> harder. Left to their own devices people will use bad hash functions and
> get weird results.
>
> Option 1A: Alternatively we could partition by the key using the existing
> default partitioning strategy which only uses the byte[] anyway but instead
> of having a partitionKey we could have a numerical partition override and
> add the getCluster() method to get the cluster metadata. That would make
> custom partitioning possible but handle the common case simply.
>
> Option 2: Partitioner plugin remains, serializers go.
>
> The problem here is that the partitioner might lose access to the
> deserialized key which would occasionally be useful for semantic
> partitioning schemes. The Partitioner could deserialize the key but that
> would be inefficient and weird.
>
> This problem could be fixed by having key and value be byte[] but retaining
> partitionKey as an Object and passing it to the partitioner as is. Then if
> you have a partitioner which requires the deserialized key you would need
> to use this partition key. One weird side effect is that if you want to
> have a custom partition key BUT want to partition by the bytes of that key
> rather than the object value you must write a customer partitioner and
> serialize it yourself.
>
> Of these I think I prefer 1A but could be convinced of 0 since that is how
> it works now.
>
> Thoughts?
>
> -Jay
>
>
> On Fri, Jan 24, 2014 at 3:30 PM, Clark Breyman <cl...@breyman.com> wrote:
>
> > Jay - Thanks for the call for comments. Here's some initial input:
> >
> > - Make message serialization a client responsibility (making all messages
> > byte[]). Reflection-based loading makes it harder to use generic codecs
> > (e.g.  Envelope<PREFIX, DATA, SUFFIX>) or build up codec
> programmatically.
> > Non-default partitioning should require an explicit partition key.
> >
> > - I really like the fact that it will be native Java. Please consider
> using
> > native maven and not sbt, gradle, ivy, etc as they don't reliably play
> nice
> > in the maven ecosystem. A jar without a well-formed pom doesn't feel
> like a
> > real artifact. The pom's generated by sbt et al. are not well formed.
> Using
> > maven will make builds and IDE integration much smoother.
> >
> > - Look at Nick Telford's dropwizard-extras package in which he defines
> some
> > Jackson-compatible POJO's for loading configuration. Seems like your
> client
> > migration is similar. The config objects should have constructors or
> > factories that accept Map<String, String> and Properties for ease of
> > migration.
> >
> > - Would you consider using the org.apache.kafka package for the new API
> > (quibble)
> >
> > - Why create your own futures rather than use
> > java.util.concurrent.Future<Long> or similar? Standard futures will play
> > nice with other reactive libs and things like J8's ComposableFuture.
> >
> > Thanks again,
> > C
> >
> >
> >
> > On Fri, Jan 24, 2014 at 2:46 PM, Roger Hoover <roger.hoover@gmail.com
> > >wrote:
> >
> > > A couple comments:
> > >
> > > 1) Why does the config use a broker list instead of discovering the
> > brokers
> > > in ZooKeeper?  It doesn't match the HighLevelConsumer API.
> > >
> > > 2) It looks like broker connections are created on demand.  I'm
> wondering
> > > if sometimes you might want to flush out config or network connectivity
> > > issues before pushing the first message through.
> > >
> > > Should there also be a KafkaProducer.connect() or .open() method or
> > > connectAll()?  I guess it would try to connect to all brokers in the
> > > BROKER_LIST_CONFIG
> > >
> > > HTH,
> > >
> > > Roger
> > >
> > >
> > > On Fri, Jan 24, 2014 at 11:54 AM, Jay Kreps <ja...@gmail.com>
> wrote:
> > >
> > > > As mentioned in a previous email we are working on a
> re-implementation
> > of
> > > > the producer. I would like to use this email thread to discuss the
> > > details
> > > > of the public API and the configuration. I would love for us to be
> > > > incredibly picky about this public api now so it is as good as
> possible
> > > and
> > > > we don't need to break it in the future.
> > > >
> > > > The best way to get a feel for the API is actually to take a look at
> > the
> > > > javadoc, my hope is to get the api docs good enough so that it is
> > > > self-explanatory:
> > > >
> > > >
> > >
> >
> http://empathybox.com/kafka-javadoc/index.html?kafka/clients/producer/KafkaProducer.html
> > > >
> > > > Please take a look at this API and give me any thoughts you may have!
> > > >
> > > > It may also be reasonable to take a look at the configs:
> > > >
> > > >
> > >
> >
> http://empathybox.com/kafka-javadoc/kafka/clients/producer/ProducerConfig.html
> > > >
> > > > The actual code is posted here:
> > > > https://issues.apache.org/jira/browse/KAFKA-1227
> > > >
> > > > A few questions or comments to kick things off:
> > > > 1. We need to make a decision on whether serialization of the user's
> > key
> > > > and value should be done by the user (with our api just taking
> byte[])
> > or
> > > > if we should take an object and allow the user to configure a
> > Serializer
> > > > class which we instantiate via reflection. We take the later approach
> > in
> > > > the current producer, and I have carried this through to this
> > prototype.
> > > > The tradeoff I see is this: taking byte[] is actually simpler, the
> user
> > > can
> > > > directly do whatever serialization they like. The complication is
> > > actually
> > > > partitioning. Currently partitioning is done by a similar plug-in api
> > > > (Partitioner) which the user can implement and configure to override
> > how
> > > > partitions are assigned. If we take byte[] as input then we have no
> > > access
> > > > to the original object and partitioning MUST be done on the byte[].
> > This
> > > is
> > > > fine for hash partitioning. However for various types of semantic
> > > > partitioning (range partitioning, or whatever) you would want access
> to
> > > the
> > > > original object. In the current approach a producer who wishes to
> send
> > > > byte[] they have serialized in their own code can configure the
> > > > BytesSerialization we supply which is just a "no op" serialization.
> > > > 2. We should obsess over naming and make sure each of the class names
> > are
> > > > good.
> > > > 3. Jun has already pointed out that we need to include the topic and
> > > > partition in the response, which is absolutely right. I haven't done
> > that
> > > > yet but that definitely needs to be there.
> > > > 4. Currently RecordSend.await will throw an exception if the request
> > > > failed. The intention here is that producer.send(message).await()
> > exactly
> > > > simulates a synchronous call. Guozhang has noted that this is a
> little
> > > > annoying since the user must then catch exceptions. However if we
> > remove
> > > > this then if the user doesn't check for errors they won't know one
> has
> > > > occurred, which I predict will be a common mistake.
> > > > 5. Perhaps there is more we could do to make the async callbacks and
> > > future
> > > > we give back intuitive and easy to program against?
> > > >
> > > > Some background info on implementation:
> > > >
> > > > At a high level the primary difference in this producer is that it
> > > removes
> > > > the distinction between the "sync" and "async" producer. Effectively
> > all
> > > > requests are sent asynchronously but always return a future response
> > > object
> > > > that gives the offset as well as any error that may have occurred
> when
> > > the
> > > > request is complete. The batching that is done in the async producer
> > only
> > > > today is done whenever possible now. This means that the sync
> producer,
> > > > under load, can get performance as good as the async producer
> > > (preliminary
> > > > results show the producer getting 1m messages/sec). This works
> similar
> > to
> > > > group commit in databases but with respect to the actual network
> > > > transmission--any messages that arrive while a send is in progress
> are
> > > > batched together. It is also possible to encourage batching even
> under
> > > low
> > > > load to save server resources by introducing a delay on the send to
> > allow
> > > > more messages to accumulate; this is done using the linger.ms config
> > > (this
> > > > is similar to Nagle's algorithm in TCP).
> > > >
> > > > This producer does all network communication asynchronously and in
> > > parallel
> > > > to all servers so the performance penalty for acks=-1 and waiting on
> > > > replication should be much reduced. I haven't done much benchmarking
> on
> > > > this yet, though.
> > > >
> > > > The high level design is described a little here, though this is now
> a
> > > > little out of date:
> > > > https://cwiki.apache.org/confluence/display/KAFKA/Client+Rewrite
> > > >
> > > > -Jay
> > > >
> > >
> >
>

Re: New Producer Public API

Posted by Jay Kreps <ja...@gmail.com>.
One downside to the 1A proposal is that without a Partitioner interface we
can't really package up and provide common partitioner implementations.
Example of these would be
1. HashPartitioner - The default hash partitioning
2. RoundRobinPartitioner - Just round-robins over partitions
3. ConnectionMinimizingPartitioner - Choose partitions to minimize the
number of nodes you need to connect maintain TCP connections to.
4. RangePartitioner - User provides break points that align partitions to
key ranges
5. LocalityPartitioner - Prefer nodes on the same rack. This would be nice
for stream-processing use cases that read from one topic and write to
another. We would have to include rack information in our metadata.

Having this kind of functionality included is actually kind of nice.

-Jay


On Fri, Jan 24, 2014 at 5:17 PM, Jay Kreps <ja...@gmail.com> wrote:

> Clark and all,
>
> I thought a little bit about the serialization question. Here are the
> options I see and the pros and cons I can think of. I'd love to hear
> people's preferences if you have a strong one.
>
> One important consideration is that however the producer works will also
> need to be how the new consumer works (which we hope to write next). That
> is if you put objects in, you should get objects out. So we need to think
> through both sides.
>
> Options:
>
> Option 0: What is in the existing scala code and the java code I
> posted--Serializer and Partitioner plugin provided by the user via config.
> Partitioner has a sane default, but Serializer needs to be specified in
> config.
>
> Pros: How it works today in the scala code.
> Cons: You have to adapt your serialization library of choice to our
> interfaces. The reflective class loading means typo in the serializer name
> give odd errors. Likewise there is little type safety--the ProducerRecord
> takes Object and any type errors between the object provided and the
> serializer give occurs at runtime.
>
> Option 1: No plugins
>
> This would mean byte[] key, byte[] value, and partitioning done by client
> by passing in a partition *number* directly.
>
> The problem with this is that it is tricky to compute the partition
> correctly and probably most people won't. We could add a getCluster()
> method to return the Cluster instance you should use for partitioning. But
> I suspect people would be lazy and not use that and instead hard-code
> partitions which would break if partitions were added or they hard coded it
> wrong. In my experience 3 partitioning strategies cover like 99% of cases
> so not having a default implementation for this makes the common case
> harder. Left to their own devices people will use bad hash functions and
> get weird results.
>
> Option 1A: Alternatively we could partition by the key using the existing
> default partitioning strategy which only uses the byte[] anyway but instead
> of having a partitionKey we could have a numerical partition override and
> add the getCluster() method to get the cluster metadata. That would make
> custom partitioning possible but handle the common case simply.
>
> Option 2: Partitioner plugin remains, serializers go.
>
> The problem here is that the partitioner might lose access to the
> deserialized key which would occasionally be useful for semantic
> partitioning schemes. The Partitioner could deserialize the key but that
> would be inefficient and weird.
>
> This problem could be fixed by having key and value be byte[] but
> retaining partitionKey as an Object and passing it to the partitioner as
> is. Then if you have a partitioner which requires the deserialized key you
> would need to use this partition key. One weird side effect is that if you
> want to have a custom partition key BUT want to partition by the bytes of
> that key rather than the object value you must write a customer partitioner
> and serialize it yourself.
>
> Of these I think I prefer 1A but could be convinced of 0 since that is how
> it works now.
>
> Thoughts?
>
> -Jay
>
>
> On Fri, Jan 24, 2014 at 3:30 PM, Clark Breyman <cl...@breyman.com> wrote:
>
>> Jay - Thanks for the call for comments. Here's some initial input:
>>
>> - Make message serialization a client responsibility (making all messages
>> byte[]). Reflection-based loading makes it harder to use generic codecs
>> (e.g.  Envelope<PREFIX, DATA, SUFFIX>) or build up codec programmatically.
>> Non-default partitioning should require an explicit partition key.
>>
>> - I really like the fact that it will be native Java. Please consider
>> using
>> native maven and not sbt, gradle, ivy, etc as they don't reliably play
>> nice
>> in the maven ecosystem. A jar without a well-formed pom doesn't feel like
>> a
>> real artifact. The pom's generated by sbt et al. are not well formed.
>> Using
>> maven will make builds and IDE integration much smoother.
>>
>> - Look at Nick Telford's dropwizard-extras package in which he defines
>> some
>> Jackson-compatible POJO's for loading configuration. Seems like your
>> client
>> migration is similar. The config objects should have constructors or
>> factories that accept Map<String, String> and Properties for ease of
>> migration.
>>
>> - Would you consider using the org.apache.kafka package for the new API
>> (quibble)
>>
>> - Why create your own futures rather than use
>> java.util.concurrent.Future<Long> or similar? Standard futures will play
>> nice with other reactive libs and things like J8's ComposableFuture.
>>
>> Thanks again,
>> C
>>
>>
>>
>> On Fri, Jan 24, 2014 at 2:46 PM, Roger Hoover <roger.hoover@gmail.com
>> >wrote:
>>
>> > A couple comments:
>> >
>> > 1) Why does the config use a broker list instead of discovering the
>> brokers
>> > in ZooKeeper?  It doesn't match the HighLevelConsumer API.
>> >
>> > 2) It looks like broker connections are created on demand.  I'm
>> wondering
>> > if sometimes you might want to flush out config or network connectivity
>> > issues before pushing the first message through.
>> >
>> > Should there also be a KafkaProducer.connect() or .open() method or
>> > connectAll()?  I guess it would try to connect to all brokers in the
>> > BROKER_LIST_CONFIG
>> >
>> > HTH,
>> >
>> > Roger
>> >
>> >
>> > On Fri, Jan 24, 2014 at 11:54 AM, Jay Kreps <ja...@gmail.com>
>> wrote:
>> >
>> > > As mentioned in a previous email we are working on a
>> re-implementation of
>> > > the producer. I would like to use this email thread to discuss the
>> > details
>> > > of the public API and the configuration. I would love for us to be
>> > > incredibly picky about this public api now so it is as good as
>> possible
>> > and
>> > > we don't need to break it in the future.
>> > >
>> > > The best way to get a feel for the API is actually to take a look at
>> the
>> > > javadoc, my hope is to get the api docs good enough so that it is
>> > > self-explanatory:
>> > >
>> > >
>> >
>> http://empathybox.com/kafka-javadoc/index.html?kafka/clients/producer/KafkaProducer.html
>> > >
>> > > Please take a look at this API and give me any thoughts you may have!
>> > >
>> > > It may also be reasonable to take a look at the configs:
>> > >
>> > >
>> >
>> http://empathybox.com/kafka-javadoc/kafka/clients/producer/ProducerConfig.html
>> > >
>> > > The actual code is posted here:
>> > > https://issues.apache.org/jira/browse/KAFKA-1227
>> > >
>> > > A few questions or comments to kick things off:
>> > > 1. We need to make a decision on whether serialization of the user's
>> key
>> > > and value should be done by the user (with our api just taking
>> byte[]) or
>> > > if we should take an object and allow the user to configure a
>> Serializer
>> > > class which we instantiate via reflection. We take the later approach
>> in
>> > > the current producer, and I have carried this through to this
>> prototype.
>> > > The tradeoff I see is this: taking byte[] is actually simpler, the
>> user
>> > can
>> > > directly do whatever serialization they like. The complication is
>> > actually
>> > > partitioning. Currently partitioning is done by a similar plug-in api
>> > > (Partitioner) which the user can implement and configure to override
>> how
>> > > partitions are assigned. If we take byte[] as input then we have no
>> > access
>> > > to the original object and partitioning MUST be done on the byte[].
>> This
>> > is
>> > > fine for hash partitioning. However for various types of semantic
>> > > partitioning (range partitioning, or whatever) you would want access
>> to
>> > the
>> > > original object. In the current approach a producer who wishes to send
>> > > byte[] they have serialized in their own code can configure the
>> > > BytesSerialization we supply which is just a "no op" serialization.
>> > > 2. We should obsess over naming and make sure each of the class names
>> are
>> > > good.
>> > > 3. Jun has already pointed out that we need to include the topic and
>> > > partition in the response, which is absolutely right. I haven't done
>> that
>> > > yet but that definitely needs to be there.
>> > > 4. Currently RecordSend.await will throw an exception if the request
>> > > failed. The intention here is that producer.send(message).await()
>> exactly
>> > > simulates a synchronous call. Guozhang has noted that this is a little
>> > > annoying since the user must then catch exceptions. However if we
>> remove
>> > > this then if the user doesn't check for errors they won't know one has
>> > > occurred, which I predict will be a common mistake.
>> > > 5. Perhaps there is more we could do to make the async callbacks and
>> > future
>> > > we give back intuitive and easy to program against?
>> > >
>> > > Some background info on implementation:
>> > >
>> > > At a high level the primary difference in this producer is that it
>> > removes
>> > > the distinction between the "sync" and "async" producer. Effectively
>> all
>> > > requests are sent asynchronously but always return a future response
>> > object
>> > > that gives the offset as well as any error that may have occurred when
>> > the
>> > > request is complete. The batching that is done in the async producer
>> only
>> > > today is done whenever possible now. This means that the sync
>> producer,
>> > > under load, can get performance as good as the async producer
>> > (preliminary
>> > > results show the producer getting 1m messages/sec). This works
>> similar to
>> > > group commit in databases but with respect to the actual network
>> > > transmission--any messages that arrive while a send is in progress are
>> > > batched together. It is also possible to encourage batching even under
>> > low
>> > > load to save server resources by introducing a delay on the send to
>> allow
>> > > more messages to accumulate; this is done using the linger.ms config
>> > (this
>> > > is similar to Nagle's algorithm in TCP).
>> > >
>> > > This producer does all network communication asynchronously and in
>> > parallel
>> > > to all servers so the performance penalty for acks=-1 and waiting on
>> > > replication should be much reduced. I haven't done much benchmarking
>> on
>> > > this yet, though.
>> > >
>> > > The high level design is described a little here, though this is now a
>> > > little out of date:
>> > > https://cwiki.apache.org/confluence/display/KAFKA/Client+Rewrite
>> > >
>> > > -Jay
>> > >
>> >
>>
>
>

Re: New Producer Public API

Posted by Jay Kreps <ja...@gmail.com>.
Clark and all,

I thought a little bit about the serialization question. Here are the
options I see and the pros and cons I can think of. I'd love to hear
people's preferences if you have a strong one.

One important consideration is that however the producer works will also
need to be how the new consumer works (which we hope to write next). That
is if you put objects in, you should get objects out. So we need to think
through both sides.

Options:

Option 0: What is in the existing scala code and the java code I
posted--Serializer and Partitioner plugin provided by the user via config.
Partitioner has a sane default, but Serializer needs to be specified in
config.

Pros: How it works today in the scala code.
Cons: You have to adapt your serialization library of choice to our
interfaces. The reflective class loading means typo in the serializer name
give odd errors. Likewise there is little type safety--the ProducerRecord
takes Object and any type errors between the object provided and the
serializer give occurs at runtime.

Option 1: No plugins

This would mean byte[] key, byte[] value, and partitioning done by client
by passing in a partition *number* directly.

The problem with this is that it is tricky to compute the partition
correctly and probably most people won't. We could add a getCluster()
method to return the Cluster instance you should use for partitioning. But
I suspect people would be lazy and not use that and instead hard-code
partitions which would break if partitions were added or they hard coded it
wrong. In my experience 3 partitioning strategies cover like 99% of cases
so not having a default implementation for this makes the common case
harder. Left to their own devices people will use bad hash functions and
get weird results.

Option 1A: Alternatively we could partition by the key using the existing
default partitioning strategy which only uses the byte[] anyway but instead
of having a partitionKey we could have a numerical partition override and
add the getCluster() method to get the cluster metadata. That would make
custom partitioning possible but handle the common case simply.

Option 2: Partitioner plugin remains, serializers go.

The problem here is that the partitioner might lose access to the
deserialized key which would occasionally be useful for semantic
partitioning schemes. The Partitioner could deserialize the key but that
would be inefficient and weird.

This problem could be fixed by having key and value be byte[] but retaining
partitionKey as an Object and passing it to the partitioner as is. Then if
you have a partitioner which requires the deserialized key you would need
to use this partition key. One weird side effect is that if you want to
have a custom partition key BUT want to partition by the bytes of that key
rather than the object value you must write a customer partitioner and
serialize it yourself.

Of these I think I prefer 1A but could be convinced of 0 since that is how
it works now.

Thoughts?

-Jay


On Fri, Jan 24, 2014 at 3:30 PM, Clark Breyman <cl...@breyman.com> wrote:

> Jay - Thanks for the call for comments. Here's some initial input:
>
> - Make message serialization a client responsibility (making all messages
> byte[]). Reflection-based loading makes it harder to use generic codecs
> (e.g.  Envelope<PREFIX, DATA, SUFFIX>) or build up codec programmatically.
> Non-default partitioning should require an explicit partition key.
>
> - I really like the fact that it will be native Java. Please consider using
> native maven and not sbt, gradle, ivy, etc as they don't reliably play nice
> in the maven ecosystem. A jar without a well-formed pom doesn't feel like a
> real artifact. The pom's generated by sbt et al. are not well formed. Using
> maven will make builds and IDE integration much smoother.
>
> - Look at Nick Telford's dropwizard-extras package in which he defines some
> Jackson-compatible POJO's for loading configuration. Seems like your client
> migration is similar. The config objects should have constructors or
> factories that accept Map<String, String> and Properties for ease of
> migration.
>
> - Would you consider using the org.apache.kafka package for the new API
> (quibble)
>
> - Why create your own futures rather than use
> java.util.concurrent.Future<Long> or similar? Standard futures will play
> nice with other reactive libs and things like J8's ComposableFuture.
>
> Thanks again,
> C
>
>
>
> On Fri, Jan 24, 2014 at 2:46 PM, Roger Hoover <roger.hoover@gmail.com
> >wrote:
>
> > A couple comments:
> >
> > 1) Why does the config use a broker list instead of discovering the
> brokers
> > in ZooKeeper?  It doesn't match the HighLevelConsumer API.
> >
> > 2) It looks like broker connections are created on demand.  I'm wondering
> > if sometimes you might want to flush out config or network connectivity
> > issues before pushing the first message through.
> >
> > Should there also be a KafkaProducer.connect() or .open() method or
> > connectAll()?  I guess it would try to connect to all brokers in the
> > BROKER_LIST_CONFIG
> >
> > HTH,
> >
> > Roger
> >
> >
> > On Fri, Jan 24, 2014 at 11:54 AM, Jay Kreps <ja...@gmail.com> wrote:
> >
> > > As mentioned in a previous email we are working on a re-implementation
> of
> > > the producer. I would like to use this email thread to discuss the
> > details
> > > of the public API and the configuration. I would love for us to be
> > > incredibly picky about this public api now so it is as good as possible
> > and
> > > we don't need to break it in the future.
> > >
> > > The best way to get a feel for the API is actually to take a look at
> the
> > > javadoc, my hope is to get the api docs good enough so that it is
> > > self-explanatory:
> > >
> > >
> >
> http://empathybox.com/kafka-javadoc/index.html?kafka/clients/producer/KafkaProducer.html
> > >
> > > Please take a look at this API and give me any thoughts you may have!
> > >
> > > It may also be reasonable to take a look at the configs:
> > >
> > >
> >
> http://empathybox.com/kafka-javadoc/kafka/clients/producer/ProducerConfig.html
> > >
> > > The actual code is posted here:
> > > https://issues.apache.org/jira/browse/KAFKA-1227
> > >
> > > A few questions or comments to kick things off:
> > > 1. We need to make a decision on whether serialization of the user's
> key
> > > and value should be done by the user (with our api just taking byte[])
> or
> > > if we should take an object and allow the user to configure a
> Serializer
> > > class which we instantiate via reflection. We take the later approach
> in
> > > the current producer, and I have carried this through to this
> prototype.
> > > The tradeoff I see is this: taking byte[] is actually simpler, the user
> > can
> > > directly do whatever serialization they like. The complication is
> > actually
> > > partitioning. Currently partitioning is done by a similar plug-in api
> > > (Partitioner) which the user can implement and configure to override
> how
> > > partitions are assigned. If we take byte[] as input then we have no
> > access
> > > to the original object and partitioning MUST be done on the byte[].
> This
> > is
> > > fine for hash partitioning. However for various types of semantic
> > > partitioning (range partitioning, or whatever) you would want access to
> > the
> > > original object. In the current approach a producer who wishes to send
> > > byte[] they have serialized in their own code can configure the
> > > BytesSerialization we supply which is just a "no op" serialization.
> > > 2. We should obsess over naming and make sure each of the class names
> are
> > > good.
> > > 3. Jun has already pointed out that we need to include the topic and
> > > partition in the response, which is absolutely right. I haven't done
> that
> > > yet but that definitely needs to be there.
> > > 4. Currently RecordSend.await will throw an exception if the request
> > > failed. The intention here is that producer.send(message).await()
> exactly
> > > simulates a synchronous call. Guozhang has noted that this is a little
> > > annoying since the user must then catch exceptions. However if we
> remove
> > > this then if the user doesn't check for errors they won't know one has
> > > occurred, which I predict will be a common mistake.
> > > 5. Perhaps there is more we could do to make the async callbacks and
> > future
> > > we give back intuitive and easy to program against?
> > >
> > > Some background info on implementation:
> > >
> > > At a high level the primary difference in this producer is that it
> > removes
> > > the distinction between the "sync" and "async" producer. Effectively
> all
> > > requests are sent asynchronously but always return a future response
> > object
> > > that gives the offset as well as any error that may have occurred when
> > the
> > > request is complete. The batching that is done in the async producer
> only
> > > today is done whenever possible now. This means that the sync producer,
> > > under load, can get performance as good as the async producer
> > (preliminary
> > > results show the producer getting 1m messages/sec). This works similar
> to
> > > group commit in databases but with respect to the actual network
> > > transmission--any messages that arrive while a send is in progress are
> > > batched together. It is also possible to encourage batching even under
> > low
> > > load to save server resources by introducing a delay on the send to
> allow
> > > more messages to accumulate; this is done using the linger.ms config
> > (this
> > > is similar to Nagle's algorithm in TCP).
> > >
> > > This producer does all network communication asynchronously and in
> > parallel
> > > to all servers so the performance penalty for acks=-1 and waiting on
> > > replication should be much reduced. I haven't done much benchmarking on
> > > this yet, though.
> > >
> > > The high level design is described a little here, though this is now a
> > > little out of date:
> > > https://cwiki.apache.org/confluence/display/KAFKA/Client+Rewrite
> > >
> > > -Jay
> > >
> >
>

Re: New Producer Public API

Posted by Xavier Stevens <xa...@gaikai.com>.
+1 all of Clark's points above.


On Fri, Jan 24, 2014 at 3:30 PM, Clark Breyman <cl...@breyman.com> wrote:

> Jay - Thanks for the call for comments. Here's some initial input:
>
> - Make message serialization a client responsibility (making all messages
> byte[]). Reflection-based loading makes it harder to use generic codecs
> (e.g.  Envelope<PREFIX, DATA, SUFFIX>) or build up codec programmatically.
> Non-default partitioning should require an explicit partition key.
>
> - I really like the fact that it will be native Java. Please consider using
> native maven and not sbt, gradle, ivy, etc as they don't reliably play nice
> in the maven ecosystem. A jar without a well-formed pom doesn't feel like a
> real artifact. The pom's generated by sbt et al. are not well formed. Using
> maven will make builds and IDE integration much smoother.
>
> - Look at Nick Telford's dropwizard-extras package in which he defines some
> Jackson-compatible POJO's for loading configuration. Seems like your client
> migration is similar. The config objects should have constructors or
> factories that accept Map<String, String> and Properties for ease of
> migration.
>
> - Would you consider using the org.apache.kafka package for the new API
> (quibble)
>
> - Why create your own futures rather than use
> java.util.concurrent.Future<Long> or similar? Standard futures will play
> nice with other reactive libs and things like J8's ComposableFuture.
>
> Thanks again,
> C
>
>
>
> On Fri, Jan 24, 2014 at 2:46 PM, Roger Hoover <roger.hoover@gmail.com
> >wrote:
>
> > A couple comments:
> >
> > 1) Why does the config use a broker list instead of discovering the
> brokers
> > in ZooKeeper?  It doesn't match the HighLevelConsumer API.
> >
> > 2) It looks like broker connections are created on demand.  I'm wondering
> > if sometimes you might want to flush out config or network connectivity
> > issues before pushing the first message through.
> >
> > Should there also be a KafkaProducer.connect() or .open() method or
> > connectAll()?  I guess it would try to connect to all brokers in the
> > BROKER_LIST_CONFIG
> >
> > HTH,
> >
> > Roger
> >
> >
> > On Fri, Jan 24, 2014 at 11:54 AM, Jay Kreps <ja...@gmail.com> wrote:
> >
> > > As mentioned in a previous email we are working on a re-implementation
> of
> > > the producer. I would like to use this email thread to discuss the
> > details
> > > of the public API and the configuration. I would love for us to be
> > > incredibly picky about this public api now so it is as good as possible
> > and
> > > we don't need to break it in the future.
> > >
> > > The best way to get a feel for the API is actually to take a look at
> the
> > > javadoc, my hope is to get the api docs good enough so that it is
> > > self-explanatory:
> > >
> > >
> >
> http://empathybox.com/kafka-javadoc/index.html?kafka/clients/producer/KafkaProducer.html
> > >
> > > Please take a look at this API and give me any thoughts you may have!
> > >
> > > It may also be reasonable to take a look at the configs:
> > >
> > >
> >
> http://empathybox.com/kafka-javadoc/kafka/clients/producer/ProducerConfig.html
> > >
> > > The actual code is posted here:
> > > https://issues.apache.org/jira/browse/KAFKA-1227
> > >
> > > A few questions or comments to kick things off:
> > > 1. We need to make a decision on whether serialization of the user's
> key
> > > and value should be done by the user (with our api just taking byte[])
> or
> > > if we should take an object and allow the user to configure a
> Serializer
> > > class which we instantiate via reflection. We take the later approach
> in
> > > the current producer, and I have carried this through to this
> prototype.
> > > The tradeoff I see is this: taking byte[] is actually simpler, the user
> > can
> > > directly do whatever serialization they like. The complication is
> > actually
> > > partitioning. Currently partitioning is done by a similar plug-in api
> > > (Partitioner) which the user can implement and configure to override
> how
> > > partitions are assigned. If we take byte[] as input then we have no
> > access
> > > to the original object and partitioning MUST be done on the byte[].
> This
> > is
> > > fine for hash partitioning. However for various types of semantic
> > > partitioning (range partitioning, or whatever) you would want access to
> > the
> > > original object. In the current approach a producer who wishes to send
> > > byte[] they have serialized in their own code can configure the
> > > BytesSerialization we supply which is just a "no op" serialization.
> > > 2. We should obsess over naming and make sure each of the class names
> are
> > > good.
> > > 3. Jun has already pointed out that we need to include the topic and
> > > partition in the response, which is absolutely right. I haven't done
> that
> > > yet but that definitely needs to be there.
> > > 4. Currently RecordSend.await will throw an exception if the request
> > > failed. The intention here is that producer.send(message).await()
> exactly
> > > simulates a synchronous call. Guozhang has noted that this is a little
> > > annoying since the user must then catch exceptions. However if we
> remove
> > > this then if the user doesn't check for errors they won't know one has
> > > occurred, which I predict will be a common mistake.
> > > 5. Perhaps there is more we could do to make the async callbacks and
> > future
> > > we give back intuitive and easy to program against?
> > >
> > > Some background info on implementation:
> > >
> > > At a high level the primary difference in this producer is that it
> > removes
> > > the distinction between the "sync" and "async" producer. Effectively
> all
> > > requests are sent asynchronously but always return a future response
> > object
> > > that gives the offset as well as any error that may have occurred when
> > the
> > > request is complete. The batching that is done in the async producer
> only
> > > today is done whenever possible now. This means that the sync producer,
> > > under load, can get performance as good as the async producer
> > (preliminary
> > > results show the producer getting 1m messages/sec). This works similar
> to
> > > group commit in databases but with respect to the actual network
> > > transmission--any messages that arrive while a send is in progress are
> > > batched together. It is also possible to encourage batching even under
> > low
> > > load to save server resources by introducing a delay on the send to
> allow
> > > more messages to accumulate; this is done using the linger.ms config
> > (this
> > > is similar to Nagle's algorithm in TCP).
> > >
> > > This producer does all network communication asynchronously and in
> > parallel
> > > to all servers so the performance penalty for acks=-1 and waiting on
> > > replication should be much reduced. I haven't done much benchmarking on
> > > this yet, though.
> > >
> > > The high level design is described a little here, though this is now a
> > > little out of date:
> > > https://cwiki.apache.org/confluence/display/KAFKA/Client+Rewrite
> > >
> > > -Jay
> > >
> >
>

Re: New Producer Public API

Posted by Jay Kreps <ja...@gmail.com>.
Clark and all,

I thought a little bit about the serialization question. Here are the
options I see and the pros and cons I can think of. I'd love to hear
people's preferences if you have a strong one.

One important consideration is that however the producer works will also
need to be how the new consumer works (which we hope to write next). That
is if you put objects in, you should get objects out. So we need to think
through both sides.

Options:

Option 0: What is in the existing scala code and the java code I
posted--Serializer and Partitioner plugin provided by the user via config.
Partitioner has a sane default, but Serializer needs to be specified in
config.

Pros: How it works today in the scala code.
Cons: You have to adapt your serialization library of choice to our
interfaces. The reflective class loading means typo in the serializer name
give odd errors. Likewise there is little type safety--the ProducerRecord
takes Object and any type errors between the object provided and the
serializer give occurs at runtime.

Option 1: No plugins

This would mean byte[] key, byte[] value, and partitioning done by client
by passing in a partition *number* directly.

The problem with this is that it is tricky to compute the partition
correctly and probably most people won't. We could add a getCluster()
method to return the Cluster instance you should use for partitioning. But
I suspect people would be lazy and not use that and instead hard-code
partitions which would break if partitions were added or they hard coded it
wrong. In my experience 3 partitioning strategies cover like 99% of cases
so not having a default implementation for this makes the common case
harder. Left to their own devices people will use bad hash functions and
get weird results.

Option 1A: Alternatively we could partition by the key using the existing
default partitioning strategy which only uses the byte[] anyway but instead
of having a partitionKey we could have a numerical partition override and
add the getCluster() method to get the cluster metadata. That would make
custom partitioning possible but handle the common case simply.

Option 2: Partitioner plugin remains, serializers go.

The problem here is that the partitioner might lose access to the
deserialized key which would occasionally be useful for semantic
partitioning schemes. The Partitioner could deserialize the key but that
would be inefficient and weird.

This problem could be fixed by having key and value be byte[] but retaining
partitionKey as an Object and passing it to the partitioner as is. Then if
you have a partitioner which requires the deserialized key you would need
to use this partition key. One weird side effect is that if you want to
have a custom partition key BUT want to partition by the bytes of that key
rather than the object value you must write a customer partitioner and
serialize it yourself.

Of these I think I prefer 1A but could be convinced of 0 since that is how
it works now.

Thoughts?

-Jay


On Fri, Jan 24, 2014 at 3:30 PM, Clark Breyman <cl...@breyman.com> wrote:

> Jay - Thanks for the call for comments. Here's some initial input:
>
> - Make message serialization a client responsibility (making all messages
> byte[]). Reflection-based loading makes it harder to use generic codecs
> (e.g.  Envelope<PREFIX, DATA, SUFFIX>) or build up codec programmatically.
> Non-default partitioning should require an explicit partition key.
>
> - I really like the fact that it will be native Java. Please consider using
> native maven and not sbt, gradle, ivy, etc as they don't reliably play nice
> in the maven ecosystem. A jar without a well-formed pom doesn't feel like a
> real artifact. The pom's generated by sbt et al. are not well formed. Using
> maven will make builds and IDE integration much smoother.
>
> - Look at Nick Telford's dropwizard-extras package in which he defines some
> Jackson-compatible POJO's for loading configuration. Seems like your client
> migration is similar. The config objects should have constructors or
> factories that accept Map<String, String> and Properties for ease of
> migration.
>
> - Would you consider using the org.apache.kafka package for the new API
> (quibble)
>
> - Why create your own futures rather than use
> java.util.concurrent.Future<Long> or similar? Standard futures will play
> nice with other reactive libs and things like J8's ComposableFuture.
>
> Thanks again,
> C
>
>
>
> On Fri, Jan 24, 2014 at 2:46 PM, Roger Hoover <roger.hoover@gmail.com
> >wrote:
>
> > A couple comments:
> >
> > 1) Why does the config use a broker list instead of discovering the
> brokers
> > in ZooKeeper?  It doesn't match the HighLevelConsumer API.
> >
> > 2) It looks like broker connections are created on demand.  I'm wondering
> > if sometimes you might want to flush out config or network connectivity
> > issues before pushing the first message through.
> >
> > Should there also be a KafkaProducer.connect() or .open() method or
> > connectAll()?  I guess it would try to connect to all brokers in the
> > BROKER_LIST_CONFIG
> >
> > HTH,
> >
> > Roger
> >
> >
> > On Fri, Jan 24, 2014 at 11:54 AM, Jay Kreps <ja...@gmail.com> wrote:
> >
> > > As mentioned in a previous email we are working on a re-implementation
> of
> > > the producer. I would like to use this email thread to discuss the
> > details
> > > of the public API and the configuration. I would love for us to be
> > > incredibly picky about this public api now so it is as good as possible
> > and
> > > we don't need to break it in the future.
> > >
> > > The best way to get a feel for the API is actually to take a look at
> the
> > > javadoc, my hope is to get the api docs good enough so that it is
> > > self-explanatory:
> > >
> > >
> >
> http://empathybox.com/kafka-javadoc/index.html?kafka/clients/producer/KafkaProducer.html
> > >
> > > Please take a look at this API and give me any thoughts you may have!
> > >
> > > It may also be reasonable to take a look at the configs:
> > >
> > >
> >
> http://empathybox.com/kafka-javadoc/kafka/clients/producer/ProducerConfig.html
> > >
> > > The actual code is posted here:
> > > https://issues.apache.org/jira/browse/KAFKA-1227
> > >
> > > A few questions or comments to kick things off:
> > > 1. We need to make a decision on whether serialization of the user's
> key
> > > and value should be done by the user (with our api just taking byte[])
> or
> > > if we should take an object and allow the user to configure a
> Serializer
> > > class which we instantiate via reflection. We take the later approach
> in
> > > the current producer, and I have carried this through to this
> prototype.
> > > The tradeoff I see is this: taking byte[] is actually simpler, the user
> > can
> > > directly do whatever serialization they like. The complication is
> > actually
> > > partitioning. Currently partitioning is done by a similar plug-in api
> > > (Partitioner) which the user can implement and configure to override
> how
> > > partitions are assigned. If we take byte[] as input then we have no
> > access
> > > to the original object and partitioning MUST be done on the byte[].
> This
> > is
> > > fine for hash partitioning. However for various types of semantic
> > > partitioning (range partitioning, or whatever) you would want access to
> > the
> > > original object. In the current approach a producer who wishes to send
> > > byte[] they have serialized in their own code can configure the
> > > BytesSerialization we supply which is just a "no op" serialization.
> > > 2. We should obsess over naming and make sure each of the class names
> are
> > > good.
> > > 3. Jun has already pointed out that we need to include the topic and
> > > partition in the response, which is absolutely right. I haven't done
> that
> > > yet but that definitely needs to be there.
> > > 4. Currently RecordSend.await will throw an exception if the request
> > > failed. The intention here is that producer.send(message).await()
> exactly
> > > simulates a synchronous call. Guozhang has noted that this is a little
> > > annoying since the user must then catch exceptions. However if we
> remove
> > > this then if the user doesn't check for errors they won't know one has
> > > occurred, which I predict will be a common mistake.
> > > 5. Perhaps there is more we could do to make the async callbacks and
> > future
> > > we give back intuitive and easy to program against?
> > >
> > > Some background info on implementation:
> > >
> > > At a high level the primary difference in this producer is that it
> > removes
> > > the distinction between the "sync" and "async" producer. Effectively
> all
> > > requests are sent asynchronously but always return a future response
> > object
> > > that gives the offset as well as any error that may have occurred when
> > the
> > > request is complete. The batching that is done in the async producer
> only
> > > today is done whenever possible now. This means that the sync producer,
> > > under load, can get performance as good as the async producer
> > (preliminary
> > > results show the producer getting 1m messages/sec). This works similar
> to
> > > group commit in databases but with respect to the actual network
> > > transmission--any messages that arrive while a send is in progress are
> > > batched together. It is also possible to encourage batching even under
> > low
> > > load to save server resources by introducing a delay on the send to
> allow
> > > more messages to accumulate; this is done using the linger.ms config
> > (this
> > > is similar to Nagle's algorithm in TCP).
> > >
> > > This producer does all network communication asynchronously and in
> > parallel
> > > to all servers so the performance penalty for acks=-1 and waiting on
> > > replication should be much reduced. I haven't done much benchmarking on
> > > this yet, though.
> > >
> > > The high level design is described a little here, though this is now a
> > > little out of date:
> > > https://cwiki.apache.org/confluence/display/KAFKA/Client+Rewrite
> > >
> > > -Jay
> > >
> >
>

Re: New Producer Public API

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

Yeah good point. Okay I'm sold on Closable. Autoclosable would be much
better, but for now we are retaining 1.6 compatibility and I suspect the
use case of temporarily creating a producer would actually be a more rare
case.

-Jay


On Mon, Jan 27, 2014 at 9:29 AM, Clark Breyman <cl...@breyman.com> wrote:

> re: "Using package to avoid ambiguity"  - Unlike Scala, this is really
> cumbersome in Java as it doesn't support package imports or import aliases,
> so the only way to distinguish is to use the fully qualified path.
>
> re: Closable - it can throw IOException but is not required to. Same with
> AutoCloseable (are J7-specific classes and methods permitted?). Closable
> and AutoClosable are really nice in eliminating finally clauses.  What to
> do with a exception on close()? Log it, use it as a signal of environment
> health and move on. If it happens too frequently, you know your process
> environment is degraded and might need to fail over and restart.
>

Re: New Producer Public API

Posted by Clark Breyman <cl...@breyman.com>.
re: "Using package to avoid ambiguity"  - Unlike Scala, this is really
cumbersome in Java as it doesn't support package imports or import aliases,
so the only way to distinguish is to use the fully qualified path.

re: Closable - it can throw IOException but is not required to. Same with
AutoCloseable (are J7-specific classes and methods permitted?). Closable
and AutoClosable are really nice in eliminating finally clauses.  What to
do with a exception on close()? Log it, use it as a signal of environment
health and move on. If it happens too frequently, you know your process
environment is degraded and might need to fail over and restart.

Re: New Producer Public API

Posted by Jay Kreps <ja...@gmail.com>.
Just to keep all the points in a single thread, here are a few other points
brought up by others:
1. Sriram/Guozhang: Should RecordSend.await() throw an exception if one
occurred during the call. The argument for this is that it is similar to
Future and an exception is the least surprising way to say that something
went wrong. The argument against (I think) is that just checking
send.hasError might be slightly cleaner code in some cases.
2. From twitter: The producer's method is called send(), but a Sender sends
while a Producer should maybe produce. I agree with this but I think send()
makes more sense and Producer/Consumer is kind of standard messaging
terminology. Or maybe I am just used to it now.
3. From twitter: The producer should implement java.io.Closable. I would do
this but the close method throws IOException. close() methods that throw
exceptions are super irritating because what are you supposed to do then,
close again?
4. From twitter: "The name Callback is too generic, make it
CompleteCallback." I actually prefer generic names (Producer, Consumer,
etc). Hopefully the package should avoid any ambiguity?


On Sun, Jan 26, 2014 at 9:21 PM, Jay Kreps <ja...@gmail.com> wrote:

> Clark,
>
> With respect to maven it would be great to know if you see any issues with
> the gradle stuff.
>
> For serialization I would love to hear if any of the options I outlined
> seemed good to you or if you have another idea.
>
> For futures, that would be awesome to see how it would help. I agree that
> just wrapping it in Future would make the concept obvious to folks familiar
> with juc which is one advantage.
>
> -Jay
>
>
> On Sun, Jan 26, 2014 at 4:47 PM, Clark Breyman <cl...@breyman.com> wrote:
>
>> Thanks Jay. I'll see if I can put together a more complete response,
>> perhaps as separate threads so that topics don't get entangled. In the
>> mean
>> time, here's a couple responses:
>>
>> Serialization: you've broken out a sub-thread so i'll reply there. My bias
>> is that I like generics (except for type-erasure) and in particular they
>> make it easy to compose serializers for compound payloads (e.g. when a
>> common header wraps a payload of parameterized type). I'll respond to your
>> 4-options message with an example.
>>
>> Build: I've seen a lot of "maven-compatible" build systems produce
>> "artifacts" that aren't really artifacts - no embedded POM or, worst,
>> malformed POM. I know the sbt-generated artifacts were this way - onus is
>> on me to see what gradle is spitting out and what a maven build might look
>> like. Maven may be old and boring, but it gets out of the way and
>> integrates really seamlessly with a lot of IDEs. When some scala projects
>> I
>> was working on in the fall of 2011 switched from sbt to maven, build
>> became
>> a non-issue.
>>
>> Config: Not a big deal  and no, I don't think a dropwizard dependency is
>> appropriate. I do like using simple entity beans (POJO's not j2EE) for
>> configuration, especially if they can be marshalled without annotation by
>> Jackson. I only mentioned the dropwizard-extras  because it has some
>> entity
>> bean versions of the ZK and Kafka configs.
>>
>> Domain-packaging: Also not a big deal - it's what's expected and it's
>> pretty free in most IDE's. The advantages I see is that it is clear
>> whether
>> something is from the Apache Kafka project and whether something is from
>> another org and related to Kafka. That said, nothing really enforces it.
>>
>> Futures: I'll see if I can create some examples to demonstrate Future
>> making interop easier.
>>
>> Regards,
>> C
>>
>>
>>
>>
>> On Fri, Jan 24, 2014 at 4:36 PM, Jay Kreps <ja...@gmail.com> wrote:
>>
>> > Hey Clark,
>> >
>> > - Serialization: Yes I agree with these though I don't consider the
>> loss of
>> > generics a big issue. I'll try to summarize what I would consider the
>> best
>> > alternative api with raw byte[].
>> >
>> > - Maven: We had this debate a few months back and the consensus was
>> gradle.
>> > Is there a specific issue with the poms gradle makes? I am extremely
>> loath
>> > to revisit the issue as build issues are a recurring thing and no one
>> ever
>> > agrees and ultimately our build needs are very simple.
>> >
>> > - Config: I'm not sure if I follow the point. Are you saying we should
>> use
>> > something in dropwizard for config? One principle here is to try to
>> remove
>> > as many client dependencies as possible as we inevitably run into
>> terrible
>> > compatibility issues with users who use the same library or its
>> > dependencies at different versions. Or are you talking about maintaining
>> > compatibility with existing config parameters? I think as much as a
>> config
>> > in the existing client makes sense it should have the same name (I was a
>> > bit sloppy about that so I'll fix any errors there). There are a few new
>> > things and we should give those reasonable defaults. I think config is
>> > important so I'll start a thread on the config package in there.
>> >
>> > - org.apache.kafka: We could do this. I always considered it kind of an
>> odd
>> > thing Java programmers do that has no real motivation (but I could be
>> > re-educated!). I don't think it ends up reducing naming conflicts in
>> > practice and it adds a lot of noise and nested directories. Is there a
>> > reason you prefer this or just to be more standard?
>> >
>> > - Future: Basically I didn't see any particular advantage. The cancel()
>> > method doesn't really make sense so probably wouldn't work. Likewise I
>> > dislike the checked exceptions it requires. Basically I just wrote out
>> some
>> > code examples and it seemed cleaner with a special purpose object. I
>> wasn't
>> > actually aware of plans for improved futures in java 8 or the other
>> > integrations. Maybe you could elaborate on this a bit and show how it
>> would
>> > be used? Sounds promising, I just don't know a lot about it.
>> >
>> > -Jay
>> >
>> >
>> > On Fri, Jan 24, 2014 at 3:30 PM, Clark Breyman <cl...@breyman.com>
>> wrote:
>> >
>> > > Jay - Thanks for the call for comments. Here's some initial input:
>> > >
>> > > - Make message serialization a client responsibility (making all
>> messages
>> > > byte[]). Reflection-based loading makes it harder to use generic
>> codecs
>> > > (e.g.  Envelope<PREFIX, DATA, SUFFIX>) or build up codec
>> > programmatically.
>> > > Non-default partitioning should require an explicit partition key.
>> > >
>> > > - I really like the fact that it will be native Java. Please consider
>> > using
>> > > native maven and not sbt, gradle, ivy, etc as they don't reliably play
>> > nice
>> > > in the maven ecosystem. A jar without a well-formed pom doesn't feel
>> > like a
>> > > real artifact. The pom's generated by sbt et al. are not well formed.
>> > Using
>> > > maven will make builds and IDE integration much smoother.
>> > >
>> > > - Look at Nick Telford's dropwizard-extras package in which he defines
>> > some
>> > > Jackson-compatible POJO's for loading configuration. Seems like your
>> > client
>> > > migration is similar. The config objects should have constructors or
>> > > factories that accept Map<String, String> and Properties for ease of
>> > > migration.
>> > >
>> > > - Would you consider using the org.apache.kafka package for the new
>> API
>> > > (quibble)
>> > >
>> > > - Why create your own futures rather than use
>> > > java.util.concurrent.Future<Long> or similar? Standard futures will
>> play
>> > > nice with other reactive libs and things like J8's ComposableFuture.
>> > >
>> > > Thanks again,
>> > > C
>> > >
>> > >
>> > >
>> > > On Fri, Jan 24, 2014 at 2:46 PM, Roger Hoover <roger.hoover@gmail.com
>> > > >wrote:
>> > >
>> > > > A couple comments:
>> > > >
>> > > > 1) Why does the config use a broker list instead of discovering the
>> > > brokers
>> > > > in ZooKeeper?  It doesn't match the HighLevelConsumer API.
>> > > >
>> > > > 2) It looks like broker connections are created on demand.  I'm
>> > wondering
>> > > > if sometimes you might want to flush out config or network
>> connectivity
>> > > > issues before pushing the first message through.
>> > > >
>> > > > Should there also be a KafkaProducer.connect() or .open() method or
>> > > > connectAll()?  I guess it would try to connect to all brokers in the
>> > > > BROKER_LIST_CONFIG
>> > > >
>> > > > HTH,
>> > > >
>> > > > Roger
>> > > >
>> > > >
>> > > > On Fri, Jan 24, 2014 at 11:54 AM, Jay Kreps <ja...@gmail.com>
>> > wrote:
>> > > >
>> > > > > As mentioned in a previous email we are working on a
>> > re-implementation
>> > > of
>> > > > > the producer. I would like to use this email thread to discuss the
>> > > > details
>> > > > > of the public API and the configuration. I would love for us to be
>> > > > > incredibly picky about this public api now so it is as good as
>> > possible
>> > > > and
>> > > > > we don't need to break it in the future.
>> > > > >
>> > > > > The best way to get a feel for the API is actually to take a look
>> at
>> > > the
>> > > > > javadoc, my hope is to get the api docs good enough so that it is
>> > > > > self-explanatory:
>> > > > >
>> > > > >
>> > > >
>> > >
>> >
>> http://empathybox.com/kafka-javadoc/index.html?kafka/clients/producer/KafkaProducer.html
>> > > > >
>> > > > > Please take a look at this API and give me any thoughts you may
>> have!
>> > > > >
>> > > > > It may also be reasonable to take a look at the configs:
>> > > > >
>> > > > >
>> > > >
>> > >
>> >
>> http://empathybox.com/kafka-javadoc/kafka/clients/producer/ProducerConfig.html
>> > > > >
>> > > > > The actual code is posted here:
>> > > > > https://issues.apache.org/jira/browse/KAFKA-1227
>> > > > >
>> > > > > A few questions or comments to kick things off:
>> > > > > 1. We need to make a decision on whether serialization of the
>> user's
>> > > key
>> > > > > and value should be done by the user (with our api just taking
>> > byte[])
>> > > or
>> > > > > if we should take an object and allow the user to configure a
>> > > Serializer
>> > > > > class which we instantiate via reflection. We take the later
>> approach
>> > > in
>> > > > > the current producer, and I have carried this through to this
>> > > prototype.
>> > > > > The tradeoff I see is this: taking byte[] is actually simpler, the
>> > user
>> > > > can
>> > > > > directly do whatever serialization they like. The complication is
>> > > > actually
>> > > > > partitioning. Currently partitioning is done by a similar plug-in
>> api
>> > > > > (Partitioner) which the user can implement and configure to
>> override
>> > > how
>> > > > > partitions are assigned. If we take byte[] as input then we have
>> no
>> > > > access
>> > > > > to the original object and partitioning MUST be done on the
>> byte[].
>> > > This
>> > > > is
>> > > > > fine for hash partitioning. However for various types of semantic
>> > > > > partitioning (range partitioning, or whatever) you would want
>> access
>> > to
>> > > > the
>> > > > > original object. In the current approach a producer who wishes to
>> > send
>> > > > > byte[] they have serialized in their own code can configure the
>> > > > > BytesSerialization we supply which is just a "no op"
>> serialization.
>> > > > > 2. We should obsess over naming and make sure each of the class
>> names
>> > > are
>> > > > > good.
>> > > > > 3. Jun has already pointed out that we need to include the topic
>> and
>> > > > > partition in the response, which is absolutely right. I haven't
>> done
>> > > that
>> > > > > yet but that definitely needs to be there.
>> > > > > 4. Currently RecordSend.await will throw an exception if the
>> request
>> > > > > failed. The intention here is that producer.send(message).await()
>> > > exactly
>> > > > > simulates a synchronous call. Guozhang has noted that this is a
>> > little
>> > > > > annoying since the user must then catch exceptions. However if we
>> > > remove
>> > > > > this then if the user doesn't check for errors they won't know one
>> > has
>> > > > > occurred, which I predict will be a common mistake.
>> > > > > 5. Perhaps there is more we could do to make the async callbacks
>> and
>> > > > future
>> > > > > we give back intuitive and easy to program against?
>> > > > >
>> > > > > Some background info on implementation:
>> > > > >
>> > > > > At a high level the primary difference in this producer is that it
>> > > > removes
>> > > > > the distinction between the "sync" and "async" producer.
>> Effectively
>> > > all
>> > > > > requests are sent asynchronously but always return a future
>> response
>> > > > object
>> > > > > that gives the offset as well as any error that may have occurred
>> > when
>> > > > the
>> > > > > request is complete. The batching that is done in the async
>> producer
>> > > only
>> > > > > today is done whenever possible now. This means that the sync
>> > producer,
>> > > > > under load, can get performance as good as the async producer
>> > > > (preliminary
>> > > > > results show the producer getting 1m messages/sec). This works
>> > similar
>> > > to
>> > > > > group commit in databases but with respect to the actual network
>> > > > > transmission--any messages that arrive while a send is in progress
>> > are
>> > > > > batched together. It is also possible to encourage batching even
>> > under
>> > > > low
>> > > > > load to save server resources by introducing a delay on the send
>> to
>> > > allow
>> > > > > more messages to accumulate; this is done using the linger.msconfig
>> > > > (this
>> > > > > is similar to Nagle's algorithm in TCP).
>> > > > >
>> > > > > This producer does all network communication asynchronously and in
>> > > > parallel
>> > > > > to all servers so the performance penalty for acks=-1 and waiting
>> on
>> > > > > replication should be much reduced. I haven't done much
>> benchmarking
>> > on
>> > > > > this yet, though.
>> > > > >
>> > > > > The high level design is described a little here, though this is
>> now
>> > a
>> > > > > little out of date:
>> > > > > https://cwiki.apache.org/confluence/display/KAFKA/Client+Rewrite
>> > > > >
>> > > > > -Jay
>> > > > >
>> > > >
>> > >
>> >
>>
>
>

Re: New Producer Public API

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

With respect to maven it would be great to know if you see any issues with
the gradle stuff.

For serialization I would love to hear if any of the options I outlined
seemed good to you or if you have another idea.

For futures, that would be awesome to see how it would help. I agree that
just wrapping it in Future would make the concept obvious to folks familiar
with juc which is one advantage.

-Jay


On Sun, Jan 26, 2014 at 4:47 PM, Clark Breyman <cl...@breyman.com> wrote:

> Thanks Jay. I'll see if I can put together a more complete response,
> perhaps as separate threads so that topics don't get entangled. In the mean
> time, here's a couple responses:
>
> Serialization: you've broken out a sub-thread so i'll reply there. My bias
> is that I like generics (except for type-erasure) and in particular they
> make it easy to compose serializers for compound payloads (e.g. when a
> common header wraps a payload of parameterized type). I'll respond to your
> 4-options message with an example.
>
> Build: I've seen a lot of "maven-compatible" build systems produce
> "artifacts" that aren't really artifacts - no embedded POM or, worst,
> malformed POM. I know the sbt-generated artifacts were this way - onus is
> on me to see what gradle is spitting out and what a maven build might look
> like. Maven may be old and boring, but it gets out of the way and
> integrates really seamlessly with a lot of IDEs. When some scala projects I
> was working on in the fall of 2011 switched from sbt to maven, build became
> a non-issue.
>
> Config: Not a big deal  and no, I don't think a dropwizard dependency is
> appropriate. I do like using simple entity beans (POJO's not j2EE) for
> configuration, especially if they can be marshalled without annotation by
> Jackson. I only mentioned the dropwizard-extras  because it has some entity
> bean versions of the ZK and Kafka configs.
>
> Domain-packaging: Also not a big deal - it's what's expected and it's
> pretty free in most IDE's. The advantages I see is that it is clear whether
> something is from the Apache Kafka project and whether something is from
> another org and related to Kafka. That said, nothing really enforces it.
>
> Futures: I'll see if I can create some examples to demonstrate Future
> making interop easier.
>
> Regards,
> C
>
>
>
>
> On Fri, Jan 24, 2014 at 4:36 PM, Jay Kreps <ja...@gmail.com> wrote:
>
> > Hey Clark,
> >
> > - Serialization: Yes I agree with these though I don't consider the loss
> of
> > generics a big issue. I'll try to summarize what I would consider the
> best
> > alternative api with raw byte[].
> >
> > - Maven: We had this debate a few months back and the consensus was
> gradle.
> > Is there a specific issue with the poms gradle makes? I am extremely
> loath
> > to revisit the issue as build issues are a recurring thing and no one
> ever
> > agrees and ultimately our build needs are very simple.
> >
> > - Config: I'm not sure if I follow the point. Are you saying we should
> use
> > something in dropwizard for config? One principle here is to try to
> remove
> > as many client dependencies as possible as we inevitably run into
> terrible
> > compatibility issues with users who use the same library or its
> > dependencies at different versions. Or are you talking about maintaining
> > compatibility with existing config parameters? I think as much as a
> config
> > in the existing client makes sense it should have the same name (I was a
> > bit sloppy about that so I'll fix any errors there). There are a few new
> > things and we should give those reasonable defaults. I think config is
> > important so I'll start a thread on the config package in there.
> >
> > - org.apache.kafka: We could do this. I always considered it kind of an
> odd
> > thing Java programmers do that has no real motivation (but I could be
> > re-educated!). I don't think it ends up reducing naming conflicts in
> > practice and it adds a lot of noise and nested directories. Is there a
> > reason you prefer this or just to be more standard?
> >
> > - Future: Basically I didn't see any particular advantage. The cancel()
> > method doesn't really make sense so probably wouldn't work. Likewise I
> > dislike the checked exceptions it requires. Basically I just wrote out
> some
> > code examples and it seemed cleaner with a special purpose object. I
> wasn't
> > actually aware of plans for improved futures in java 8 or the other
> > integrations. Maybe you could elaborate on this a bit and show how it
> would
> > be used? Sounds promising, I just don't know a lot about it.
> >
> > -Jay
> >
> >
> > On Fri, Jan 24, 2014 at 3:30 PM, Clark Breyman <cl...@breyman.com>
> wrote:
> >
> > > Jay - Thanks for the call for comments. Here's some initial input:
> > >
> > > - Make message serialization a client responsibility (making all
> messages
> > > byte[]). Reflection-based loading makes it harder to use generic codecs
> > > (e.g.  Envelope<PREFIX, DATA, SUFFIX>) or build up codec
> > programmatically.
> > > Non-default partitioning should require an explicit partition key.
> > >
> > > - I really like the fact that it will be native Java. Please consider
> > using
> > > native maven and not sbt, gradle, ivy, etc as they don't reliably play
> > nice
> > > in the maven ecosystem. A jar without a well-formed pom doesn't feel
> > like a
> > > real artifact. The pom's generated by sbt et al. are not well formed.
> > Using
> > > maven will make builds and IDE integration much smoother.
> > >
> > > - Look at Nick Telford's dropwizard-extras package in which he defines
> > some
> > > Jackson-compatible POJO's for loading configuration. Seems like your
> > client
> > > migration is similar. The config objects should have constructors or
> > > factories that accept Map<String, String> and Properties for ease of
> > > migration.
> > >
> > > - Would you consider using the org.apache.kafka package for the new API
> > > (quibble)
> > >
> > > - Why create your own futures rather than use
> > > java.util.concurrent.Future<Long> or similar? Standard futures will
> play
> > > nice with other reactive libs and things like J8's ComposableFuture.
> > >
> > > Thanks again,
> > > C
> > >
> > >
> > >
> > > On Fri, Jan 24, 2014 at 2:46 PM, Roger Hoover <roger.hoover@gmail.com
> > > >wrote:
> > >
> > > > A couple comments:
> > > >
> > > > 1) Why does the config use a broker list instead of discovering the
> > > brokers
> > > > in ZooKeeper?  It doesn't match the HighLevelConsumer API.
> > > >
> > > > 2) It looks like broker connections are created on demand.  I'm
> > wondering
> > > > if sometimes you might want to flush out config or network
> connectivity
> > > > issues before pushing the first message through.
> > > >
> > > > Should there also be a KafkaProducer.connect() or .open() method or
> > > > connectAll()?  I guess it would try to connect to all brokers in the
> > > > BROKER_LIST_CONFIG
> > > >
> > > > HTH,
> > > >
> > > > Roger
> > > >
> > > >
> > > > On Fri, Jan 24, 2014 at 11:54 AM, Jay Kreps <ja...@gmail.com>
> > wrote:
> > > >
> > > > > As mentioned in a previous email we are working on a
> > re-implementation
> > > of
> > > > > the producer. I would like to use this email thread to discuss the
> > > > details
> > > > > of the public API and the configuration. I would love for us to be
> > > > > incredibly picky about this public api now so it is as good as
> > possible
> > > > and
> > > > > we don't need to break it in the future.
> > > > >
> > > > > The best way to get a feel for the API is actually to take a look
> at
> > > the
> > > > > javadoc, my hope is to get the api docs good enough so that it is
> > > > > self-explanatory:
> > > > >
> > > > >
> > > >
> > >
> >
> http://empathybox.com/kafka-javadoc/index.html?kafka/clients/producer/KafkaProducer.html
> > > > >
> > > > > Please take a look at this API and give me any thoughts you may
> have!
> > > > >
> > > > > It may also be reasonable to take a look at the configs:
> > > > >
> > > > >
> > > >
> > >
> >
> http://empathybox.com/kafka-javadoc/kafka/clients/producer/ProducerConfig.html
> > > > >
> > > > > The actual code is posted here:
> > > > > https://issues.apache.org/jira/browse/KAFKA-1227
> > > > >
> > > > > A few questions or comments to kick things off:
> > > > > 1. We need to make a decision on whether serialization of the
> user's
> > > key
> > > > > and value should be done by the user (with our api just taking
> > byte[])
> > > or
> > > > > if we should take an object and allow the user to configure a
> > > Serializer
> > > > > class which we instantiate via reflection. We take the later
> approach
> > > in
> > > > > the current producer, and I have carried this through to this
> > > prototype.
> > > > > The tradeoff I see is this: taking byte[] is actually simpler, the
> > user
> > > > can
> > > > > directly do whatever serialization they like. The complication is
> > > > actually
> > > > > partitioning. Currently partitioning is done by a similar plug-in
> api
> > > > > (Partitioner) which the user can implement and configure to
> override
> > > how
> > > > > partitions are assigned. If we take byte[] as input then we have no
> > > > access
> > > > > to the original object and partitioning MUST be done on the byte[].
> > > This
> > > > is
> > > > > fine for hash partitioning. However for various types of semantic
> > > > > partitioning (range partitioning, or whatever) you would want
> access
> > to
> > > > the
> > > > > original object. In the current approach a producer who wishes to
> > send
> > > > > byte[] they have serialized in their own code can configure the
> > > > > BytesSerialization we supply which is just a "no op" serialization.
> > > > > 2. We should obsess over naming and make sure each of the class
> names
> > > are
> > > > > good.
> > > > > 3. Jun has already pointed out that we need to include the topic
> and
> > > > > partition in the response, which is absolutely right. I haven't
> done
> > > that
> > > > > yet but that definitely needs to be there.
> > > > > 4. Currently RecordSend.await will throw an exception if the
> request
> > > > > failed. The intention here is that producer.send(message).await()
> > > exactly
> > > > > simulates a synchronous call. Guozhang has noted that this is a
> > little
> > > > > annoying since the user must then catch exceptions. However if we
> > > remove
> > > > > this then if the user doesn't check for errors they won't know one
> > has
> > > > > occurred, which I predict will be a common mistake.
> > > > > 5. Perhaps there is more we could do to make the async callbacks
> and
> > > > future
> > > > > we give back intuitive and easy to program against?
> > > > >
> > > > > Some background info on implementation:
> > > > >
> > > > > At a high level the primary difference in this producer is that it
> > > > removes
> > > > > the distinction between the "sync" and "async" producer.
> Effectively
> > > all
> > > > > requests are sent asynchronously but always return a future
> response
> > > > object
> > > > > that gives the offset as well as any error that may have occurred
> > when
> > > > the
> > > > > request is complete. The batching that is done in the async
> producer
> > > only
> > > > > today is done whenever possible now. This means that the sync
> > producer,
> > > > > under load, can get performance as good as the async producer
> > > > (preliminary
> > > > > results show the producer getting 1m messages/sec). This works
> > similar
> > > to
> > > > > group commit in databases but with respect to the actual network
> > > > > transmission--any messages that arrive while a send is in progress
> > are
> > > > > batched together. It is also possible to encourage batching even
> > under
> > > > low
> > > > > load to save server resources by introducing a delay on the send to
> > > allow
> > > > > more messages to accumulate; this is done using the linger.msconfig
> > > > (this
> > > > > is similar to Nagle's algorithm in TCP).
> > > > >
> > > > > This producer does all network communication asynchronously and in
> > > > parallel
> > > > > to all servers so the performance penalty for acks=-1 and waiting
> on
> > > > > replication should be much reduced. I haven't done much
> benchmarking
> > on
> > > > > this yet, though.
> > > > >
> > > > > The high level design is described a little here, though this is
> now
> > a
> > > > > little out of date:
> > > > > https://cwiki.apache.org/confluence/display/KAFKA/Client+Rewrite
> > > > >
> > > > > -Jay
> > > > >
> > > >
> > >
> >
>

Re: New Producer Public API

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

1. My thinking was that it represents the in progress sending of a record,
but I take your point. It sounds like the consensus is that it would be
better to have the return type be something like Future<RecordMetadata>
where the metadata has the offset/topic/partition and the Future has the
exception if there is one.

2. Yeah I totally agree. The question here is whether we provide constants
for the config strings or just refer people to the docs. People have
requested constants. There are some issues. It means that removing configs
is not backwards compatible. That javadoc does give the string of constant
values.

3-6. Agree, all of these need to be reconsidered more carefully.

7. Fixed.

8. Ack, actually the problem is that that should be under internals and not
in the generated javadocs at all. That is an internal class, what you
actually get is a RecordSend.

9. Yeah it's an internal class so it maps more closely to the response from
the server. The server doesn't give the offset for each record but rather
just the base offset since the offsets are guaranteed to be sequential.

10. This is a point worth discussing. I have used the scala style method()
rather than the java style getMethod()/setMethod(). My objection to getters
and setters is that they imply something about the implementation (namely
that you are getting and setting something) and somehow because of that
naming people tend to make one getter/setter per member variable instead of
exposing the higher level operations on the type. I realize this is at odds
with basically the rest of the Java code in the world. Either way let's
pick a style and use it throughout.

11. Cool.

12. Cool.

13. Cool.

-Jay

On Wed, Jan 29, 2014 at 2:41 PM, Chris Riccomini <cr...@linkedin.com>wrote:

> Hey Guys,
>
> My 2c.
>
> 1. RecordSend is a confusing name to me. Shouldn't it be
> RecordSendResponse?
> 2. Random nit: it's annoying to have the Javadoc info for the contstants
> on
> http://empathybox.com/kafka-javadoc/kafka/clients/producer/ProducerConfig.h
> tml<http://empathybox.com/kafka-javadoc/kafka/clients/producer/ProducerConfig.html>,
> but the string constant values on
> http://empathybox.com/kafka-javadoc/constant-values.html#kafka.clients.prod
> ucer.ProducerConfig.MAX_REQUEST_SIZE_CONFIG. Find myself toggling between
> the two a lot. Not sure if this can be fixed easily.
> 3. MAX_PARTITION_SIZE_CONFIG - this name is kind of confusing.
> Specifically, use of the term "partition". thought it was related to Kafka
> topic partitions, not grouping together/batching.
> 4. METADATA_FETCH_TIMEOUT_CONFIG - what happens if this timeout is
> exceeded? Do we get an exception on send()?
> 5. METADATA_REFRESH_MS_CONFIG - why is this useful? Has to do with acks=0,
> right? Worth documenting, I think.
> 6. PARTITIONER_CLASS_CONFIG - link to partitioner interface in javadocs.
> Also, missing a period.
> 7. KafkaProducer.html - send() documentation says "archive" when you mean
> achieve, I think.
> 8. No javadoc for ProduceRequestResult.
> 9. In ProduceRequestResult, I understand baseOffset to be the first offset
> of the set. Is it possible to get the last offset, as well? If I send
> messages A, B, C, D, I'm most interested in D's offset.
> 10. In ProduceRequestResult, prefer Java-bean style (getError,
> isCompleted).
> 11. At first glance, I like option 1A in your serialization list.
> 12. We should definitely not introduce a ZK dependency for bootstrapping
> broker host/ports.
> 13. No favor on the Future discussion. I really^Int.Max hate checked
> exceptions, but I also like standard interfaces. It's a wash in my book.
>
>
> Cheers,
> Chris
>
> On 1/29/14 10:34 AM, "Neha Narkhede" <ne...@gmail.com> wrote:
>
> >>> The challenge of directly exposing ProduceRequestResult is that the
> >offset
> >provided is just the base offset and there is no way to know for a
> >particular message where it was in relation to that base offset because
> >the
> >batching is transparent and non-deterministic.
> >
> >That's a good point. I need to look into the code more closely to see if
> >it
> >is possible to expose
> >something like Future<RequestResult> send(...) where RequestResult has the
> >right metadata
> >as well as helper APIs that the user would want. For example
> >
> >Future<RequestResult> messageResponse;
> >try {
> >  messageResponse = send(...)
> >} catch(InterruptedException ie) {
> >} catch(ExecutionException ee) {
> >}
> >
> >if(messageResponse.hasError())
> >  // handle error
> >else {
> >   String topic = messageResponse.topic();
> >   int partition = messageResponse.partition();
> >   long offset = messageResponse.offset();   // can this offset return the
> >absolute offset instead of just the relative offset?
> >   ...
> >}
> >
> >I could've missed some reasons why we can't do the above. I just think
> >that
> >separating the future-like functionality of RecordSend
> >from the actual response metadata could be useful while supporting Future
> >at the same time.
> >
> >Thanks,
> >Neha
> >
> >
> >
> >On Wed, Jan 29, 2014 at 10:23 AM, Tom Brown <to...@gmail.com> wrote:
> >
> >> I strongly support the user of Future. In fact, the cancel method may
> >>not
> >> be useless. Since the producer is meant to be used by N threads, it
> >>could
> >> easily get overloaded such that a produce request could not be sent
> >> immediately and had to be queued. In that case, cancelling should cause
> >>it
> >> to not actually get sent.
> >>
> >> --Tom
> >>
> >>
> >> On Wed, Jan 29, 2014 at 11:06 AM, Jay Kreps <ja...@gmail.com>
> wrote:
> >>
> >> > Hey Neha,
> >> >
> >> > Error handling in RecordSend works as in Future you will get the
> >> exception
> >> > if there is one from any of the accessor methods or await().
> >> >
> >> > The purpose of hasError was that you can write things slightly more
> >> simply
> >> > (which some people expressed preference for):
> >> >   if(send.hasError())
> >> >     // do something
> >> >   long offset = send.offset();
> >> >
> >> > Instead of the more the slightly longer:
> >> > try {
> >> >    long offset = send.offset();
> >> > } catch (KafkaException e) {
> >> >    // do something
> >> > }
> >> >
> >> >
> >> > On Wed, Jan 29, 2014 at 10:01 AM, Neha Narkhede
> >><neha.narkhede@gmail.com
> >> > >wrote:
> >> >
> >> > > Regarding the use of Futures -
> >> > >
> >> > > Agree that there are some downsides to using Futures but both
> >> approaches
> >> > > have some tradeoffs.
> >> > >
> >> > > - Standardization and usability
> >> > > Future is a widely used and understood Java API and given that the
> >> > > functionality that RecordSend hopes to provide is essentially that
> >>of
> >> > > Future, I think it makes sense to expose a widely understood public
> >>API
> >> > for
> >> > > our clients. RecordSend, on the other hand, seems to provide some
> >>APIs
> >> > that
> >> > > are very similar to that of Future, in addition to exposing a bunch
> >>of
> >> > APIs
> >> > > that belong to ProduceRequestResult. As a user, I would've really
> >> > preferred
> >> > > to deal with ProduceRequestResult directly -
> >> > > Future<ProduceRequestResult> send(...)
> >> > >
> >> > > - Error handling
> >> > > RecordSend's error handling is quite unintuitive where the user has
> >>to
> >> > > remember to invoke hasError and error, instead of just throwing the
> >> > > exception. Now there are
> >> > > some downsides regarding error handling with the Future as well,
> >>where
> >> > the
> >> > > user has to catch InterruptedException when we would never run into
> >>it.
> >> > > However, it seems like a price worth paying for supporting a
> >>standard
> >> API
> >> > > and error handling
> >> > >
> >> > > - Unused APIs
> >> > > This is a downside of using Future, where the cancel() operation
> >>would
> >> > > always return false and mean nothing. But we can mention that
> >>caveat in
> >> > our
> >> > > Java docs.
> >> > >
> >> > > To summarize, I would prefer to expose a well understood and widely
> >> > adopted
> >> > > Java API and put up with the overhead of catching one unnecessary
> >> checked
> >> > > exception, rather than wrap the useful ProduceRequestResult in a
> >>custom
> >> > > async object (RecordSend) and explain that to our many users.
> >> > >
> >> > > Thanks,
> >> > > Neha
> >> > >
> >> > >
> >> > >
> >> > >
> >> > > On Tue, Jan 28, 2014 at 8:10 PM, Jay Kreps <ja...@gmail.com>
> >> wrote:
> >> > >
> >> > > > Hey Neha,
> >> > > >
> >> > > > Can you elaborate on why you prefer using Java's Future? The
> >>downside
> >> > in
> >> > > my
> >> > > > mind is the use of the checked InterruptedException and
> >> > > ExecutionException.
> >> > > > ExecutionException is arguable, but forcing you to catch
> >> > > > InterruptedException, often in code that can't be interrupted,
> >>seems
> >> > > > perverse. It also leaves us with the cancel() method which I don't
> >> > think
> >> > > we
> >> > > > really can implement.
> >> > > >
> >> > > > Option 1A, to recap/elaborate, was the following. There is no
> >> > Serializer
> >> > > or
> >> > > > Partitioner api. We take a byte[] key and value and an optional
> >> integer
> >> > > > partition. If you specify the integer partition it will be used.
> >>If
> >> you
> >> > > do
> >> > > > not specify a key or a partition the partition will be chosen in a
> >> > round
> >> > > > robin fashion. If you specify a key but no partition we will
> >>chose a
> >> > > > partition based on a hash of the key. In order to let the user
> >>find
> >> the
> >> > > > partition we will need to given them access to the Cluster
> >>instance
> >> > > > directly from the producer.
> >> > > >
> >> > > > -Jay
> >> > > >
> >> > > >
> >> > > > On Tue, Jan 28, 2014 at 6:25 PM, Neha Narkhede <
> >> > neha.narkhede@gmail.com
> >> > > > >wrote:
> >> > > >
> >> > > > > Here are more thoughts on the public APIs -
> >> > > > >
> >> > > > > - I suggest we use java's Future instead of custom Future
> >> especially
> >> > > > since
> >> > > > > it is part of the public API
> >> > > > >
> >> > > > > - Serialization: I like the simplicity of the producer APIs with
> >> the
> >> > > > > absence of serialization where we just deal with byte arrays for
> >> keys
> >> > > and
> >> > > > > values. What I don't like about this is the performance
> >>overhead on
> >> > the
> >> > > > > Partitioner for any kind of custom partitioning based on the
> >> > > > partitionKey.
> >> > > > > Since the only purpose of partitionKey is to do custom
> >> partitioning,
> >> > > why
> >> > > > > can't we take it in directly as an integer and let the user
> >>figure
> >> > out
> >> > > > the
> >> > > > > mapping from partition_key -> partition_id using the
> >>getCluster()
> >> > API?
> >> > > > If I
> >> > > > > understand correctly, this is similar to what you suggested as
> >>part
> >> > of
> >> > > > > option 1A. I like this approach since it maintains the
> >>simplicity
> >> of
> >> > > APIs
> >> > > > > by allowing us to deal with bytes and does not compromise
> >> performance
> >> > > in
> >> > > > > the custom partitioning case.
> >> > > > >
> >> > > > > Thanks,
> >> > > > > Neha
> >> > > > >
> >> > > > >
> >> > > > >
> >> > > > > On Tue, Jan 28, 2014 at 5:42 PM, Jay Kreps <jay.kreps@gmail.com
> >
> >> > > wrote:
> >> > > > >
> >> > > > > > Hey Tom,
> >> > > > > >
> >> > > > > > That sounds cool. How did you end up handling parallel I/O if
> >>you
> >> > > wrap
> >> > > > > the
> >> > > > > > individual connections? Don't you need some selector that
> >>selects
> >> > > over
> >> > > > > all
> >> > > > > > the connections?
> >> > > > > >
> >> > > > > > -Jay
> >> > > > > >
> >> > > > > >
> >> > > > > > On Tue, Jan 28, 2014 at 2:31 PM, Tom Brown
> >><tombrown52@gmail.com
> >> >
> >> > > > wrote:
> >> > > > > >
> >> > > > > > > I implemented a 0.7 client in pure java, and its API very
> >> closely
> >> > > > > > resembled
> >> > > > > > > this. (When multiple people independently engineer the same
> >> > > solution,
> >> > > > > > it's
> >> > > > > > > probably good... right?). However, there were a few
> >> architectural
> >> > > > > > > differences with my client:
> >> > > > > > >
> >> > > > > > > 1. The basic client itself was just an asynchronous layer
> >> around
> >> > > the
> >> > > > > > > different server functions. In and of itself it had no
> >> knowledge
> >> > of
> >> > > > > > > partitions, only servers (and maintained TCP connections to
> >> > them).
> >> > > > > > >
> >> > > > > > > 2. The main producer was an additional layer that provided a
> >> > > > high-level
> >> > > > > > > interface that could batch individual messages based on
> >> > partition.
> >> > > > > > >
> >> > > > > > > 3. Knowledge of partitioning was done via an interface so
> >>that
> >> > > > > different
> >> > > > > > > strategies could be used.
> >> > > > > > >
> >> > > > > > > 4. Partitioning was done by the user, with knowledge of the
> >> > > available
> >> > > > > > > partitions provided by #3.
> >> > > > > > >
> >> > > > > > > 5. Serialization was done by the user to simplify the API.
> >> > > > > > >
> >> > > > > > > 6. Futures were used to make asynchronous emulate
> >>synchronous
> >> > > calls.
> >> > > > > > >
> >> > > > > > >
> >> > > > > > > The main benefit of this approach is flexibility. For
> >>example,
> >> > > since
> >> > > > > the
> >> > > > > > > base client was just a managed connection (and not
> >>inherently a
> >> > > > > > producer),
> >> > > > > > > it was easy to composite a produce request and an offsets
> >> request
> >> > > > > > together
> >> > > > > > > into a confirmed produce request (officially not available
> >>in
> >> > 0.7).
> >> > > > > > >
> >> > > > > > > Decoupling the basic client from partition management
> >>allowed
> >> the
> >> > > me
> >> > > > to
> >> > > > > > > implement zk discovery as a separate project so that the
> >>main
> >> > > project
> >> > > > > had
> >> > > > > > > no complex dependencies. The same was true of decoupling
> >> > > > serialization.
> >> > > > > > > It's trivial to build an optional layer that adds those
> >> features
> >> > > in,
> >> > > > > > while
> >> > > > > > > allowing access to the base APIs for those that need it.
> >> > > > > > >
> >> > > > > > > Using standard Future objects was also beneficial, since I
> >> could
> >> > > > > combine
> >> > > > > > > them with existing tools (such as guava).
> >> > > > > > >
> >> > > > > > > It may be too late to be of use, but I have been working
> >>with
> >> my
> >> > > > > > company's
> >> > > > > > > legal department to release the implementation I described
> >> above.
> >> > > If
> >> > > > > > you're
> >> > > > > > > interested in it, let me know.
> >> > > > > > >
> >> > > > > > >
> >> > > > > > > To sum up my thoughts regarding the new API, I think it's a
> >> great
> >> > > > > start.
> >> > > > > > I
> >> > > > > > > would like to see a more layered approach so I can use the
> >> parts
> >> > I
> >> > > > > want,
> >> > > > > > > and adapt the other parts as needed. I would also like to
> >>see
> >> > > > standard
> >> > > > > > > interfaces (especially Future) used where they makes sense.
> >> > > > > > >
> >> > > > > > > --Tom
> >> > > > > > >
> >> > > > > > >
> >> > > > > > >
> >> > > > > > >
> >> > > > > > >
> >> > > > > > > On Tue, Jan 28, 2014 at 1:33 PM, Roger Hoover <
> >> > > > roger.hoover@gmail.com
> >> > > > > > > >wrote:
> >> > > > > > >
> >> > > > > > > > +1 ListenableFuture: If this works similar to Deferreds in
> >> > > Twisted
> >> > > > > > Python
> >> > > > > > > > or Promised IO in Javascript, I think this is a great
> >>pattern
> >> > for
> >> > > > > > > > decoupling your callback logic from the place where the
> >> Future
> >> > is
> >> > > > > > > > generated.  You can register as many callbacks as you
> >>like,
> >> > each
> >> > > in
> >> > > > > the
> >> > > > > > > > appropriate layer of the code and have each observer get
> >> > notified
> >> > > > > when
> >> > > > > > > the
> >> > > > > > > > promised i/o is complete without any of them knowing about
> >> each
> >> > > > > other.
> >> > > > > > > >
> >> > > > > > > >
> >> > > > > > > > On Tue, Jan 28, 2014 at 11:32 AM, Jay Kreps <
> >> > jay.kreps@gmail.com
> >> > > >
> >> > > > > > wrote:
> >> > > > > > > >
> >> > > > > > > > > Hey Ross,
> >> > > > > > > > >
> >> > > > > > > > > - ListenableFuture: Interesting. That would be an
> >> alternative
> >> > > to
> >> > > > > the
> >> > > > > > > > direct
> >> > > > > > > > > callback support we provide. There could be pros to
> >>this,
> >> let
> >> > > me
> >> > > > > > think
> >> > > > > > > > > about it.
> >> > > > > > > > > - We could provide layering, but I feel that the
> >> > serialization
> >> > > is
> >> > > > > > such
> >> > > > > > > a
> >> > > > > > > > > small thing we should just make a decision and chose
> >>one,
> >> it
> >> > > > > doesn't
> >> > > > > > > seem
> >> > > > > > > > > to me to justify a whole public facing layer.
> >> > > > > > > > > - Yes, this is fairly esoteric, essentially I think it
> >>is
> >> > > fairly
> >> > > > > > > similar
> >> > > > > > > > to
> >> > > > > > > > > databases like DynamoDB that allow you to specify two
> >> > partition
> >> > > > > keys
> >> > > > > > (I
> >> > > > > > > > > think DynamoDB does this...). The reasoning is that in
> >>fact
> >> > > there
> >> > > > > are
> >> > > > > > > > > several things you can use the key field for: (1) to
> >> compute
> >> > > the
> >> > > > > > > > partition
> >> > > > > > > > > to store the data in, (2) as a unique identifier to
> >> > deduplicate
> >> > > > > that
> >> > > > > > > > > partition's records within a log. These two things are
> >> almost
> >> > > > > always
> >> > > > > > > the
> >> > > > > > > > > same, but occationally may differ when you want to group
> >> data
> >> > > in
> >> > > > a
> >> > > > > > more
> >> > > > > > > > > sophisticated way then just a hash of the primary key
> >>but
> >> > still
> >> > > > > > retain
> >> > > > > > > > the
> >> > > > > > > > > proper primary key for delivery to the consumer and log
> >> > > > compaction.
> >> > > > > > > > >
> >> > > > > > > > > -Jay
> >> > > > > > > > >
> >> > > > > > > > >
> >> > > > > > > > > On Tue, Jan 28, 2014 at 3:24 AM, Ross Black <
> >> > > > > ross.w.black@gmail.com>
> >> > > > > > > > > wrote:
> >> > > > > > > > >
> >> > > > > > > > > > Hi Jay,
> >> > > > > > > > > >
> >> > > > > > > > > > - Just to add some more info/confusion about possibly
> >> using
> >> > > > > Future
> >> > > > > > > ...
> >> > > > > > > > > >   If Kafka uses a JDK future, it plays nicely with
> >>other
> >> > > > > frameworks
> >> > > > > > > as
> >> > > > > > > > > > well.
> >> > > > > > > > > >   Google Guava has a ListenableFuture that allows
> >> callback
> >> > > > > handling
> >> > > > > > > to
> >> > > > > > > > be
> >> > > > > > > > > > added via the returned future, and allows the
> >>callbacks
> >> to
> >> > be
> >> > > > > > passed
> >> > > > > > > > off
> >> > > > > > > > > to
> >> > > > > > > > > > a specified executor.
> >> > > > > > > > > >
> >> > > > > > > > > >
> >> > > > > > > > > >
> >> > > > > > > > >
> >> > > > > > > >
> >> > > > > > >
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> >>
> http://docs.guava-libraries.googlecode.com/git-history/release/javadoc/co
> >>m/google/common/util/concurrent/ListenableFuture.html
> >> > > > > > > > > >   The JDK future can easily be converted to a
> >>listenable
> >> > > > future.
> >> > > > > > > > > >
> >> > > > > > > > > > - On the question of byte[] vs Object, could this be
> >> solved
> >> > > by
> >> > > > > > > layering
> >> > > > > > > > > the
> >> > > > > > > > > > API?  eg. a raw producer (use byte[] and specify the
> >> > > partition
> >> > > > > > > number)
> >> > > > > > > > > and
> >> > > > > > > > > > a normal producer (use generic object and specify a
> >> > > > Partitioner)?
> >> > > > > > > > > >
> >> > > > > > > > > > - I am confused by the keys in ProducerRecord and
> >> > > Partitioner.
> >> > > > > >  What
> >> > > > > > > is
> >> > > > > > > > > the
> >> > > > > > > > > > usage for both a key and a partition key? (I am not
> >>yet
> >> > using
> >> > > > > 0.8)
> >> > > > > > > > > >
> >> > > > > > > > > >
> >> > > > > > > > > > Thanks,
> >> > > > > > > > > > Ross
> >> > > > > > > > > >
> >> > > > > > > > > >
> >> > > > > > > > > >
> >> > > > > > > > > > On 28 January 2014 05:00, Xavier Stevens <
> >> > xavier@gaikai.com>
> >> > > > > > wrote:
> >> > > > > > > > > >
> >> > > > > > > > > > > AutoCloseable would be nice for us as most of our
> >>code
> >> is
> >> > > > using
> >> > > > > > > Java
> >> > > > > > > > 7
> >> > > > > > > > > at
> >> > > > > > > > > > > this point.
> >> > > > > > > > > > >
> >> > > > > > > > > > > I like Dropwizard's configuration mapping to POJOs
> >>via
> >> > > > Jackson,
> >> > > > > > but
> >> > > > > > > > if
> >> > > > > > > > > > you
> >> > > > > > > > > > > wanted to stick with property maps I don't care
> >>enough
> >> to
> >> > > > > object.
> >> > > > > > > > > > >
> >> > > > > > > > > > > If the producer only dealt with bytes, is there a
> >>way
> >> we
> >> > > > could
> >> > > > > > > still
> >> > > > > > > > > due
> >> > > > > > > > > > > partition plugins without specifying the number
> >> > > explicitly? I
> >> > > > > > would
> >> > > > > > > > > > prefer
> >> > > > > > > > > > > to be able to pass in field(s) that would be used by
> >> the
> >> > > > > > > partitioner.
> >> > > > > > > > > > > Obviously if this wasn't possible you could always
> >> > > > deserialize
> >> > > > > > the
> >> > > > > > > > > object
> >> > > > > > > > > > > in the partitioner and grab the fields you want, but
> >> that
> >> > > > seems
> >> > > > > > > > really
> >> > > > > > > > > > > expensive to do on every message.
> >> > > > > > > > > > >
> >> > > > > > > > > > > It would also be nice to have a Java API Encoder
> >> > > constructor
> >> > > > > > taking
> >> > > > > > > > in
> >> > > > > > > > > > > VerifiableProperties. Scala understands how to
> >>handle
> >> > > "props:
> >> > > > > > > > > > > VerifiableProperties = null", but Java doesn't. So
> >>you
> >> > > don't
> >> > > > > run
> >> > > > > > > into
> >> > > > > > > > > > this
> >> > > > > > > > > > > problem until runtime.
> >> > > > > > > > > > >
> >> > > > > > > > > > >
> >> > > > > > > > > > > -Xavier
> >> > > > > > > > > > >
> >> > > > > > > > > > >
> >> > > > > > > > > > > On Mon, Jan 27, 2014 at 9:37 AM, Clark Breyman <
> >> > > > > > clark@breyman.com>
> >> > > > > > > > > > wrote:
> >> > > > > > > > > > >
> >> > > > > > > > > > > > Jay -
> >> > > > > > > > > > > >
> >> > > > > > > > > > > > Config - your explanation makes sense. I'm just so
> >> > > > accustomed
> >> > > > > > to
> >> > > > > > > > > having
> >> > > > > > > > > > > > Jackson automatically map my configuration
> >>objects to
> >> > > POJOs
> >> > > > > > that
> >> > > > > > > > I've
> >> > > > > > > > > > > > stopped using property files. They are lingua
> >>franca.
> >> > The
> >> > > > > only
> >> > > > > > > > > thought
> >> > > > > > > > > > > > might be to separate the config interface from the
> >> > > > > > implementation
> >> > > > > > > > to
> >> > > > > > > > > > > allow
> >> > > > > > > > > > > > for alternatives, but that might undermine your
> >>point
> >> > of
> >> > > > "do
> >> > > > > it
> >> > > > > > > > this
> >> > > > > > > > > > way
> >> > > > > > > > > > > so
> >> > > > > > > > > > > > that everyone can find it where they expect it".
> >> > > > > > > > > > > >
> >> > > > > > > > > > > > Serialization: Of the options, I like 1A the best,
> >> > though
> >> > > > > > > possibly
> >> > > > > > > > > with
> >> > > > > > > > > > > > either an option to specify a partition key rather
> >> than
> >> > > ID
> >> > > > > or a
> >> > > > > > > > > helper
> >> > > > > > > > > > to
> >> > > > > > > > > > > > translate an arbitrary byte[] or long into a
> >> partition
> >> > > > > number.
> >> > > > > > > > > > > >
> >> > > > > > > > > > > > Thanks
> >> > > > > > > > > > > > Clark
> >> > > > > > > > > > > >
> >> > > > > > > > > > > >
> >> > > > > > > > > > > > On Sun, Jan 26, 2014 at 9:13 PM, Jay Kreps <
> >> > > > > > jay.kreps@gmail.com>
> >> > > > > > > > > > wrote:
> >> > > > > > > > > > > >
> >> > > > > > > > > > > > > Thanks for the detailed thoughts. Let me
> >>elaborate
> >> on
> >> > > the
> >> > > > > > > config
> >> > > > > > > > > > thing.
> >> > > > > > > > > > > > >
> >> > > > > > > > > > > > > I agree that at first glance key-value strings
> >> don't
> >> > > seem
> >> > > > > > like
> >> > > > > > > a
> >> > > > > > > > > very
> >> > > > > > > > > > > > good
> >> > > > > > > > > > > > > configuration api for a client. Surely a
> >>well-typed
> >> > > > config
> >> > > > > > > class
> >> > > > > > > > > > would
> >> > > > > > > > > > > be
> >> > > > > > > > > > > > > better! I actually disagree and let me see if I
> >>can
> >> > > > > convince
> >> > > > > > > you.
> >> > > > > > > > > > > > >
> >> > > > > > > > > > > > > My reasoning has nothing to do with the api and
> >> > > > everything
> >> > > > > to
> >> > > > > > > do
> >> > > > > > > > > with
> >> > > > > > > > > > > > > operations.
> >> > > > > > > > > > > > >
> >> > > > > > > > > > > > > Clients are embedded in applications which are
> >> > > themselves
> >> > > > > > > > > configured.
> >> > > > > > > > > > > In
> >> > > > > > > > > > > > > any place that takes operations seriously the
> >> > > > configuration
> >> > > > > > for
> >> > > > > > > > > these
> >> > > > > > > > > > > > > applications will be version controlled and
> >> > maintained
> >> > > > > > through
> >> > > > > > > > some
> >> > > > > > > > > > > kind
> >> > > > > > > > > > > > of
> >> > > > > > > > > > > > > config management system. If we give a config
> >>class
> >> > > with
> >> > > > > > > getters
> >> > > > > > > > > and
> >> > > > > > > > > > > > > setters the application has to expose those
> >> > properties
> >> > > to
> >> > > > > its
> >> > > > > > > > > > > > > configuration. What invariably happens is that
> >>the
> >> > > > > > application
> >> > > > > > > > > > exposes
> >> > > > > > > > > > > > only
> >> > > > > > > > > > > > > a choice few properties that they thought they
> >> would
> >> > > > > change.
> >> > > > > > > > > > > Furthermore
> >> > > > > > > > > > > > > the application will make up a name for these
> >> configs
> >> > > > that
> >> > > > > > > seems
> >> > > > > > > > > > > > intuitive
> >> > > > > > > > > > > > > at the time in the 2 seconds the engineer spends
> >> > > thinking
> >> > > > > > about
> >> > > > > > > > it.
> >> > > > > > > > > > > > >
> >> > > > > > > > > > > > > Now consider the result of this in the large.
> >>You
> >> end
> >> > > up
> >> > > > > with
> >> > > > > > > > > dozens
> >> > > > > > > > > > or
> >> > > > > > > > > > > > > hundreds of applications that have the client
> >> > embedded.
> >> > > > > Each
> >> > > > > > > > > exposes
> >> > > > > > > > > > a
> >> > > > > > > > > > > > > different, inadequate subset of the possible
> >> configs,
> >> > > > each
> >> > > > > > with
> >> > > > > > > > > > > different
> >> > > > > > > > > > > > > names. It is a nightmare.
> >> > > > > > > > > > > > >
> >> > > > > > > > > > > > > If you use a string-string map the config system
> >> can
> >> > > > > directly
> >> > > > > > > > get a
> >> > > > > > > > > > > > bundle
> >> > > > > > > > > > > > > of config key-value pairs and put them into the
> >> > client.
> >> > > > > This
> >> > > > > > > > means
> >> > > > > > > > > > that
> >> > > > > > > > > > > > all
> >> > > > > > > > > > > > > configuration is automatically available with
> >>the
> >> > name
> >> > > > > > > documented
> >> > > > > > > > > on
> >> > > > > > > > > > > the
> >> > > > > > > > > > > > > website in every application that does this. If
> >>you
> >> > > > upgrade
> >> > > > > > to
> >> > > > > > > a
> >> > > > > > > > > new
> >> > > > > > > > > > > > kafka
> >> > > > > > > > > > > > > version with more configs those will be exposed
> >> too.
> >> > If
> >> > > > you
> >> > > > > > > > realize
> >> > > > > > > > > > > that
> >> > > > > > > > > > > > > you need to change a default you can just go
> >> through
> >> > > your
> >> > > > > > > configs
> >> > > > > > > > > and
> >> > > > > > > > > > > > > change it everywhere as it will have the same
> >>name
> >> > > > > > everywhere.
> >> > > > > > > > > > > > >
> >> > > > > > > > > > > > > -Jay
> >> > > > > > > > > > > > >
> >> > > > > > > > > > > > >
> >> > > > > > > > > > > > >
> >> > > > > > > > > > > > >
> >> > > > > > > > > > > > > On Sun, Jan 26, 2014 at 4:47 PM, Clark Breyman <
> >> > > > > > > > clark@breyman.com>
> >> > > > > > > > > > > > wrote:
> >> > > > > > > > > > > > >
> >> > > > > > > > > > > > > > Thanks Jay. I'll see if I can put together a
> >>more
> >> > > > > complete
> >> > > > > > > > > > response,
> >> > > > > > > > > > > > > > perhaps as separate threads so that topics
> >>don't
> >> > get
> >> > > > > > > entangled.
> >> > > > > > > > > In
> >> > > > > > > > > > > the
> >> > > > > > > > > > > > > mean
> >> > > > > > > > > > > > > > time, here's a couple responses:
> >> > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > Serialization: you've broken out a sub-thread
> >>so
> >> > i'll
> >> > > > > reply
> >> > > > > > > > > there.
> >> > > > > > > > > > My
> >> > > > > > > > > > > > > bias
> >> > > > > > > > > > > > > > is that I like generics (except for
> >>type-erasure)
> >> > and
> >> > > > in
> >> > > > > > > > > particular
> >> > > > > > > > > > > > they
> >> > > > > > > > > > > > > > make it easy to compose serializers for
> >>compound
> >> > > > payloads
> >> > > > > > > (e.g.
> >> > > > > > > > > > when
> >> > > > > > > > > > > a
> >> > > > > > > > > > > > > > common header wraps a payload of parameterized
> >> > type).
> >> > > > > I'll
> >> > > > > > > > > respond
> >> > > > > > > > > > to
> >> > > > > > > > > > > > > your
> >> > > > > > > > > > > > > > 4-options message with an example.
> >> > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > Build: I've seen a lot of "maven-compatible"
> >> build
> >> > > > > systems
> >> > > > > > > > > produce
> >> > > > > > > > > > > > > > "artifacts" that aren't really artifacts - no
> >> > > embedded
> >> > > > > POM
> >> > > > > > > or,
> >> > > > > > > > > > worst,
> >> > > > > > > > > > > > > > malformed POM. I know the sbt-generated
> >>artifacts
> >> > > were
> >> > > > > this
> >> > > > > > > > way -
> >> > > > > > > > > > > onus
> >> > > > > > > > > > > > is
> >> > > > > > > > > > > > > > on me to see what gradle is spitting out and
> >> what a
> >> > > > maven
> >> > > > > > > build
> >> > > > > > > > > > might
> >> > > > > > > > > > > > > look
> >> > > > > > > > > > > > > > like. Maven may be old and boring, but it gets
> >> out
> >> > of
> >> > > > the
> >> > > > > > way
> >> > > > > > > > and
> >> > > > > > > > > > > > > > integrates really seamlessly with a lot of
> >>IDEs.
> >> > When
> >> > > > > some
> >> > > > > > > > scala
> >> > > > > > > > > > > > > projects I
> >> > > > > > > > > > > > > > was working on in the fall of 2011 switched
> >>from
> >> > sbt
> >> > > to
> >> > > > > > > maven,
> >> > > > > > > > > > build
> >> > > > > > > > > > > > > became
> >> > > > > > > > > > > > > > a non-issue.
> >> > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > Config: Not a big deal  and no, I don't think
> >>a
> >> > > > > dropwizard
> >> > > > > > > > > > dependency
> >> > > > > > > > > > > > is
> >> > > > > > > > > > > > > > appropriate. I do like using simple entity
> >>beans
> >> > > > (POJO's
> >> > > > > > not
> >> > > > > > > > > j2EE)
> >> > > > > > > > > > > for
> >> > > > > > > > > > > > > > configuration, especially if they can be
> >> marshalled
> >> > > > > without
> >> > > > > > > > > > > annotation
> >> > > > > > > > > > > > by
> >> > > > > > > > > > > > > > Jackson. I only mentioned the
> >>dropwizard-extras
> >> > > >  because
> >> > > > > it
> >> > > > > > > has
> >> > > > > > > > > > some
> >> > > > > > > > > > > > > entity
> >> > > > > > > > > > > > > > bean versions of the ZK and Kafka configs.
> >> > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > Domain-packaging: Also not a big deal - it's
> >> what's
> >> > > > > > expected
> >> > > > > > > > and
> >> > > > > > > > > > it's
> >> > > > > > > > > > > > > > pretty free in most IDE's. The advantages I
> >>see
> >> is
> >> > > that
> >> > > > > it
> >> > > > > > is
> >> > > > > > > > > clear
> >> > > > > > > > > > > > > whether
> >> > > > > > > > > > > > > > something is from the Apache Kafka project and
> >> > > whether
> >> > > > > > > > something
> >> > > > > > > > > is
> >> > > > > > > > > > > > from
> >> > > > > > > > > > > > > > another org and related to Kafka. That said,
> >> > nothing
> >> > > > > really
> >> > > > > > > > > > enforces
> >> > > > > > > > > > > > it.
> >> > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > Futures: I'll see if I can create some
> >>examples
> >> to
> >> > > > > > > demonstrate
> >> > > > > > > > > > Future
> >> > > > > > > > > > > > > > making interop easier.
> >> > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > Regards,
> >> > > > > > > > > > > > > > C
> >> > > > > > > > > > > > > >
> >> > > > > > > > > > > > > >
> >> > > > > > > > > > > > > >
> >> > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > On Fri, Jan 24, 2014 at 4:36 PM, Jay Kreps <
> >> > > > > > > > jay.kreps@gmail.com>
> >> > > > > > > > > > > > wrote:
> >> > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > Hey Clark,
> >> > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > - Serialization: Yes I agree with these
> >>though
> >> I
> >> > > > don't
> >> > > > > > > > consider
> >> > > > > > > > > > the
> >> > > > > > > > > > > > > loss
> >> > > > > > > > > > > > > > of
> >> > > > > > > > > > > > > > > generics a big issue. I'll try to summarize
> >> what
> >> > I
> >> > > > > would
> >> > > > > > > > > consider
> >> > > > > > > > > > > the
> >> > > > > > > > > > > > > > best
> >> > > > > > > > > > > > > > > alternative api with raw byte[].
> >> > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > - Maven: We had this debate a few months
> >>back
> >> and
> >> > > the
> >> > > > > > > > consensus
> >> > > > > > > > > > was
> >> > > > > > > > > > > > > > gradle.
> >> > > > > > > > > > > > > > > Is there a specific issue with the poms
> >>gradle
> >> > > > makes? I
> >> > > > > > am
> >> > > > > > > > > > > extremely
> >> > > > > > > > > > > > > > loath
> >> > > > > > > > > > > > > > > to revisit the issue as build issues are a
> >> > > recurring
> >> > > > > > thing
> >> > > > > > > > and
> >> > > > > > > > > no
> >> > > > > > > > > > > one
> >> > > > > > > > > > > > > > ever
> >> > > > > > > > > > > > > > > agrees and ultimately our build needs are
> >>very
> >> > > > simple.
> >> > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > - Config: I'm not sure if I follow the
> >>point.
> >> Are
> >> > > you
> >> > > > > > > saying
> >> > > > > > > > we
> >> > > > > > > > > > > > should
> >> > > > > > > > > > > > > > use
> >> > > > > > > > > > > > > > > something in dropwizard for config? One
> >> principle
> >> > > > here
> >> > > > > is
> >> > > > > > > to
> >> > > > > > > > > try
> >> > > > > > > > > > to
> >> > > > > > > > > > > > > > remove
> >> > > > > > > > > > > > > > > as many client dependencies as possible as
> >>we
> >> > > > > inevitably
> >> > > > > > > run
> >> > > > > > > > > into
> >> > > > > > > > > > > > > > terrible
> >> > > > > > > > > > > > > > > compatibility issues with users who use the
> >> same
> >> > > > > library
> >> > > > > > or
> >> > > > > > > > its
> >> > > > > > > > > > > > > > > dependencies at different versions. Or are
> >>you
> >> > > > talking
> >> > > > > > > about
> >> > > > > > > > > > > > > maintaining
> >> > > > > > > > > > > > > > > compatibility with existing config
> >>parameters?
> >> I
> >> > > > think
> >> > > > > as
> >> > > > > > > > much
> >> > > > > > > > > > as a
> >> > > > > > > > > > > > > > config
> >> > > > > > > > > > > > > > > in the existing client makes sense it should
> >> have
> >> > > the
> >> > > > > > same
> >> > > > > > > > name
> >> > > > > > > > > > (I
> >> > > > > > > > > > > > was
> >> > > > > > > > > > > > > a
> >> > > > > > > > > > > > > > > bit sloppy about that so I'll fix any errors
> >> > > there).
> >> > > > > > There
> >> > > > > > > > are
> >> > > > > > > > > a
> >> > > > > > > > > > > few
> >> > > > > > > > > > > > > new
> >> > > > > > > > > > > > > > > things and we should give those reasonable
> >> > > defaults.
> >> > > > I
> >> > > > > > > think
> >> > > > > > > > > > config
> >> > > > > > > > > > > > is
> >> > > > > > > > > > > > > > > important so I'll start a thread on the
> >>config
> >> > > > package
> >> > > > > in
> >> > > > > > > > > there.
> >> > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > - org.apache.kafka: We could do this. I
> >>always
> >> > > > > considered
> >> > > > > > > it
> >> > > > > > > > > kind
> >> > > > > > > > > > > of
> >> > > > > > > > > > > > an
> >> > > > > > > > > > > > > > odd
> >> > > > > > > > > > > > > > > thing Java programmers do that has no real
> >> > > motivation
> >> > > > > > (but
> >> > > > > > > I
> >> > > > > > > > > > could
> >> > > > > > > > > > > be
> >> > > > > > > > > > > > > > > re-educated!). I don't think it ends up
> >> reducing
> >> > > > naming
> >> > > > > > > > > conflicts
> >> > > > > > > > > > > in
> >> > > > > > > > > > > > > > > practice and it adds a lot of noise and
> >>nested
> >> > > > > > directories.
> >> > > > > > > > Is
> >> > > > > > > > > > > there
> >> > > > > > > > > > > > a
> >> > > > > > > > > > > > > > > reason you prefer this or just to be more
> >> > standard?
> >> > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > - Future: Basically I didn't see any
> >>particular
> >> > > > > > advantage.
> >> > > > > > > > The
> >> > > > > > > > > > > > cancel()
> >> > > > > > > > > > > > > > > method doesn't really make sense so probably
> >> > > wouldn't
> >> > > > > > work.
> >> > > > > > > > > > > Likewise
> >> > > > > > > > > > > > I
> >> > > > > > > > > > > > > > > dislike the checked exceptions it requires.
> >> > > > Basically I
> >> > > > > > > just
> >> > > > > > > > > > wrote
> >> > > > > > > > > > > > out
> >> > > > > > > > > > > > > > some
> >> > > > > > > > > > > > > > > code examples and it seemed cleaner with a
> >> > special
> >> > > > > > purpose
> >> > > > > > > > > > object.
> >> > > > > > > > > > > I
> >> > > > > > > > > > > > > > wasn't
> >> > > > > > > > > > > > > > > actually aware of plans for improved
> >>futures in
> >> > > java
> >> > > > 8
> >> > > > > or
> >> > > > > > > the
> >> > > > > > > > > > other
> >> > > > > > > > > > > > > > > integrations. Maybe you could elaborate on
> >> this a
> >> > > bit
> >> > > > > and
> >> > > > > > > > show
> >> > > > > > > > > > how
> >> > > > > > > > > > > it
> >> > > > > > > > > > > > > > would
> >> > > > > > > > > > > > > > > be used? Sounds promising, I just don't
> >>know a
> >> > lot
> >> > > > > about
> >> > > > > > > it.
> >> > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > -Jay
> >> > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > On Fri, Jan 24, 2014 at 3:30 PM, Clark
> >>Breyman
> >> <
> >> > > > > > > > > > clark@breyman.com>
> >> > > > > > > > > > > > > > wrote:
> >> > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > Jay - Thanks for the call for comments.
> >> Here's
> >> > > some
> >> > > > > > > initial
> >> > > > > > > > > > > input:
> >> > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > - Make message serialization a client
> >> > > > responsibility
> >> > > > > > > > (making
> >> > > > > > > > > > all
> >> > > > > > > > > > > > > > messages
> >> > > > > > > > > > > > > > > > byte[]). Reflection-based loading makes it
> >> > harder
> >> > > > to
> >> > > > > > use
> >> > > > > > > > > > generic
> >> > > > > > > > > > > > > codecs
> >> > > > > > > > > > > > > > > > (e.g.  Envelope<PREFIX, DATA, SUFFIX>) or
> >> build
> >> > > up
> >> > > > > > codec
> >> > > > > > > > > > > > > > > programmatically.
> >> > > > > > > > > > > > > > > > Non-default partitioning should require an
> >> > > explicit
> >> > > > > > > > partition
> >> > > > > > > > > > > key.
> >> > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > - I really like the fact that it will be
> >> native
> >> > > > Java.
> >> > > > > > > > Please
> >> > > > > > > > > > > > consider
> >> > > > > > > > > > > > > > > using
> >> > > > > > > > > > > > > > > > native maven and not sbt, gradle, ivy,
> >>etc as
> >> > > they
> >> > > > > > don't
> >> > > > > > > > > > reliably
> >> > > > > > > > > > > > > play
> >> > > > > > > > > > > > > > > nice
> >> > > > > > > > > > > > > > > > in the maven ecosystem. A jar without a
> >> > > well-formed
> >> > > > > pom
> >> > > > > > > > > doesn't
> >> > > > > > > > > > > > feel
> >> > > > > > > > > > > > > > > like a
> >> > > > > > > > > > > > > > > > real artifact. The pom's generated by sbt
> >>et
> >> > al.
> >> > > > are
> >> > > > > > not
> >> > > > > > > > well
> >> > > > > > > > > > > > formed.
> >> > > > > > > > > > > > > > > Using
> >> > > > > > > > > > > > > > > > maven will make builds and IDE integration
> >> much
> >> > > > > > smoother.
> >> > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > - Look at Nick Telford's dropwizard-extras
> >> > > package
> >> > > > in
> >> > > > > > > which
> >> > > > > > > > > he
> >> > > > > > > > > > > > > defines
> >> > > > > > > > > > > > > > > some
> >> > > > > > > > > > > > > > > > Jackson-compatible POJO's for loading
> >> > > > configuration.
> >> > > > > > > Seems
> >> > > > > > > > > like
> >> > > > > > > > > > > > your
> >> > > > > > > > > > > > > > > client
> >> > > > > > > > > > > > > > > > migration is similar. The config objects
> >> should
> >> > > > have
> >> > > > > > > > > > constructors
> >> > > > > > > > > > > > or
> >> > > > > > > > > > > > > > > > factories that accept Map<String, String>
> >>and
> >> > > > > > Properties
> >> > > > > > > > for
> >> > > > > > > > > > ease
> >> > > > > > > > > > > > of
> >> > > > > > > > > > > > > > > > migration.
> >> > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > - Would you consider using the
> >> org.apache.kafka
> >> > > > > package
> >> > > > > > > for
> >> > > > > > > > > the
> >> > > > > > > > > > > new
> >> > > > > > > > > > > > > API
> >> > > > > > > > > > > > > > > > (quibble)
> >> > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > - Why create your own futures rather than
> >>use
> >> > > > > > > > > > > > > > > > java.util.concurrent.Future<Long> or
> >>similar?
> >> > > > > Standard
> >> > > > > > > > > futures
> >> > > > > > > > > > > will
> >> > > > > > > > > > > > > > play
> >> > > > > > > > > > > > > > > > nice with other reactive libs and things
> >>like
> >> > > J8's
> >> > > > > > > > > > > > ComposableFuture.
> >> > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > Thanks again,
> >> > > > > > > > > > > > > > > > C
> >> > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > On Fri, Jan 24, 2014 at 2:46 PM, Roger
> >> Hoover <
> >> > > > > > > > > > > > > roger.hoover@gmail.com
> >> > > > > > > > > > > > > > > > >wrote:
> >> > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > A couple comments:
> >> > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > 1) Why does the config use a broker list
> >> > > instead
> >> > > > of
> >> > > > > > > > > > discovering
> >> > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > brokers
> >> > > > > > > > > > > > > > > > > in ZooKeeper?  It doesn't match the
> >> > > > > HighLevelConsumer
> >> > > > > > > > API.
> >> > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > 2) It looks like broker connections are
> >> > created
> >> > > > on
> >> > > > > > > > demand.
> >> > > > > > > > > >  I'm
> >> > > > > > > > > > > > > > > wondering
> >> > > > > > > > > > > > > > > > > if sometimes you might want to flush out
> >> > config
> >> > > > or
> >> > > > > > > > network
> >> > > > > > > > > > > > > > connectivity
> >> > > > > > > > > > > > > > > > > issues before pushing the first message
> >> > > through.
> >> > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > Should there also be a
> >> > KafkaProducer.connect()
> >> > > or
> >> > > > > > > .open()
> >> > > > > > > > > > > method
> >> > > > > > > > > > > > or
> >> > > > > > > > > > > > > > > > > connectAll()?  I guess it would try to
> >> > connect
> >> > > to
> >> > > > > all
> >> > > > > > > > > brokers
> >> > > > > > > > > > > in
> >> > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > BROKER_LIST_CONFIG
> >> > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > HTH,
> >> > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > Roger
> >> > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > On Fri, Jan 24, 2014 at 11:54 AM, Jay
> >> Kreps <
> >> > > > > > > > > > > jay.kreps@gmail.com
> >> > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > wrote:
> >> > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > As mentioned in a previous email we
> >>are
> >> > > working
> >> > > > > on
> >> > > > > > a
> >> > > > > > > > > > > > > > > re-implementation
> >> > > > > > > > > > > > > > > > of
> >> > > > > > > > > > > > > > > > > > the producer. I would like to use this
> >> > email
> >> > > > > thread
> >> > > > > > > to
> >> > > > > > > > > > > discuss
> >> > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > details
> >> > > > > > > > > > > > > > > > > > of the public API and the
> >>configuration.
> >> I
> >> > > > would
> >> > > > > > love
> >> > > > > > > > for
> >> > > > > > > > > > us
> >> > > > > > > > > > > to
> >> > > > > > > > > > > > > be
> >> > > > > > > > > > > > > > > > > > incredibly picky about this public api
> >> now
> >> > so
> >> > > > it
> >> > > > > is
> >> > > > > > > as
> >> > > > > > > > > good
> >> > > > > > > > > > > as
> >> > > > > > > > > > > > > > > possible
> >> > > > > > > > > > > > > > > > > and
> >> > > > > > > > > > > > > > > > > > we don't need to break it in the
> >>future.
> >> > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > The best way to get a feel for the
> >>API is
> >> > > > > actually
> >> > > > > > to
> >> > > > > > > > > take
> >> > > > > > > > > > a
> >> > > > > > > > > > > > look
> >> > > > > > > > > > > > > > at
> >> > > > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > javadoc, my hope is to get the api
> >>docs
> >> > good
> >> > > > > enough
> >> > > > > > > so
> >> > > > > > > > > that
> >> > > > > > > > > > > it
> >> > > > > > > > > > > > is
> >> > > > > > > > > > > > > > > > > > self-explanatory:
> >> > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > >
> >> > > > > > > > > > > > >
> >> > > > > > > > > > > >
> >> > > > > > > > > > >
> >> > > > > > > > > >
> >> > > > > > > > >
> >> > > > > > > >
> >> > > > > > >
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> >>
> http://empathybox.com/kafka-javadoc/index.html?kafka/clients/producer/Kaf
> >>kaProducer.html
> >> > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > Please take a look at this API and
> >>give
> >> me
> >> > > any
> >> > > > > > > thoughts
> >> > > > > > > > > you
> >> > > > > > > > > > > may
> >> > > > > > > > > > > > > > have!
> >> > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > It may also be reasonable to take a
> >>look
> >> at
> >> > > the
> >> > > > > > > > configs:
> >> > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > >
> >> > > > > > > > > > > > >
> >> > > > > > > > > > > >
> >> > > > > > > > > > >
> >> > > > > > > > > >
> >> > > > > > > > >
> >> > > > > > > >
> >> > > > > > >
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> >>
> http://empathybox.com/kafka-javadoc/kafka/clients/producer/ProducerConfig
> >>.html
> >> > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > The actual code is posted here:
> >> > > > > > > > > > > > > > > > > >
> >> > > > https://issues.apache.org/jira/browse/KAFKA-1227
> >> > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > A few questions or comments to kick
> >> things
> >> > > off:
> >> > > > > > > > > > > > > > > > > > 1. We need to make a decision on
> >>whether
> >> > > > > > > serialization
> >> > > > > > > > of
> >> > > > > > > > > > the
> >> > > > > > > > > > > > > > user's
> >> > > > > > > > > > > > > > > > key
> >> > > > > > > > > > > > > > > > > > and value should be done by the user
> >> (with
> >> > > our
> >> > > > > api
> >> > > > > > > just
> >> > > > > > > > > > > taking
> >> > > > > > > > > > > > > > > byte[])
> >> > > > > > > > > > > > > > > > or
> >> > > > > > > > > > > > > > > > > > if we should take an object and allow
> >>the
> >> > > user
> >> > > > to
> >> > > > > > > > > > configure a
> >> > > > > > > > > > > > > > > > Serializer
> >> > > > > > > > > > > > > > > > > > class which we instantiate via
> >> reflection.
> >> > We
> >> > > > > take
> >> > > > > > > the
> >> > > > > > > > > > later
> >> > > > > > > > > > > > > > approach
> >> > > > > > > > > > > > > > > > in
> >> > > > > > > > > > > > > > > > > > the current producer, and I have
> >>carried
> >> > this
> >> > > > > > through
> >> > > > > > > > to
> >> > > > > > > > > > this
> >> > > > > > > > > > > > > > > > prototype.
> >> > > > > > > > > > > > > > > > > > The tradeoff I see is this: taking
> >>byte[]
> >> > is
> >> > > > > > actually
> >> > > > > > > > > > > simpler,
> >> > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > user
> >> > > > > > > > > > > > > > > > > can
> >> > > > > > > > > > > > > > > > > > directly do whatever serialization
> >>they
> >> > like.
> >> > > > The
> >> > > > > > > > > > > complication
> >> > > > > > > > > > > > is
> >> > > > > > > > > > > > > > > > > actually
> >> > > > > > > > > > > > > > > > > > partitioning. Currently partitioning
> >>is
> >> > done
> >> > > > by a
> >> > > > > > > > similar
> >> > > > > > > > > > > > plug-in
> >> > > > > > > > > > > > > > api
> >> > > > > > > > > > > > > > > > > > (Partitioner) which the user can
> >> implement
> >> > > and
> >> > > > > > > > configure
> >> > > > > > > > > to
> >> > > > > > > > > > > > > > override
> >> > > > > > > > > > > > > > > > how
> >> > > > > > > > > > > > > > > > > > partitions are assigned. If we take
> >> byte[]
> >> > as
> >> > > > > input
> >> > > > > > > > then
> >> > > > > > > > > we
> >> > > > > > > > > > > > have
> >> > > > > > > > > > > > > no
> >> > > > > > > > > > > > > > > > > access
> >> > > > > > > > > > > > > > > > > > to the original object and
> >>partitioning
> >> > MUST
> >> > > be
> >> > > > > > done
> >> > > > > > > on
> >> > > > > > > > > the
> >> > > > > > > > > > > > > byte[].
> >> > > > > > > > > > > > > > > > This
> >> > > > > > > > > > > > > > > > > is
> >> > > > > > > > > > > > > > > > > > fine for hash partitioning. However
> >>for
> >> > > various
> >> > > > > > types
> >> > > > > > > > of
> >> > > > > > > > > > > > semantic
> >> > > > > > > > > > > > > > > > > > partitioning (range partitioning, or
> >> > > whatever)
> >> > > > > you
> >> > > > > > > > would
> >> > > > > > > > > > want
> >> > > > > > > > > > > > > > access
> >> > > > > > > > > > > > > > > to
> >> > > > > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > original object. In the current
> >>approach
> >> a
> >> > > > > producer
> >> > > > > > > who
> >> > > > > > > > > > > wishes
> >> > > > > > > > > > > > to
> >> > > > > > > > > > > > > > > send
> >> > > > > > > > > > > > > > > > > > byte[] they have serialized in their
> >>own
> >> > code
> >> > > > can
> >> > > > > > > > > configure
> >> > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > BytesSerialization we supply which is
> >> just
> >> > a
> >> > > > "no
> >> > > > > > op"
> >> > > > > > > > > > > > > serialization.
> >> > > > > > > > > > > > > > > > > > 2. We should obsess over naming and
> >>make
> >> > sure
> >> > > > > each
> >> > > > > > of
> >> > > > > > > > the
> >> > > > > > > > > > > class
> >> > > > > > > > > > > > > > names
> >> > > > > > > > > > > > > > > > are
> >> > > > > > > > > > > > > > > > > > good.
> >> > > > > > > > > > > > > > > > > > 3. Jun has already pointed out that we
> >> need
> >> > > to
> >> > > > > > > include
> >> > > > > > > > > the
> >> > > > > > > > > > > > topic
> >> > > > > > > > > > > > > > and
> >> > > > > > > > > > > > > > > > > > partition in the response, which is
> >> > > absolutely
> >> > > > > > > right. I
> >> > > > > > > > > > > haven't
> >> > > > > > > > > > > > > > done
> >> > > > > > > > > > > > > > > > that
> >> > > > > > > > > > > > > > > > > > yet but that definitely needs to be
> >> there.
> >> > > > > > > > > > > > > > > > > > 4. Currently RecordSend.await will
> >>throw
> >> an
> >> > > > > > exception
> >> > > > > > > > if
> >> > > > > > > > > > the
> >> > > > > > > > > > > > > > request
> >> > > > > > > > > > > > > > > > > > failed. The intention here is that
> >> > > > > > > > > > > > producer.send(message).await()
> >> > > > > > > > > > > > > > > > exactly
> >> > > > > > > > > > > > > > > > > > simulates a synchronous call. Guozhang
> >> has
> >> > > > noted
> >> > > > > > that
> >> > > > > > > > > this
> >> > > > > > > > > > > is a
> >> > > > > > > > > > > > > > > little
> >> > > > > > > > > > > > > > > > > > annoying since the user must then
> >>catch
> >> > > > > exceptions.
> >> > > > > > > > > However
> >> > > > > > > > > > > if
> >> > > > > > > > > > > > we
> >> > > > > > > > > > > > > > > > remove
> >> > > > > > > > > > > > > > > > > > this then if the user doesn't check
> >>for
> >> > > errors
> >> > > > > they
> >> > > > > > > > won't
> >> > > > > > > > > > > know
> >> > > > > > > > > > > > > one
> >> > > > > > > > > > > > > > > has
> >> > > > > > > > > > > > > > > > > > occurred, which I predict will be a
> >> common
> >> > > > > mistake.
> >> > > > > > > > > > > > > > > > > > 5. Perhaps there is more we could do
> >>to
> >> > make
> >> > > > the
> >> > > > > > > async
> >> > > > > > > > > > > > callbacks
> >> > > > > > > > > > > > > > and
> >> > > > > > > > > > > > > > > > > future
> >> > > > > > > > > > > > > > > > > > we give back intuitive and easy to
> >> program
> >> > > > > against?
> >> > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > Some background info on
> >>implementation:
> >> > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > At a high level the primary
> >>difference in
> >> > > this
> >> > > > > > > producer
> >> > > > > > > > > is
> >> > > > > > > > > > > that
> >> > > > > > > > > > > > > it
> >> > > > > > > > > > > > > > > > > removes
> >> > > > > > > > > > > > > > > > > > the distinction between the "sync" and
> >> > > "async"
> >> > > > > > > > producer.
> >> > > > > > > > > > > > > > Effectively
> >> > > > > > > > > > > > > > > > all
> >> > > > > > > > > > > > > > > > > > requests are sent asynchronously but
> >> always
> >> > > > > return
> >> > > > > > a
> >> > > > > > > > > future
> >> > > > > > > > > > > > > > response
> >> > > > > > > > > > > > > > > > > object
> >> > > > > > > > > > > > > > > > > > that gives the offset as well as any
> >> error
> >> > > that
> >> > > > > may
> >> > > > > > > > have
> >> > > > > > > > > > > > occurred
> >> > > > > > > > > > > > > > > when
> >> > > > > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > request is complete. The batching
> >>that is
> >> > > done
> >> > > > in
> >> > > > > > the
> >> > > > > > > > > async
> >> > > > > > > > > > > > > > producer
> >> > > > > > > > > > > > > > > > only
> >> > > > > > > > > > > > > > > > > > today is done whenever possible now.
> >>This
> >> > > means
> >> > > > > > that
> >> > > > > > > > the
> >> > > > > > > > > > sync
> >> > > > > > > > > > > > > > > producer,
> >> > > > > > > > > > > > > > > > > > under load, can get performance as
> >>good
> >> as
> >> > > the
> >> > > > > > async
> >> > > > > > > > > > producer
> >> > > > > > > > > > > > > > > > > (preliminary
> >> > > > > > > > > > > > > > > > > > results show the producer getting 1m
> >> > > > > messages/sec).
> >> > > > > > > > This
> >> > > > > > > > > > > works
> >> > > > > > > > > > > > > > > similar
> >> > > > > > > > > > > > > > > > to
> >> > > > > > > > > > > > > > > > > > group commit in databases but with
> >> respect
> >> > to
> >> > > > the
> >> > > > > > > > actual
> >> > > > > > > > > > > > network
> >> > > > > > > > > > > > > > > > > > transmission--any messages that arrive
> >> > while
> >> > > a
> >> > > > > send
> >> > > > > > > is
> >> > > > > > > > in
> >> > > > > > > > > > > > > progress
> >> > > > > > > > > > > > > > > are
> >> > > > > > > > > > > > > > > > > > batched together. It is also possible
> >>to
> >> > > > > encourage
> >> > > > > > > > > batching
> >> > > > > > > > > > > > even
> >> > > > > > > > > > > > > > > under
> >> > > > > > > > > > > > > > > > > low
> >> > > > > > > > > > > > > > > > > > load to save server resources by
> >> > introducing
> >> > > a
> >> > > > > > delay
> >> > > > > > > on
> >> > > > > > > > > the
> >> > > > > > > > > > > > send
> >> > > > > > > > > > > > > to
> >> > > > > > > > > > > > > > > > allow
> >> > > > > > > > > > > > > > > > > > more messages to accumulate; this is
> >>done
> >> > > using
> >> > > > > the
> >> > > > > > > > > > > > > linger.msconfig
> >> > > > > > > > > > > > > > > > > (this
> >> > > > > > > > > > > > > > > > > > is similar to Nagle's algorithm in
> >>TCP).
> >> > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > This producer does all network
> >> > communication
> >> > > > > > > > > asynchronously
> >> > > > > > > > > > > and
> >> > > > > > > > > > > > > in
> >> > > > > > > > > > > > > > > > > parallel
> >> > > > > > > > > > > > > > > > > > to all servers so the performance
> >>penalty
> >> > for
> >> > > > > > acks=-1
> >> > > > > > > > and
> >> > > > > > > > > > > > waiting
> >> > > > > > > > > > > > > > on
> >> > > > > > > > > > > > > > > > > > replication should be much reduced. I
> >> > haven't
> >> > > > > done
> >> > > > > > > much
> >> > > > > > > > > > > > > > benchmarking
> >> > > > > > > > > > > > > > > on
> >> > > > > > > > > > > > > > > > > > this yet, though.
> >> > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > The high level design is described a
> >> little
> >> > > > here,
> >> > > > > > > > though
> >> > > > > > > > > > this
> >> > > > > > > > > > > > is
> >> > > > > > > > > > > > > > now
> >> > > > > > > > > > > > > > > a
> >> > > > > > > > > > > > > > > > > > little out of date:
> >> > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > >
> >> > > > > >
> >>https://cwiki.apache.org/confluence/display/KAFKA/Client+Rewrite
> >> > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > -Jay
> >> > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > >
> >> > > > > > > > > > > > >
> >> > > > > > > > > > > >
> >> > > > > > > > > > >
> >> > > > > > > > > >
> >> > > > > > > > >
> >> > > > > > > >
> >> > > > > > >
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
>
>

Re: New Producer Public API

Posted by Chris Riccomini <cr...@linkedin.com>.
Hey Guys,

My 2c.

1. RecordSend is a confusing name to me. Shouldn't it be
RecordSendResponse?
2. Random nit: it's annoying to have the Javadoc info for the contstants
on 
http://empathybox.com/kafka-javadoc/kafka/clients/producer/ProducerConfig.h
tml, but the string constant values on
http://empathybox.com/kafka-javadoc/constant-values.html#kafka.clients.prod
ucer.ProducerConfig.MAX_REQUEST_SIZE_CONFIG. Find myself toggling between
the two a lot. Not sure if this can be fixed easily.
3. MAX_PARTITION_SIZE_CONFIG - this name is kind of confusing.
Specifically, use of the term "partition". thought it was related to Kafka
topic partitions, not grouping together/batching.
4. METADATA_FETCH_TIMEOUT_CONFIG - what happens if this timeout is
exceeded? Do we get an exception on send()?
5. METADATA_REFRESH_MS_CONFIG - why is this useful? Has to do with acks=0,
right? Worth documenting, I think.
6. PARTITIONER_CLASS_CONFIG - link to partitioner interface in javadocs.
Also, missing a period.
7. KafkaProducer.html - send() documentation says "archive" when you mean
achieve, I think.
8. No javadoc for ProduceRequestResult.
9. In ProduceRequestResult, I understand baseOffset to be the first offset
of the set. Is it possible to get the last offset, as well? If I send
messages A, B, C, D, I'm most interested in D's offset.
10. In ProduceRequestResult, prefer Java-bean style (getError,
isCompleted).
11. At first glance, I like option 1A in your serialization list.
12. We should definitely not introduce a ZK dependency for bootstrapping
broker host/ports.
13. No favor on the Future discussion. I really^Int.Max hate checked
exceptions, but I also like standard interfaces. It's a wash in my book.


Cheers,
Chris

On 1/29/14 10:34 AM, "Neha Narkhede" <ne...@gmail.com> wrote:

>>> The challenge of directly exposing ProduceRequestResult is that the
>offset
>provided is just the base offset and there is no way to know for a
>particular message where it was in relation to that base offset because
>the
>batching is transparent and non-deterministic.
>
>That's a good point. I need to look into the code more closely to see if
>it
>is possible to expose
>something like Future<RequestResult> send(...) where RequestResult has the
>right metadata
>as well as helper APIs that the user would want. For example
>
>Future<RequestResult> messageResponse;
>try {
>  messageResponse = send(...)
>} catch(InterruptedException ie) {
>} catch(ExecutionException ee) {
>}
>
>if(messageResponse.hasError())
>  // handle error
>else {
>   String topic = messageResponse.topic();
>   int partition = messageResponse.partition();
>   long offset = messageResponse.offset();   // can this offset return the
>absolute offset instead of just the relative offset?
>   ...
>}
>
>I could've missed some reasons why we can't do the above. I just think
>that
>separating the future-like functionality of RecordSend
>from the actual response metadata could be useful while supporting Future
>at the same time.
>
>Thanks,
>Neha
>
>
>
>On Wed, Jan 29, 2014 at 10:23 AM, Tom Brown <to...@gmail.com> wrote:
>
>> I strongly support the user of Future. In fact, the cancel method may
>>not
>> be useless. Since the producer is meant to be used by N threads, it
>>could
>> easily get overloaded such that a produce request could not be sent
>> immediately and had to be queued. In that case, cancelling should cause
>>it
>> to not actually get sent.
>>
>> --Tom
>>
>>
>> On Wed, Jan 29, 2014 at 11:06 AM, Jay Kreps <ja...@gmail.com> wrote:
>>
>> > Hey Neha,
>> >
>> > Error handling in RecordSend works as in Future you will get the
>> exception
>> > if there is one from any of the accessor methods or await().
>> >
>> > The purpose of hasError was that you can write things slightly more
>> simply
>> > (which some people expressed preference for):
>> >   if(send.hasError())
>> >     // do something
>> >   long offset = send.offset();
>> >
>> > Instead of the more the slightly longer:
>> > try {
>> >    long offset = send.offset();
>> > } catch (KafkaException e) {
>> >    // do something
>> > }
>> >
>> >
>> > On Wed, Jan 29, 2014 at 10:01 AM, Neha Narkhede
>><neha.narkhede@gmail.com
>> > >wrote:
>> >
>> > > Regarding the use of Futures -
>> > >
>> > > Agree that there are some downsides to using Futures but both
>> approaches
>> > > have some tradeoffs.
>> > >
>> > > - Standardization and usability
>> > > Future is a widely used and understood Java API and given that the
>> > > functionality that RecordSend hopes to provide is essentially that
>>of
>> > > Future, I think it makes sense to expose a widely understood public
>>API
>> > for
>> > > our clients. RecordSend, on the other hand, seems to provide some
>>APIs
>> > that
>> > > are very similar to that of Future, in addition to exposing a bunch
>>of
>> > APIs
>> > > that belong to ProduceRequestResult. As a user, I would've really
>> > preferred
>> > > to deal with ProduceRequestResult directly -
>> > > Future<ProduceRequestResult> send(...)
>> > >
>> > > - Error handling
>> > > RecordSend's error handling is quite unintuitive where the user has
>>to
>> > > remember to invoke hasError and error, instead of just throwing the
>> > > exception. Now there are
>> > > some downsides regarding error handling with the Future as well,
>>where
>> > the
>> > > user has to catch InterruptedException when we would never run into
>>it.
>> > > However, it seems like a price worth paying for supporting a
>>standard
>> API
>> > > and error handling
>> > >
>> > > - Unused APIs
>> > > This is a downside of using Future, where the cancel() operation
>>would
>> > > always return false and mean nothing. But we can mention that
>>caveat in
>> > our
>> > > Java docs.
>> > >
>> > > To summarize, I would prefer to expose a well understood and widely
>> > adopted
>> > > Java API and put up with the overhead of catching one unnecessary
>> checked
>> > > exception, rather than wrap the useful ProduceRequestResult in a
>>custom
>> > > async object (RecordSend) and explain that to our many users.
>> > >
>> > > Thanks,
>> > > Neha
>> > >
>> > >
>> > >
>> > >
>> > > On Tue, Jan 28, 2014 at 8:10 PM, Jay Kreps <ja...@gmail.com>
>> wrote:
>> > >
>> > > > Hey Neha,
>> > > >
>> > > > Can you elaborate on why you prefer using Java's Future? The
>>downside
>> > in
>> > > my
>> > > > mind is the use of the checked InterruptedException and
>> > > ExecutionException.
>> > > > ExecutionException is arguable, but forcing you to catch
>> > > > InterruptedException, often in code that can't be interrupted,
>>seems
>> > > > perverse. It also leaves us with the cancel() method which I don't
>> > think
>> > > we
>> > > > really can implement.
>> > > >
>> > > > Option 1A, to recap/elaborate, was the following. There is no
>> > Serializer
>> > > or
>> > > > Partitioner api. We take a byte[] key and value and an optional
>> integer
>> > > > partition. If you specify the integer partition it will be used.
>>If
>> you
>> > > do
>> > > > not specify a key or a partition the partition will be chosen in a
>> > round
>> > > > robin fashion. If you specify a key but no partition we will
>>chose a
>> > > > partition based on a hash of the key. In order to let the user
>>find
>> the
>> > > > partition we will need to given them access to the Cluster
>>instance
>> > > > directly from the producer.
>> > > >
>> > > > -Jay
>> > > >
>> > > >
>> > > > On Tue, Jan 28, 2014 at 6:25 PM, Neha Narkhede <
>> > neha.narkhede@gmail.com
>> > > > >wrote:
>> > > >
>> > > > > Here are more thoughts on the public APIs -
>> > > > >
>> > > > > - I suggest we use java's Future instead of custom Future
>> especially
>> > > > since
>> > > > > it is part of the public API
>> > > > >
>> > > > > - Serialization: I like the simplicity of the producer APIs with
>> the
>> > > > > absence of serialization where we just deal with byte arrays for
>> keys
>> > > and
>> > > > > values. What I don't like about this is the performance
>>overhead on
>> > the
>> > > > > Partitioner for any kind of custom partitioning based on the
>> > > > partitionKey.
>> > > > > Since the only purpose of partitionKey is to do custom
>> partitioning,
>> > > why
>> > > > > can't we take it in directly as an integer and let the user
>>figure
>> > out
>> > > > the
>> > > > > mapping from partition_key -> partition_id using the
>>getCluster()
>> > API?
>> > > > If I
>> > > > > understand correctly, this is similar to what you suggested as
>>part
>> > of
>> > > > > option 1A. I like this approach since it maintains the
>>simplicity
>> of
>> > > APIs
>> > > > > by allowing us to deal with bytes and does not compromise
>> performance
>> > > in
>> > > > > the custom partitioning case.
>> > > > >
>> > > > > Thanks,
>> > > > > Neha
>> > > > >
>> > > > >
>> > > > >
>> > > > > On Tue, Jan 28, 2014 at 5:42 PM, Jay Kreps <ja...@gmail.com>
>> > > wrote:
>> > > > >
>> > > > > > Hey Tom,
>> > > > > >
>> > > > > > That sounds cool. How did you end up handling parallel I/O if
>>you
>> > > wrap
>> > > > > the
>> > > > > > individual connections? Don't you need some selector that
>>selects
>> > > over
>> > > > > all
>> > > > > > the connections?
>> > > > > >
>> > > > > > -Jay
>> > > > > >
>> > > > > >
>> > > > > > On Tue, Jan 28, 2014 at 2:31 PM, Tom Brown
>><tombrown52@gmail.com
>> >
>> > > > wrote:
>> > > > > >
>> > > > > > > I implemented a 0.7 client in pure java, and its API very
>> closely
>> > > > > > resembled
>> > > > > > > this. (When multiple people independently engineer the same
>> > > solution,
>> > > > > > it's
>> > > > > > > probably good... right?). However, there were a few
>> architectural
>> > > > > > > differences with my client:
>> > > > > > >
>> > > > > > > 1. The basic client itself was just an asynchronous layer
>> around
>> > > the
>> > > > > > > different server functions. In and of itself it had no
>> knowledge
>> > of
>> > > > > > > partitions, only servers (and maintained TCP connections to
>> > them).
>> > > > > > >
>> > > > > > > 2. The main producer was an additional layer that provided a
>> > > > high-level
>> > > > > > > interface that could batch individual messages based on
>> > partition.
>> > > > > > >
>> > > > > > > 3. Knowledge of partitioning was done via an interface so
>>that
>> > > > > different
>> > > > > > > strategies could be used.
>> > > > > > >
>> > > > > > > 4. Partitioning was done by the user, with knowledge of the
>> > > available
>> > > > > > > partitions provided by #3.
>> > > > > > >
>> > > > > > > 5. Serialization was done by the user to simplify the API.
>> > > > > > >
>> > > > > > > 6. Futures were used to make asynchronous emulate
>>synchronous
>> > > calls.
>> > > > > > >
>> > > > > > >
>> > > > > > > The main benefit of this approach is flexibility. For
>>example,
>> > > since
>> > > > > the
>> > > > > > > base client was just a managed connection (and not
>>inherently a
>> > > > > > producer),
>> > > > > > > it was easy to composite a produce request and an offsets
>> request
>> > > > > > together
>> > > > > > > into a confirmed produce request (officially not available
>>in
>> > 0.7).
>> > > > > > >
>> > > > > > > Decoupling the basic client from partition management
>>allowed
>> the
>> > > me
>> > > > to
>> > > > > > > implement zk discovery as a separate project so that the
>>main
>> > > project
>> > > > > had
>> > > > > > > no complex dependencies. The same was true of decoupling
>> > > > serialization.
>> > > > > > > It's trivial to build an optional layer that adds those
>> features
>> > > in,
>> > > > > > while
>> > > > > > > allowing access to the base APIs for those that need it.
>> > > > > > >
>> > > > > > > Using standard Future objects was also beneficial, since I
>> could
>> > > > > combine
>> > > > > > > them with existing tools (such as guava).
>> > > > > > >
>> > > > > > > It may be too late to be of use, but I have been working
>>with
>> my
>> > > > > > company's
>> > > > > > > legal department to release the implementation I described
>> above.
>> > > If
>> > > > > > you're
>> > > > > > > interested in it, let me know.
>> > > > > > >
>> > > > > > >
>> > > > > > > To sum up my thoughts regarding the new API, I think it's a
>> great
>> > > > > start.
>> > > > > > I
>> > > > > > > would like to see a more layered approach so I can use the
>> parts
>> > I
>> > > > > want,
>> > > > > > > and adapt the other parts as needed. I would also like to
>>see
>> > > > standard
>> > > > > > > interfaces (especially Future) used where they makes sense.
>> > > > > > >
>> > > > > > > --Tom
>> > > > > > >
>> > > > > > >
>> > > > > > >
>> > > > > > >
>> > > > > > >
>> > > > > > > On Tue, Jan 28, 2014 at 1:33 PM, Roger Hoover <
>> > > > roger.hoover@gmail.com
>> > > > > > > >wrote:
>> > > > > > >
>> > > > > > > > +1 ListenableFuture: If this works similar to Deferreds in
>> > > Twisted
>> > > > > > Python
>> > > > > > > > or Promised IO in Javascript, I think this is a great
>>pattern
>> > for
>> > > > > > > > decoupling your callback logic from the place where the
>> Future
>> > is
>> > > > > > > > generated.  You can register as many callbacks as you
>>like,
>> > each
>> > > in
>> > > > > the
>> > > > > > > > appropriate layer of the code and have each observer get
>> > notified
>> > > > > when
>> > > > > > > the
>> > > > > > > > promised i/o is complete without any of them knowing about
>> each
>> > > > > other.
>> > > > > > > >
>> > > > > > > >
>> > > > > > > > On Tue, Jan 28, 2014 at 11:32 AM, Jay Kreps <
>> > jay.kreps@gmail.com
>> > > >
>> > > > > > wrote:
>> > > > > > > >
>> > > > > > > > > Hey Ross,
>> > > > > > > > >
>> > > > > > > > > - ListenableFuture: Interesting. That would be an
>> alternative
>> > > to
>> > > > > the
>> > > > > > > > direct
>> > > > > > > > > callback support we provide. There could be pros to
>>this,
>> let
>> > > me
>> > > > > > think
>> > > > > > > > > about it.
>> > > > > > > > > - We could provide layering, but I feel that the
>> > serialization
>> > > is
>> > > > > > such
>> > > > > > > a
>> > > > > > > > > small thing we should just make a decision and chose
>>one,
>> it
>> > > > > doesn't
>> > > > > > > seem
>> > > > > > > > > to me to justify a whole public facing layer.
>> > > > > > > > > - Yes, this is fairly esoteric, essentially I think it
>>is
>> > > fairly
>> > > > > > > similar
>> > > > > > > > to
>> > > > > > > > > databases like DynamoDB that allow you to specify two
>> > partition
>> > > > > keys
>> > > > > > (I
>> > > > > > > > > think DynamoDB does this...). The reasoning is that in
>>fact
>> > > there
>> > > > > are
>> > > > > > > > > several things you can use the key field for: (1) to
>> compute
>> > > the
>> > > > > > > > partition
>> > > > > > > > > to store the data in, (2) as a unique identifier to
>> > deduplicate
>> > > > > that
>> > > > > > > > > partition's records within a log. These two things are
>> almost
>> > > > > always
>> > > > > > > the
>> > > > > > > > > same, but occationally may differ when you want to group
>> data
>> > > in
>> > > > a
>> > > > > > more
>> > > > > > > > > sophisticated way then just a hash of the primary key
>>but
>> > still
>> > > > > > retain
>> > > > > > > > the
>> > > > > > > > > proper primary key for delivery to the consumer and log
>> > > > compaction.
>> > > > > > > > >
>> > > > > > > > > -Jay
>> > > > > > > > >
>> > > > > > > > >
>> > > > > > > > > On Tue, Jan 28, 2014 at 3:24 AM, Ross Black <
>> > > > > ross.w.black@gmail.com>
>> > > > > > > > > wrote:
>> > > > > > > > >
>> > > > > > > > > > Hi Jay,
>> > > > > > > > > >
>> > > > > > > > > > - Just to add some more info/confusion about possibly
>> using
>> > > > > Future
>> > > > > > > ...
>> > > > > > > > > >   If Kafka uses a JDK future, it plays nicely with
>>other
>> > > > > frameworks
>> > > > > > > as
>> > > > > > > > > > well.
>> > > > > > > > > >   Google Guava has a ListenableFuture that allows
>> callback
>> > > > > handling
>> > > > > > > to
>> > > > > > > > be
>> > > > > > > > > > added via the returned future, and allows the
>>callbacks
>> to
>> > be
>> > > > > > passed
>> > > > > > > > off
>> > > > > > > > > to
>> > > > > > > > > > a specified executor.
>> > > > > > > > > >
>> > > > > > > > > >
>> > > > > > > > > >
>> > > > > > > > >
>> > > > > > > >
>> > > > > > >
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>> 
>>http://docs.guava-libraries.googlecode.com/git-history/release/javadoc/co
>>m/google/common/util/concurrent/ListenableFuture.html
>> > > > > > > > > >   The JDK future can easily be converted to a
>>listenable
>> > > > future.
>> > > > > > > > > >
>> > > > > > > > > > - On the question of byte[] vs Object, could this be
>> solved
>> > > by
>> > > > > > > layering
>> > > > > > > > > the
>> > > > > > > > > > API?  eg. a raw producer (use byte[] and specify the
>> > > partition
>> > > > > > > number)
>> > > > > > > > > and
>> > > > > > > > > > a normal producer (use generic object and specify a
>> > > > Partitioner)?
>> > > > > > > > > >
>> > > > > > > > > > - I am confused by the keys in ProducerRecord and
>> > > Partitioner.
>> > > > > >  What
>> > > > > > > is
>> > > > > > > > > the
>> > > > > > > > > > usage for both a key and a partition key? (I am not
>>yet
>> > using
>> > > > > 0.8)
>> > > > > > > > > >
>> > > > > > > > > >
>> > > > > > > > > > Thanks,
>> > > > > > > > > > Ross
>> > > > > > > > > >
>> > > > > > > > > >
>> > > > > > > > > >
>> > > > > > > > > > On 28 January 2014 05:00, Xavier Stevens <
>> > xavier@gaikai.com>
>> > > > > > wrote:
>> > > > > > > > > >
>> > > > > > > > > > > AutoCloseable would be nice for us as most of our
>>code
>> is
>> > > > using
>> > > > > > > Java
>> > > > > > > > 7
>> > > > > > > > > at
>> > > > > > > > > > > this point.
>> > > > > > > > > > >
>> > > > > > > > > > > I like Dropwizard's configuration mapping to POJOs
>>via
>> > > > Jackson,
>> > > > > > but
>> > > > > > > > if
>> > > > > > > > > > you
>> > > > > > > > > > > wanted to stick with property maps I don't care
>>enough
>> to
>> > > > > object.
>> > > > > > > > > > >
>> > > > > > > > > > > If the producer only dealt with bytes, is there a
>>way
>> we
>> > > > could
>> > > > > > > still
>> > > > > > > > > due
>> > > > > > > > > > > partition plugins without specifying the number
>> > > explicitly? I
>> > > > > > would
>> > > > > > > > > > prefer
>> > > > > > > > > > > to be able to pass in field(s) that would be used by
>> the
>> > > > > > > partitioner.
>> > > > > > > > > > > Obviously if this wasn't possible you could always
>> > > > deserialize
>> > > > > > the
>> > > > > > > > > object
>> > > > > > > > > > > in the partitioner and grab the fields you want, but
>> that
>> > > > seems
>> > > > > > > > really
>> > > > > > > > > > > expensive to do on every message.
>> > > > > > > > > > >
>> > > > > > > > > > > It would also be nice to have a Java API Encoder
>> > > constructor
>> > > > > > taking
>> > > > > > > > in
>> > > > > > > > > > > VerifiableProperties. Scala understands how to
>>handle
>> > > "props:
>> > > > > > > > > > > VerifiableProperties = null", but Java doesn't. So
>>you
>> > > don't
>> > > > > run
>> > > > > > > into
>> > > > > > > > > > this
>> > > > > > > > > > > problem until runtime.
>> > > > > > > > > > >
>> > > > > > > > > > >
>> > > > > > > > > > > -Xavier
>> > > > > > > > > > >
>> > > > > > > > > > >
>> > > > > > > > > > > On Mon, Jan 27, 2014 at 9:37 AM, Clark Breyman <
>> > > > > > clark@breyman.com>
>> > > > > > > > > > wrote:
>> > > > > > > > > > >
>> > > > > > > > > > > > Jay -
>> > > > > > > > > > > >
>> > > > > > > > > > > > Config - your explanation makes sense. I'm just so
>> > > > accustomed
>> > > > > > to
>> > > > > > > > > having
>> > > > > > > > > > > > Jackson automatically map my configuration
>>objects to
>> > > POJOs
>> > > > > > that
>> > > > > > > > I've
>> > > > > > > > > > > > stopped using property files. They are lingua
>>franca.
>> > The
>> > > > > only
>> > > > > > > > > thought
>> > > > > > > > > > > > might be to separate the config interface from the
>> > > > > > implementation
>> > > > > > > > to
>> > > > > > > > > > > allow
>> > > > > > > > > > > > for alternatives, but that might undermine your
>>point
>> > of
>> > > > "do
>> > > > > it
>> > > > > > > > this
>> > > > > > > > > > way
>> > > > > > > > > > > so
>> > > > > > > > > > > > that everyone can find it where they expect it".
>> > > > > > > > > > > >
>> > > > > > > > > > > > Serialization: Of the options, I like 1A the best,
>> > though
>> > > > > > > possibly
>> > > > > > > > > with
>> > > > > > > > > > > > either an option to specify a partition key rather
>> than
>> > > ID
>> > > > > or a
>> > > > > > > > > helper
>> > > > > > > > > > to
>> > > > > > > > > > > > translate an arbitrary byte[] or long into a
>> partition
>> > > > > number.
>> > > > > > > > > > > >
>> > > > > > > > > > > > Thanks
>> > > > > > > > > > > > Clark
>> > > > > > > > > > > >
>> > > > > > > > > > > >
>> > > > > > > > > > > > On Sun, Jan 26, 2014 at 9:13 PM, Jay Kreps <
>> > > > > > jay.kreps@gmail.com>
>> > > > > > > > > > wrote:
>> > > > > > > > > > > >
>> > > > > > > > > > > > > Thanks for the detailed thoughts. Let me
>>elaborate
>> on
>> > > the
>> > > > > > > config
>> > > > > > > > > > thing.
>> > > > > > > > > > > > >
>> > > > > > > > > > > > > I agree that at first glance key-value strings
>> don't
>> > > seem
>> > > > > > like
>> > > > > > > a
>> > > > > > > > > very
>> > > > > > > > > > > > good
>> > > > > > > > > > > > > configuration api for a client. Surely a
>>well-typed
>> > > > config
>> > > > > > > class
>> > > > > > > > > > would
>> > > > > > > > > > > be
>> > > > > > > > > > > > > better! I actually disagree and let me see if I
>>can
>> > > > > convince
>> > > > > > > you.
>> > > > > > > > > > > > >
>> > > > > > > > > > > > > My reasoning has nothing to do with the api and
>> > > > everything
>> > > > > to
>> > > > > > > do
>> > > > > > > > > with
>> > > > > > > > > > > > > operations.
>> > > > > > > > > > > > >
>> > > > > > > > > > > > > Clients are embedded in applications which are
>> > > themselves
>> > > > > > > > > configured.
>> > > > > > > > > > > In
>> > > > > > > > > > > > > any place that takes operations seriously the
>> > > > configuration
>> > > > > > for
>> > > > > > > > > these
>> > > > > > > > > > > > > applications will be version controlled and
>> > maintained
>> > > > > > through
>> > > > > > > > some
>> > > > > > > > > > > kind
>> > > > > > > > > > > > of
>> > > > > > > > > > > > > config management system. If we give a config
>>class
>> > > with
>> > > > > > > getters
>> > > > > > > > > and
>> > > > > > > > > > > > > setters the application has to expose those
>> > properties
>> > > to
>> > > > > its
>> > > > > > > > > > > > > configuration. What invariably happens is that
>>the
>> > > > > > application
>> > > > > > > > > > exposes
>> > > > > > > > > > > > only
>> > > > > > > > > > > > > a choice few properties that they thought they
>> would
>> > > > > change.
>> > > > > > > > > > > Furthermore
>> > > > > > > > > > > > > the application will make up a name for these
>> configs
>> > > > that
>> > > > > > > seems
>> > > > > > > > > > > > intuitive
>> > > > > > > > > > > > > at the time in the 2 seconds the engineer spends
>> > > thinking
>> > > > > > about
>> > > > > > > > it.
>> > > > > > > > > > > > >
>> > > > > > > > > > > > > Now consider the result of this in the large.
>>You
>> end
>> > > up
>> > > > > with
>> > > > > > > > > dozens
>> > > > > > > > > > or
>> > > > > > > > > > > > > hundreds of applications that have the client
>> > embedded.
>> > > > > Each
>> > > > > > > > > exposes
>> > > > > > > > > > a
>> > > > > > > > > > > > > different, inadequate subset of the possible
>> configs,
>> > > > each
>> > > > > > with
>> > > > > > > > > > > different
>> > > > > > > > > > > > > names. It is a nightmare.
>> > > > > > > > > > > > >
>> > > > > > > > > > > > > If you use a string-string map the config system
>> can
>> > > > > directly
>> > > > > > > > get a
>> > > > > > > > > > > > bundle
>> > > > > > > > > > > > > of config key-value pairs and put them into the
>> > client.
>> > > > > This
>> > > > > > > > means
>> > > > > > > > > > that
>> > > > > > > > > > > > all
>> > > > > > > > > > > > > configuration is automatically available with
>>the
>> > name
>> > > > > > > documented
>> > > > > > > > > on
>> > > > > > > > > > > the
>> > > > > > > > > > > > > website in every application that does this. If
>>you
>> > > > upgrade
>> > > > > > to
>> > > > > > > a
>> > > > > > > > > new
>> > > > > > > > > > > > kafka
>> > > > > > > > > > > > > version with more configs those will be exposed
>> too.
>> > If
>> > > > you
>> > > > > > > > realize
>> > > > > > > > > > > that
>> > > > > > > > > > > > > you need to change a default you can just go
>> through
>> > > your
>> > > > > > > configs
>> > > > > > > > > and
>> > > > > > > > > > > > > change it everywhere as it will have the same
>>name
>> > > > > > everywhere.
>> > > > > > > > > > > > >
>> > > > > > > > > > > > > -Jay
>> > > > > > > > > > > > >
>> > > > > > > > > > > > >
>> > > > > > > > > > > > >
>> > > > > > > > > > > > >
>> > > > > > > > > > > > > On Sun, Jan 26, 2014 at 4:47 PM, Clark Breyman <
>> > > > > > > > clark@breyman.com>
>> > > > > > > > > > > > wrote:
>> > > > > > > > > > > > >
>> > > > > > > > > > > > > > Thanks Jay. I'll see if I can put together a
>>more
>> > > > > complete
>> > > > > > > > > > response,
>> > > > > > > > > > > > > > perhaps as separate threads so that topics
>>don't
>> > get
>> > > > > > > entangled.
>> > > > > > > > > In
>> > > > > > > > > > > the
>> > > > > > > > > > > > > mean
>> > > > > > > > > > > > > > time, here's a couple responses:
>> > > > > > > > > > > > > >
>> > > > > > > > > > > > > > Serialization: you've broken out a sub-thread
>>so
>> > i'll
>> > > > > reply
>> > > > > > > > > there.
>> > > > > > > > > > My
>> > > > > > > > > > > > > bias
>> > > > > > > > > > > > > > is that I like generics (except for
>>type-erasure)
>> > and
>> > > > in
>> > > > > > > > > particular
>> > > > > > > > > > > > they
>> > > > > > > > > > > > > > make it easy to compose serializers for
>>compound
>> > > > payloads
>> > > > > > > (e.g.
>> > > > > > > > > > when
>> > > > > > > > > > > a
>> > > > > > > > > > > > > > common header wraps a payload of parameterized
>> > type).
>> > > > > I'll
>> > > > > > > > > respond
>> > > > > > > > > > to
>> > > > > > > > > > > > > your
>> > > > > > > > > > > > > > 4-options message with an example.
>> > > > > > > > > > > > > >
>> > > > > > > > > > > > > > Build: I've seen a lot of "maven-compatible"
>> build
>> > > > > systems
>> > > > > > > > > produce
>> > > > > > > > > > > > > > "artifacts" that aren't really artifacts - no
>> > > embedded
>> > > > > POM
>> > > > > > > or,
>> > > > > > > > > > worst,
>> > > > > > > > > > > > > > malformed POM. I know the sbt-generated
>>artifacts
>> > > were
>> > > > > this
>> > > > > > > > way -
>> > > > > > > > > > > onus
>> > > > > > > > > > > > is
>> > > > > > > > > > > > > > on me to see what gradle is spitting out and
>> what a
>> > > > maven
>> > > > > > > build
>> > > > > > > > > > might
>> > > > > > > > > > > > > look
>> > > > > > > > > > > > > > like. Maven may be old and boring, but it gets
>> out
>> > of
>> > > > the
>> > > > > > way
>> > > > > > > > and
>> > > > > > > > > > > > > > integrates really seamlessly with a lot of
>>IDEs.
>> > When
>> > > > > some
>> > > > > > > > scala
>> > > > > > > > > > > > > projects I
>> > > > > > > > > > > > > > was working on in the fall of 2011 switched
>>from
>> > sbt
>> > > to
>> > > > > > > maven,
>> > > > > > > > > > build
>> > > > > > > > > > > > > became
>> > > > > > > > > > > > > > a non-issue.
>> > > > > > > > > > > > > >
>> > > > > > > > > > > > > > Config: Not a big deal  and no, I don't think
>>a
>> > > > > dropwizard
>> > > > > > > > > > dependency
>> > > > > > > > > > > > is
>> > > > > > > > > > > > > > appropriate. I do like using simple entity
>>beans
>> > > > (POJO's
>> > > > > > not
>> > > > > > > > > j2EE)
>> > > > > > > > > > > for
>> > > > > > > > > > > > > > configuration, especially if they can be
>> marshalled
>> > > > > without
>> > > > > > > > > > > annotation
>> > > > > > > > > > > > by
>> > > > > > > > > > > > > > Jackson. I only mentioned the
>>dropwizard-extras
>> > > >  because
>> > > > > it
>> > > > > > > has
>> > > > > > > > > > some
>> > > > > > > > > > > > > entity
>> > > > > > > > > > > > > > bean versions of the ZK and Kafka configs.
>> > > > > > > > > > > > > >
>> > > > > > > > > > > > > > Domain-packaging: Also not a big deal - it's
>> what's
>> > > > > > expected
>> > > > > > > > and
>> > > > > > > > > > it's
>> > > > > > > > > > > > > > pretty free in most IDE's. The advantages I
>>see
>> is
>> > > that
>> > > > > it
>> > > > > > is
>> > > > > > > > > clear
>> > > > > > > > > > > > > whether
>> > > > > > > > > > > > > > something is from the Apache Kafka project and
>> > > whether
>> > > > > > > > something
>> > > > > > > > > is
>> > > > > > > > > > > > from
>> > > > > > > > > > > > > > another org and related to Kafka. That said,
>> > nothing
>> > > > > really
>> > > > > > > > > > enforces
>> > > > > > > > > > > > it.
>> > > > > > > > > > > > > >
>> > > > > > > > > > > > > > Futures: I'll see if I can create some
>>examples
>> to
>> > > > > > > demonstrate
>> > > > > > > > > > Future
>> > > > > > > > > > > > > > making interop easier.
>> > > > > > > > > > > > > >
>> > > > > > > > > > > > > > Regards,
>> > > > > > > > > > > > > > C
>> > > > > > > > > > > > > >
>> > > > > > > > > > > > > >
>> > > > > > > > > > > > > >
>> > > > > > > > > > > > > >
>> > > > > > > > > > > > > > On Fri, Jan 24, 2014 at 4:36 PM, Jay Kreps <
>> > > > > > > > jay.kreps@gmail.com>
>> > > > > > > > > > > > wrote:
>> > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > Hey Clark,
>> > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > - Serialization: Yes I agree with these
>>though
>> I
>> > > > don't
>> > > > > > > > consider
>> > > > > > > > > > the
>> > > > > > > > > > > > > loss
>> > > > > > > > > > > > > > of
>> > > > > > > > > > > > > > > generics a big issue. I'll try to summarize
>> what
>> > I
>> > > > > would
>> > > > > > > > > consider
>> > > > > > > > > > > the
>> > > > > > > > > > > > > > best
>> > > > > > > > > > > > > > > alternative api with raw byte[].
>> > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > - Maven: We had this debate a few months
>>back
>> and
>> > > the
>> > > > > > > > consensus
>> > > > > > > > > > was
>> > > > > > > > > > > > > > gradle.
>> > > > > > > > > > > > > > > Is there a specific issue with the poms
>>gradle
>> > > > makes? I
>> > > > > > am
>> > > > > > > > > > > extremely
>> > > > > > > > > > > > > > loath
>> > > > > > > > > > > > > > > to revisit the issue as build issues are a
>> > > recurring
>> > > > > > thing
>> > > > > > > > and
>> > > > > > > > > no
>> > > > > > > > > > > one
>> > > > > > > > > > > > > > ever
>> > > > > > > > > > > > > > > agrees and ultimately our build needs are
>>very
>> > > > simple.
>> > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > - Config: I'm not sure if I follow the
>>point.
>> Are
>> > > you
>> > > > > > > saying
>> > > > > > > > we
>> > > > > > > > > > > > should
>> > > > > > > > > > > > > > use
>> > > > > > > > > > > > > > > something in dropwizard for config? One
>> principle
>> > > > here
>> > > > > is
>> > > > > > > to
>> > > > > > > > > try
>> > > > > > > > > > to
>> > > > > > > > > > > > > > remove
>> > > > > > > > > > > > > > > as many client dependencies as possible as
>>we
>> > > > > inevitably
>> > > > > > > run
>> > > > > > > > > into
>> > > > > > > > > > > > > > terrible
>> > > > > > > > > > > > > > > compatibility issues with users who use the
>> same
>> > > > > library
>> > > > > > or
>> > > > > > > > its
>> > > > > > > > > > > > > > > dependencies at different versions. Or are
>>you
>> > > > talking
>> > > > > > > about
>> > > > > > > > > > > > > maintaining
>> > > > > > > > > > > > > > > compatibility with existing config
>>parameters?
>> I
>> > > > think
>> > > > > as
>> > > > > > > > much
>> > > > > > > > > > as a
>> > > > > > > > > > > > > > config
>> > > > > > > > > > > > > > > in the existing client makes sense it should
>> have
>> > > the
>> > > > > > same
>> > > > > > > > name
>> > > > > > > > > > (I
>> > > > > > > > > > > > was
>> > > > > > > > > > > > > a
>> > > > > > > > > > > > > > > bit sloppy about that so I'll fix any errors
>> > > there).
>> > > > > > There
>> > > > > > > > are
>> > > > > > > > > a
>> > > > > > > > > > > few
>> > > > > > > > > > > > > new
>> > > > > > > > > > > > > > > things and we should give those reasonable
>> > > defaults.
>> > > > I
>> > > > > > > think
>> > > > > > > > > > config
>> > > > > > > > > > > > is
>> > > > > > > > > > > > > > > important so I'll start a thread on the 
>>config
>> > > > package
>> > > > > in
>> > > > > > > > > there.
>> > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > - org.apache.kafka: We could do this. I 
>>always
>> > > > > considered
>> > > > > > > it
>> > > > > > > > > kind
>> > > > > > > > > > > of
>> > > > > > > > > > > > an
>> > > > > > > > > > > > > > odd
>> > > > > > > > > > > > > > > thing Java programmers do that has no real
>> > > motivation
>> > > > > > (but
>> > > > > > > I
>> > > > > > > > > > could
>> > > > > > > > > > > be
>> > > > > > > > > > > > > > > re-educated!). I don't think it ends up
>> reducing
>> > > > naming
>> > > > > > > > > conflicts
>> > > > > > > > > > > in
>> > > > > > > > > > > > > > > practice and it adds a lot of noise and 
>>nested
>> > > > > > directories.
>> > > > > > > > Is
>> > > > > > > > > > > there
>> > > > > > > > > > > > a
>> > > > > > > > > > > > > > > reason you prefer this or just to be more
>> > standard?
>> > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > - Future: Basically I didn't see any 
>>particular
>> > > > > > advantage.
>> > > > > > > > The
>> > > > > > > > > > > > cancel()
>> > > > > > > > > > > > > > > method doesn't really make sense so probably
>> > > wouldn't
>> > > > > > work.
>> > > > > > > > > > > Likewise
>> > > > > > > > > > > > I
>> > > > > > > > > > > > > > > dislike the checked exceptions it requires.
>> > > > Basically I
>> > > > > > > just
>> > > > > > > > > > wrote
>> > > > > > > > > > > > out
>> > > > > > > > > > > > > > some
>> > > > > > > > > > > > > > > code examples and it seemed cleaner with a
>> > special
>> > > > > > purpose
>> > > > > > > > > > object.
>> > > > > > > > > > > I
>> > > > > > > > > > > > > > wasn't
>> > > > > > > > > > > > > > > actually aware of plans for improved 
>>futures in
>> > > java
>> > > > 8
>> > > > > or
>> > > > > > > the
>> > > > > > > > > > other
>> > > > > > > > > > > > > > > integrations. Maybe you could elaborate on
>> this a
>> > > bit
>> > > > > and
>> > > > > > > > show
>> > > > > > > > > > how
>> > > > > > > > > > > it
>> > > > > > > > > > > > > > would
>> > > > > > > > > > > > > > > be used? Sounds promising, I just don't 
>>know a
>> > lot
>> > > > > about
>> > > > > > > it.
>> > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > -Jay
>> > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > On Fri, Jan 24, 2014 at 3:30 PM, Clark 
>>Breyman
>> <
>> > > > > > > > > > clark@breyman.com>
>> > > > > > > > > > > > > > wrote:
>> > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > Jay - Thanks for the call for comments.
>> Here's
>> > > some
>> > > > > > > initial
>> > > > > > > > > > > input:
>> > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > - Make message serialization a client
>> > > > responsibility
>> > > > > > > > (making
>> > > > > > > > > > all
>> > > > > > > > > > > > > > messages
>> > > > > > > > > > > > > > > > byte[]). Reflection-based loading makes it
>> > harder
>> > > > to
>> > > > > > use
>> > > > > > > > > > generic
>> > > > > > > > > > > > > codecs
>> > > > > > > > > > > > > > > > (e.g.  Envelope<PREFIX, DATA, SUFFIX>) or
>> build
>> > > up
>> > > > > > codec
>> > > > > > > > > > > > > > > programmatically.
>> > > > > > > > > > > > > > > > Non-default partitioning should require an
>> > > explicit
>> > > > > > > > partition
>> > > > > > > > > > > key.
>> > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > - I really like the fact that it will be
>> native
>> > > > Java.
>> > > > > > > > Please
>> > > > > > > > > > > > consider
>> > > > > > > > > > > > > > > using
>> > > > > > > > > > > > > > > > native maven and not sbt, gradle, ivy, 
>>etc as
>> > > they
>> > > > > > don't
>> > > > > > > > > > reliably
>> > > > > > > > > > > > > play
>> > > > > > > > > > > > > > > nice
>> > > > > > > > > > > > > > > > in the maven ecosystem. A jar without a
>> > > well-formed
>> > > > > pom
>> > > > > > > > > doesn't
>> > > > > > > > > > > > feel
>> > > > > > > > > > > > > > > like a
>> > > > > > > > > > > > > > > > real artifact. The pom's generated by sbt 
>>et
>> > al.
>> > > > are
>> > > > > > not
>> > > > > > > > well
>> > > > > > > > > > > > formed.
>> > > > > > > > > > > > > > > Using
>> > > > > > > > > > > > > > > > maven will make builds and IDE integration
>> much
>> > > > > > smoother.
>> > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > - Look at Nick Telford's dropwizard-extras
>> > > package
>> > > > in
>> > > > > > > which
>> > > > > > > > > he
>> > > > > > > > > > > > > defines
>> > > > > > > > > > > > > > > some
>> > > > > > > > > > > > > > > > Jackson-compatible POJO's for loading
>> > > > configuration.
>> > > > > > > Seems
>> > > > > > > > > like
>> > > > > > > > > > > > your
>> > > > > > > > > > > > > > > client
>> > > > > > > > > > > > > > > > migration is similar. The config objects
>> should
>> > > > have
>> > > > > > > > > > constructors
>> > > > > > > > > > > > or
>> > > > > > > > > > > > > > > > factories that accept Map<String, String> 
>>and
>> > > > > > Properties
>> > > > > > > > for
>> > > > > > > > > > ease
>> > > > > > > > > > > > of
>> > > > > > > > > > > > > > > > migration.
>> > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > - Would you consider using the
>> org.apache.kafka
>> > > > > package
>> > > > > > > for
>> > > > > > > > > the
>> > > > > > > > > > > new
>> > > > > > > > > > > > > API
>> > > > > > > > > > > > > > > > (quibble)
>> > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > - Why create your own futures rather than 
>>use
>> > > > > > > > > > > > > > > > java.util.concurrent.Future<Long> or 
>>similar?
>> > > > > Standard
>> > > > > > > > > futures
>> > > > > > > > > > > will
>> > > > > > > > > > > > > > play
>> > > > > > > > > > > > > > > > nice with other reactive libs and things 
>>like
>> > > J8's
>> > > > > > > > > > > > ComposableFuture.
>> > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > Thanks again,
>> > > > > > > > > > > > > > > > C
>> > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > On Fri, Jan 24, 2014 at 2:46 PM, Roger
>> Hoover <
>> > > > > > > > > > > > > roger.hoover@gmail.com
>> > > > > > > > > > > > > > > > >wrote:
>> > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > A couple comments:
>> > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > 1) Why does the config use a broker list
>> > > instead
>> > > > of
>> > > > > > > > > > discovering
>> > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > brokers
>> > > > > > > > > > > > > > > > > in ZooKeeper?  It doesn't match the
>> > > > > HighLevelConsumer
>> > > > > > > > API.
>> > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > 2) It looks like broker connections are
>> > created
>> > > > on
>> > > > > > > > demand.
>> > > > > > > > > >  I'm
>> > > > > > > > > > > > > > > wondering
>> > > > > > > > > > > > > > > > > if sometimes you might want to flush out
>> > config
>> > > > or
>> > > > > > > > network
>> > > > > > > > > > > > > > connectivity
>> > > > > > > > > > > > > > > > > issues before pushing the first message
>> > > through.
>> > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > Should there also be a
>> > KafkaProducer.connect()
>> > > or
>> > > > > > > .open()
>> > > > > > > > > > > method
>> > > > > > > > > > > > or
>> > > > > > > > > > > > > > > > > connectAll()?  I guess it would try to
>> > connect
>> > > to
>> > > > > all
>> > > > > > > > > brokers
>> > > > > > > > > > > in
>> > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > BROKER_LIST_CONFIG
>> > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > HTH,
>> > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > Roger
>> > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > On Fri, Jan 24, 2014 at 11:54 AM, Jay
>> Kreps <
>> > > > > > > > > > > jay.kreps@gmail.com
>> > > > > > > > > > > > >
>> > > > > > > > > > > > > > > wrote:
>> > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > As mentioned in a previous email we 
>>are
>> > > working
>> > > > > on
>> > > > > > a
>> > > > > > > > > > > > > > > re-implementation
>> > > > > > > > > > > > > > > > of
>> > > > > > > > > > > > > > > > > > the producer. I would like to use this
>> > email
>> > > > > thread
>> > > > > > > to
>> > > > > > > > > > > discuss
>> > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > details
>> > > > > > > > > > > > > > > > > > of the public API and the 
>>configuration.
>> I
>> > > > would
>> > > > > > love
>> > > > > > > > for
>> > > > > > > > > > us
>> > > > > > > > > > > to
>> > > > > > > > > > > > > be
>> > > > > > > > > > > > > > > > > > incredibly picky about this public api
>> now
>> > so
>> > > > it
>> > > > > is
>> > > > > > > as
>> > > > > > > > > good
>> > > > > > > > > > > as
>> > > > > > > > > > > > > > > possible
>> > > > > > > > > > > > > > > > > and
>> > > > > > > > > > > > > > > > > > we don't need to break it in the 
>>future.
>> > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > The best way to get a feel for the 
>>API is
>> > > > > actually
>> > > > > > to
>> > > > > > > > > take
>> > > > > > > > > > a
>> > > > > > > > > > > > look
>> > > > > > > > > > > > > > at
>> > > > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > javadoc, my hope is to get the api 
>>docs
>> > good
>> > > > > enough
>> > > > > > > so
>> > > > > > > > > that
>> > > > > > > > > > > it
>> > > > > > > > > > > > is
>> > > > > > > > > > > > > > > > > > self-explanatory:
>> > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > >
>> > > > > > > > > > > > > >
>> > > > > > > > > > > > >
>> > > > > > > > > > > >
>> > > > > > > > > > >
>> > > > > > > > > >
>> > > > > > > > >
>> > > > > > > >
>> > > > > > >
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>> 
>>http://empathybox.com/kafka-javadoc/index.html?kafka/clients/producer/Kaf
>>kaProducer.html
>> > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > Please take a look at this API and 
>>give
>> me
>> > > any
>> > > > > > > thoughts
>> > > > > > > > > you
>> > > > > > > > > > > may
>> > > > > > > > > > > > > > have!
>> > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > It may also be reasonable to take a 
>>look
>> at
>> > > the
>> > > > > > > > configs:
>> > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > >
>> > > > > > > > > > > > > >
>> > > > > > > > > > > > >
>> > > > > > > > > > > >
>> > > > > > > > > > >
>> > > > > > > > > >
>> > > > > > > > >
>> > > > > > > >
>> > > > > > >
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>> 
>>http://empathybox.com/kafka-javadoc/kafka/clients/producer/ProducerConfig
>>.html
>> > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > The actual code is posted here:
>> > > > > > > > > > > > > > > > > >
>> > > > https://issues.apache.org/jira/browse/KAFKA-1227
>> > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > A few questions or comments to kick
>> things
>> > > off:
>> > > > > > > > > > > > > > > > > > 1. We need to make a decision on 
>>whether
>> > > > > > > serialization
>> > > > > > > > of
>> > > > > > > > > > the
>> > > > > > > > > > > > > > user's
>> > > > > > > > > > > > > > > > key
>> > > > > > > > > > > > > > > > > > and value should be done by the user
>> (with
>> > > our
>> > > > > api
>> > > > > > > just
>> > > > > > > > > > > taking
>> > > > > > > > > > > > > > > byte[])
>> > > > > > > > > > > > > > > > or
>> > > > > > > > > > > > > > > > > > if we should take an object and allow 
>>the
>> > > user
>> > > > to
>> > > > > > > > > > configure a
>> > > > > > > > > > > > > > > > Serializer
>> > > > > > > > > > > > > > > > > > class which we instantiate via
>> reflection.
>> > We
>> > > > > take
>> > > > > > > the
>> > > > > > > > > > later
>> > > > > > > > > > > > > > approach
>> > > > > > > > > > > > > > > > in
>> > > > > > > > > > > > > > > > > > the current producer, and I have 
>>carried
>> > this
>> > > > > > through
>> > > > > > > > to
>> > > > > > > > > > this
>> > > > > > > > > > > > > > > > prototype.
>> > > > > > > > > > > > > > > > > > The tradeoff I see is this: taking 
>>byte[]
>> > is
>> > > > > > actually
>> > > > > > > > > > > simpler,
>> > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > user
>> > > > > > > > > > > > > > > > > can
>> > > > > > > > > > > > > > > > > > directly do whatever serialization 
>>they
>> > like.
>> > > > The
>> > > > > > > > > > > complication
>> > > > > > > > > > > > is
>> > > > > > > > > > > > > > > > > actually
>> > > > > > > > > > > > > > > > > > partitioning. Currently partitioning 
>>is
>> > done
>> > > > by a
>> > > > > > > > similar
>> > > > > > > > > > > > plug-in
>> > > > > > > > > > > > > > api
>> > > > > > > > > > > > > > > > > > (Partitioner) which the user can
>> implement
>> > > and
>> > > > > > > > configure
>> > > > > > > > > to
>> > > > > > > > > > > > > > override
>> > > > > > > > > > > > > > > > how
>> > > > > > > > > > > > > > > > > > partitions are assigned. If we take
>> byte[]
>> > as
>> > > > > input
>> > > > > > > > then
>> > > > > > > > > we
>> > > > > > > > > > > > have
>> > > > > > > > > > > > > no
>> > > > > > > > > > > > > > > > > access
>> > > > > > > > > > > > > > > > > > to the original object and 
>>partitioning
>> > MUST
>> > > be
>> > > > > > done
>> > > > > > > on
>> > > > > > > > > the
>> > > > > > > > > > > > > byte[].
>> > > > > > > > > > > > > > > > This
>> > > > > > > > > > > > > > > > > is
>> > > > > > > > > > > > > > > > > > fine for hash partitioning. However 
>>for
>> > > various
>> > > > > > types
>> > > > > > > > of
>> > > > > > > > > > > > semantic
>> > > > > > > > > > > > > > > > > > partitioning (range partitioning, or
>> > > whatever)
>> > > > > you
>> > > > > > > > would
>> > > > > > > > > > want
>> > > > > > > > > > > > > > access
>> > > > > > > > > > > > > > > to
>> > > > > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > original object. In the current 
>>approach
>> a
>> > > > > producer
>> > > > > > > who
>> > > > > > > > > > > wishes
>> > > > > > > > > > > > to
>> > > > > > > > > > > > > > > send
>> > > > > > > > > > > > > > > > > > byte[] they have serialized in their 
>>own
>> > code
>> > > > can
>> > > > > > > > > configure
>> > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > BytesSerialization we supply which is
>> just
>> > a
>> > > > "no
>> > > > > > op"
>> > > > > > > > > > > > > serialization.
>> > > > > > > > > > > > > > > > > > 2. We should obsess over naming and 
>>make
>> > sure
>> > > > > each
>> > > > > > of
>> > > > > > > > the
>> > > > > > > > > > > class
>> > > > > > > > > > > > > > names
>> > > > > > > > > > > > > > > > are
>> > > > > > > > > > > > > > > > > > good.
>> > > > > > > > > > > > > > > > > > 3. Jun has already pointed out that we
>> need
>> > > to
>> > > > > > > include
>> > > > > > > > > the
>> > > > > > > > > > > > topic
>> > > > > > > > > > > > > > and
>> > > > > > > > > > > > > > > > > > partition in the response, which is
>> > > absolutely
>> > > > > > > right. I
>> > > > > > > > > > > haven't
>> > > > > > > > > > > > > > done
>> > > > > > > > > > > > > > > > that
>> > > > > > > > > > > > > > > > > > yet but that definitely needs to be
>> there.
>> > > > > > > > > > > > > > > > > > 4. Currently RecordSend.await will 
>>throw
>> an
>> > > > > > exception
>> > > > > > > > if
>> > > > > > > > > > the
>> > > > > > > > > > > > > > request
>> > > > > > > > > > > > > > > > > > failed. The intention here is that
>> > > > > > > > > > > > producer.send(message).await()
>> > > > > > > > > > > > > > > > exactly
>> > > > > > > > > > > > > > > > > > simulates a synchronous call. Guozhang
>> has
>> > > > noted
>> > > > > > that
>> > > > > > > > > this
>> > > > > > > > > > > is a
>> > > > > > > > > > > > > > > little
>> > > > > > > > > > > > > > > > > > annoying since the user must then 
>>catch
>> > > > > exceptions.
>> > > > > > > > > However
>> > > > > > > > > > > if
>> > > > > > > > > > > > we
>> > > > > > > > > > > > > > > > remove
>> > > > > > > > > > > > > > > > > > this then if the user doesn't check 
>>for
>> > > errors
>> > > > > they
>> > > > > > > > won't
>> > > > > > > > > > > know
>> > > > > > > > > > > > > one
>> > > > > > > > > > > > > > > has
>> > > > > > > > > > > > > > > > > > occurred, which I predict will be a
>> common
>> > > > > mistake.
>> > > > > > > > > > > > > > > > > > 5. Perhaps there is more we could do 
>>to
>> > make
>> > > > the
>> > > > > > > async
>> > > > > > > > > > > > callbacks
>> > > > > > > > > > > > > > and
>> > > > > > > > > > > > > > > > > future
>> > > > > > > > > > > > > > > > > > we give back intuitive and easy to
>> program
>> > > > > against?
>> > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > Some background info on 
>>implementation:
>> > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > At a high level the primary 
>>difference in
>> > > this
>> > > > > > > producer
>> > > > > > > > > is
>> > > > > > > > > > > that
>> > > > > > > > > > > > > it
>> > > > > > > > > > > > > > > > > removes
>> > > > > > > > > > > > > > > > > > the distinction between the "sync" and
>> > > "async"
>> > > > > > > > producer.
>> > > > > > > > > > > > > > Effectively
>> > > > > > > > > > > > > > > > all
>> > > > > > > > > > > > > > > > > > requests are sent asynchronously but
>> always
>> > > > > return
>> > > > > > a
>> > > > > > > > > future
>> > > > > > > > > > > > > > response
>> > > > > > > > > > > > > > > > > object
>> > > > > > > > > > > > > > > > > > that gives the offset as well as any
>> error
>> > > that
>> > > > > may
>> > > > > > > > have
>> > > > > > > > > > > > occurred
>> > > > > > > > > > > > > > > when
>> > > > > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > request is complete. The batching 
>>that is
>> > > done
>> > > > in
>> > > > > > the
>> > > > > > > > > async
>> > > > > > > > > > > > > > producer
>> > > > > > > > > > > > > > > > only
>> > > > > > > > > > > > > > > > > > today is done whenever possible now. 
>>This
>> > > means
>> > > > > > that
>> > > > > > > > the
>> > > > > > > > > > sync
>> > > > > > > > > > > > > > > producer,
>> > > > > > > > > > > > > > > > > > under load, can get performance as 
>>good
>> as
>> > > the
>> > > > > > async
>> > > > > > > > > > producer
>> > > > > > > > > > > > > > > > > (preliminary
>> > > > > > > > > > > > > > > > > > results show the producer getting 1m
>> > > > > messages/sec).
>> > > > > > > > This
>> > > > > > > > > > > works
>> > > > > > > > > > > > > > > similar
>> > > > > > > > > > > > > > > > to
>> > > > > > > > > > > > > > > > > > group commit in databases but with
>> respect
>> > to
>> > > > the
>> > > > > > > > actual
>> > > > > > > > > > > > network
>> > > > > > > > > > > > > > > > > > transmission--any messages that arrive
>> > while
>> > > a
>> > > > > send
>> > > > > > > is
>> > > > > > > > in
>> > > > > > > > > > > > > progress
>> > > > > > > > > > > > > > > are
>> > > > > > > > > > > > > > > > > > batched together. It is also possible 
>>to
>> > > > > encourage
>> > > > > > > > > batching
>> > > > > > > > > > > > even
>> > > > > > > > > > > > > > > under
>> > > > > > > > > > > > > > > > > low
>> > > > > > > > > > > > > > > > > > load to save server resources by
>> > introducing
>> > > a
>> > > > > > delay
>> > > > > > > on
>> > > > > > > > > the
>> > > > > > > > > > > > send
>> > > > > > > > > > > > > to
>> > > > > > > > > > > > > > > > allow
>> > > > > > > > > > > > > > > > > > more messages to accumulate; this is 
>>done
>> > > using
>> > > > > the
>> > > > > > > > > > > > > linger.msconfig
>> > > > > > > > > > > > > > > > > (this
>> > > > > > > > > > > > > > > > > > is similar to Nagle's algorithm in 
>>TCP).
>> > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > This producer does all network
>> > communication
>> > > > > > > > > asynchronously
>> > > > > > > > > > > and
>> > > > > > > > > > > > > in
>> > > > > > > > > > > > > > > > > parallel
>> > > > > > > > > > > > > > > > > > to all servers so the performance 
>>penalty
>> > for
>> > > > > > acks=-1
>> > > > > > > > and
>> > > > > > > > > > > > waiting
>> > > > > > > > > > > > > > on
>> > > > > > > > > > > > > > > > > > replication should be much reduced. I
>> > haven't
>> > > > > done
>> > > > > > > much
>> > > > > > > > > > > > > > benchmarking
>> > > > > > > > > > > > > > > on
>> > > > > > > > > > > > > > > > > > this yet, though.
>> > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > The high level design is described a
>> little
>> > > > here,
>> > > > > > > > though
>> > > > > > > > > > this
>> > > > > > > > > > > > is
>> > > > > > > > > > > > > > now
>> > > > > > > > > > > > > > > a
>> > > > > > > > > > > > > > > > > > little out of date:
>> > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > >
>> > > > > > 
>>https://cwiki.apache.org/confluence/display/KAFKA/Client+Rewrite
>> > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > -Jay
>> > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > >
>> > > > > > > > > > > > > >
>> > > > > > > > > > > > >
>> > > > > > > > > > > >
>> > > > > > > > > > >
>> > > > > > > > > >
>> > > > > > > > >
>> > > > > > > >
>> > > > > > >
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>>


Re: New Producer Public API

Posted by Neha Narkhede <ne...@gmail.com>.
>> The challenge of directly exposing ProduceRequestResult is that the
offset
provided is just the base offset and there is no way to know for a
particular message where it was in relation to that base offset because the
batching is transparent and non-deterministic.

That's a good point. I need to look into the code more closely to see if it
is possible to expose
something like Future<RequestResult> send(...) where RequestResult has the
right metadata
as well as helper APIs that the user would want. For example

Future<RequestResult> messageResponse;
try {
  messageResponse = send(...)
} catch(InterruptedException ie) {
} catch(ExecutionException ee) {
}

if(messageResponse.hasError())
  // handle error
else {
   String topic = messageResponse.topic();
   int partition = messageResponse.partition();
   long offset = messageResponse.offset();   // can this offset return the
absolute offset instead of just the relative offset?
   ...
}

I could've missed some reasons why we can't do the above. I just think that
separating the future-like functionality of RecordSend
from the actual response metadata could be useful while supporting Future
at the same time.

Thanks,
Neha



On Wed, Jan 29, 2014 at 10:23 AM, Tom Brown <to...@gmail.com> wrote:

> I strongly support the user of Future. In fact, the cancel method may not
> be useless. Since the producer is meant to be used by N threads, it could
> easily get overloaded such that a produce request could not be sent
> immediately and had to be queued. In that case, cancelling should cause it
> to not actually get sent.
>
> --Tom
>
>
> On Wed, Jan 29, 2014 at 11:06 AM, Jay Kreps <ja...@gmail.com> wrote:
>
> > Hey Neha,
> >
> > Error handling in RecordSend works as in Future you will get the
> exception
> > if there is one from any of the accessor methods or await().
> >
> > The purpose of hasError was that you can write things slightly more
> simply
> > (which some people expressed preference for):
> >   if(send.hasError())
> >     // do something
> >   long offset = send.offset();
> >
> > Instead of the more the slightly longer:
> > try {
> >    long offset = send.offset();
> > } catch (KafkaException e) {
> >    // do something
> > }
> >
> >
> > On Wed, Jan 29, 2014 at 10:01 AM, Neha Narkhede <neha.narkhede@gmail.com
> > >wrote:
> >
> > > Regarding the use of Futures -
> > >
> > > Agree that there are some downsides to using Futures but both
> approaches
> > > have some tradeoffs.
> > >
> > > - Standardization and usability
> > > Future is a widely used and understood Java API and given that the
> > > functionality that RecordSend hopes to provide is essentially that of
> > > Future, I think it makes sense to expose a widely understood public API
> > for
> > > our clients. RecordSend, on the other hand, seems to provide some APIs
> > that
> > > are very similar to that of Future, in addition to exposing a bunch of
> > APIs
> > > that belong to ProduceRequestResult. As a user, I would've really
> > preferred
> > > to deal with ProduceRequestResult directly -
> > > Future<ProduceRequestResult> send(...)
> > >
> > > - Error handling
> > > RecordSend's error handling is quite unintuitive where the user has to
> > > remember to invoke hasError and error, instead of just throwing the
> > > exception. Now there are
> > > some downsides regarding error handling with the Future as well, where
> > the
> > > user has to catch InterruptedException when we would never run into it.
> > > However, it seems like a price worth paying for supporting a standard
> API
> > > and error handling
> > >
> > > - Unused APIs
> > > This is a downside of using Future, where the cancel() operation would
> > > always return false and mean nothing. But we can mention that caveat in
> > our
> > > Java docs.
> > >
> > > To summarize, I would prefer to expose a well understood and widely
> > adopted
> > > Java API and put up with the overhead of catching one unnecessary
> checked
> > > exception, rather than wrap the useful ProduceRequestResult in a custom
> > > async object (RecordSend) and explain that to our many users.
> > >
> > > Thanks,
> > > Neha
> > >
> > >
> > >
> > >
> > > On Tue, Jan 28, 2014 at 8:10 PM, Jay Kreps <ja...@gmail.com>
> wrote:
> > >
> > > > Hey Neha,
> > > >
> > > > Can you elaborate on why you prefer using Java's Future? The downside
> > in
> > > my
> > > > mind is the use of the checked InterruptedException and
> > > ExecutionException.
> > > > ExecutionException is arguable, but forcing you to catch
> > > > InterruptedException, often in code that can't be interrupted, seems
> > > > perverse. It also leaves us with the cancel() method which I don't
> > think
> > > we
> > > > really can implement.
> > > >
> > > > Option 1A, to recap/elaborate, was the following. There is no
> > Serializer
> > > or
> > > > Partitioner api. We take a byte[] key and value and an optional
> integer
> > > > partition. If you specify the integer partition it will be used. If
> you
> > > do
> > > > not specify a key or a partition the partition will be chosen in a
> > round
> > > > robin fashion. If you specify a key but no partition we will chose a
> > > > partition based on a hash of the key. In order to let the user find
> the
> > > > partition we will need to given them access to the Cluster instance
> > > > directly from the producer.
> > > >
> > > > -Jay
> > > >
> > > >
> > > > On Tue, Jan 28, 2014 at 6:25 PM, Neha Narkhede <
> > neha.narkhede@gmail.com
> > > > >wrote:
> > > >
> > > > > Here are more thoughts on the public APIs -
> > > > >
> > > > > - I suggest we use java's Future instead of custom Future
> especially
> > > > since
> > > > > it is part of the public API
> > > > >
> > > > > - Serialization: I like the simplicity of the producer APIs with
> the
> > > > > absence of serialization where we just deal with byte arrays for
> keys
> > > and
> > > > > values. What I don't like about this is the performance overhead on
> > the
> > > > > Partitioner for any kind of custom partitioning based on the
> > > > partitionKey.
> > > > > Since the only purpose of partitionKey is to do custom
> partitioning,
> > > why
> > > > > can't we take it in directly as an integer and let the user figure
> > out
> > > > the
> > > > > mapping from partition_key -> partition_id using the getCluster()
> > API?
> > > > If I
> > > > > understand correctly, this is similar to what you suggested as part
> > of
> > > > > option 1A. I like this approach since it maintains the simplicity
> of
> > > APIs
> > > > > by allowing us to deal with bytes and does not compromise
> performance
> > > in
> > > > > the custom partitioning case.
> > > > >
> > > > > Thanks,
> > > > > Neha
> > > > >
> > > > >
> > > > >
> > > > > On Tue, Jan 28, 2014 at 5:42 PM, Jay Kreps <ja...@gmail.com>
> > > wrote:
> > > > >
> > > > > > Hey Tom,
> > > > > >
> > > > > > That sounds cool. How did you end up handling parallel I/O if you
> > > wrap
> > > > > the
> > > > > > individual connections? Don't you need some selector that selects
> > > over
> > > > > all
> > > > > > the connections?
> > > > > >
> > > > > > -Jay
> > > > > >
> > > > > >
> > > > > > On Tue, Jan 28, 2014 at 2:31 PM, Tom Brown <tombrown52@gmail.com
> >
> > > > wrote:
> > > > > >
> > > > > > > I implemented a 0.7 client in pure java, and its API very
> closely
> > > > > > resembled
> > > > > > > this. (When multiple people independently engineer the same
> > > solution,
> > > > > > it's
> > > > > > > probably good... right?). However, there were a few
> architectural
> > > > > > > differences with my client:
> > > > > > >
> > > > > > > 1. The basic client itself was just an asynchronous layer
> around
> > > the
> > > > > > > different server functions. In and of itself it had no
> knowledge
> > of
> > > > > > > partitions, only servers (and maintained TCP connections to
> > them).
> > > > > > >
> > > > > > > 2. The main producer was an additional layer that provided a
> > > > high-level
> > > > > > > interface that could batch individual messages based on
> > partition.
> > > > > > >
> > > > > > > 3. Knowledge of partitioning was done via an interface so that
> > > > > different
> > > > > > > strategies could be used.
> > > > > > >
> > > > > > > 4. Partitioning was done by the user, with knowledge of the
> > > available
> > > > > > > partitions provided by #3.
> > > > > > >
> > > > > > > 5. Serialization was done by the user to simplify the API.
> > > > > > >
> > > > > > > 6. Futures were used to make asynchronous emulate synchronous
> > > calls.
> > > > > > >
> > > > > > >
> > > > > > > The main benefit of this approach is flexibility. For example,
> > > since
> > > > > the
> > > > > > > base client was just a managed connection (and not inherently a
> > > > > > producer),
> > > > > > > it was easy to composite a produce request and an offsets
> request
> > > > > > together
> > > > > > > into a confirmed produce request (officially not available in
> > 0.7).
> > > > > > >
> > > > > > > Decoupling the basic client from partition management allowed
> the
> > > me
> > > > to
> > > > > > > implement zk discovery as a separate project so that the main
> > > project
> > > > > had
> > > > > > > no complex dependencies. The same was true of decoupling
> > > > serialization.
> > > > > > > It's trivial to build an optional layer that adds those
> features
> > > in,
> > > > > > while
> > > > > > > allowing access to the base APIs for those that need it.
> > > > > > >
> > > > > > > Using standard Future objects was also beneficial, since I
> could
> > > > > combine
> > > > > > > them with existing tools (such as guava).
> > > > > > >
> > > > > > > It may be too late to be of use, but I have been working with
> my
> > > > > > company's
> > > > > > > legal department to release the implementation I described
> above.
> > > If
> > > > > > you're
> > > > > > > interested in it, let me know.
> > > > > > >
> > > > > > >
> > > > > > > To sum up my thoughts regarding the new API, I think it's a
> great
> > > > > start.
> > > > > > I
> > > > > > > would like to see a more layered approach so I can use the
> parts
> > I
> > > > > want,
> > > > > > > and adapt the other parts as needed. I would also like to see
> > > > standard
> > > > > > > interfaces (especially Future) used where they makes sense.
> > > > > > >
> > > > > > > --Tom
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > On Tue, Jan 28, 2014 at 1:33 PM, Roger Hoover <
> > > > roger.hoover@gmail.com
> > > > > > > >wrote:
> > > > > > >
> > > > > > > > +1 ListenableFuture: If this works similar to Deferreds in
> > > Twisted
> > > > > > Python
> > > > > > > > or Promised IO in Javascript, I think this is a great pattern
> > for
> > > > > > > > decoupling your callback logic from the place where the
> Future
> > is
> > > > > > > > generated.  You can register as many callbacks as you like,
> > each
> > > in
> > > > > the
> > > > > > > > appropriate layer of the code and have each observer get
> > notified
> > > > > when
> > > > > > > the
> > > > > > > > promised i/o is complete without any of them knowing about
> each
> > > > > other.
> > > > > > > >
> > > > > > > >
> > > > > > > > On Tue, Jan 28, 2014 at 11:32 AM, Jay Kreps <
> > jay.kreps@gmail.com
> > > >
> > > > > > wrote:
> > > > > > > >
> > > > > > > > > Hey Ross,
> > > > > > > > >
> > > > > > > > > - ListenableFuture: Interesting. That would be an
> alternative
> > > to
> > > > > the
> > > > > > > > direct
> > > > > > > > > callback support we provide. There could be pros to this,
> let
> > > me
> > > > > > think
> > > > > > > > > about it.
> > > > > > > > > - We could provide layering, but I feel that the
> > serialization
> > > is
> > > > > > such
> > > > > > > a
> > > > > > > > > small thing we should just make a decision and chose one,
> it
> > > > > doesn't
> > > > > > > seem
> > > > > > > > > to me to justify a whole public facing layer.
> > > > > > > > > - Yes, this is fairly esoteric, essentially I think it is
> > > fairly
> > > > > > > similar
> > > > > > > > to
> > > > > > > > > databases like DynamoDB that allow you to specify two
> > partition
> > > > > keys
> > > > > > (I
> > > > > > > > > think DynamoDB does this...). The reasoning is that in fact
> > > there
> > > > > are
> > > > > > > > > several things you can use the key field for: (1) to
> compute
> > > the
> > > > > > > > partition
> > > > > > > > > to store the data in, (2) as a unique identifier to
> > deduplicate
> > > > > that
> > > > > > > > > partition's records within a log. These two things are
> almost
> > > > > always
> > > > > > > the
> > > > > > > > > same, but occationally may differ when you want to group
> data
> > > in
> > > > a
> > > > > > more
> > > > > > > > > sophisticated way then just a hash of the primary key but
> > still
> > > > > > retain
> > > > > > > > the
> > > > > > > > > proper primary key for delivery to the consumer and log
> > > > compaction.
> > > > > > > > >
> > > > > > > > > -Jay
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > On Tue, Jan 28, 2014 at 3:24 AM, Ross Black <
> > > > > ross.w.black@gmail.com>
> > > > > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > Hi Jay,
> > > > > > > > > >
> > > > > > > > > > - Just to add some more info/confusion about possibly
> using
> > > > > Future
> > > > > > > ...
> > > > > > > > > >   If Kafka uses a JDK future, it plays nicely with other
> > > > > frameworks
> > > > > > > as
> > > > > > > > > > well.
> > > > > > > > > >   Google Guava has a ListenableFuture that allows
> callback
> > > > > handling
> > > > > > > to
> > > > > > > > be
> > > > > > > > > > added via the returned future, and allows the callbacks
> to
> > be
> > > > > > passed
> > > > > > > > off
> > > > > > > > > to
> > > > > > > > > > a specified executor.
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> http://docs.guava-libraries.googlecode.com/git-history/release/javadoc/com/google/common/util/concurrent/ListenableFuture.html
> > > > > > > > > >   The JDK future can easily be converted to a listenable
> > > > future.
> > > > > > > > > >
> > > > > > > > > > - On the question of byte[] vs Object, could this be
> solved
> > > by
> > > > > > > layering
> > > > > > > > > the
> > > > > > > > > > API?  eg. a raw producer (use byte[] and specify the
> > > partition
> > > > > > > number)
> > > > > > > > > and
> > > > > > > > > > a normal producer (use generic object and specify a
> > > > Partitioner)?
> > > > > > > > > >
> > > > > > > > > > - I am confused by the keys in ProducerRecord and
> > > Partitioner.
> > > > > >  What
> > > > > > > is
> > > > > > > > > the
> > > > > > > > > > usage for both a key and a partition key? (I am not yet
> > using
> > > > > 0.8)
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > Thanks,
> > > > > > > > > > Ross
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > On 28 January 2014 05:00, Xavier Stevens <
> > xavier@gaikai.com>
> > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > AutoCloseable would be nice for us as most of our code
> is
> > > > using
> > > > > > > Java
> > > > > > > > 7
> > > > > > > > > at
> > > > > > > > > > > this point.
> > > > > > > > > > >
> > > > > > > > > > > I like Dropwizard's configuration mapping to POJOs via
> > > > Jackson,
> > > > > > but
> > > > > > > > if
> > > > > > > > > > you
> > > > > > > > > > > wanted to stick with property maps I don't care enough
> to
> > > > > object.
> > > > > > > > > > >
> > > > > > > > > > > If the producer only dealt with bytes, is there a way
> we
> > > > could
> > > > > > > still
> > > > > > > > > due
> > > > > > > > > > > partition plugins without specifying the number
> > > explicitly? I
> > > > > > would
> > > > > > > > > > prefer
> > > > > > > > > > > to be able to pass in field(s) that would be used by
> the
> > > > > > > partitioner.
> > > > > > > > > > > Obviously if this wasn't possible you could always
> > > > deserialize
> > > > > > the
> > > > > > > > > object
> > > > > > > > > > > in the partitioner and grab the fields you want, but
> that
> > > > seems
> > > > > > > > really
> > > > > > > > > > > expensive to do on every message.
> > > > > > > > > > >
> > > > > > > > > > > It would also be nice to have a Java API Encoder
> > > constructor
> > > > > > taking
> > > > > > > > in
> > > > > > > > > > > VerifiableProperties. Scala understands how to handle
> > > "props:
> > > > > > > > > > > VerifiableProperties = null", but Java doesn't. So you
> > > don't
> > > > > run
> > > > > > > into
> > > > > > > > > > this
> > > > > > > > > > > problem until runtime.
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > -Xavier
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > On Mon, Jan 27, 2014 at 9:37 AM, Clark Breyman <
> > > > > > clark@breyman.com>
> > > > > > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > Jay -
> > > > > > > > > > > >
> > > > > > > > > > > > Config - your explanation makes sense. I'm just so
> > > > accustomed
> > > > > > to
> > > > > > > > > having
> > > > > > > > > > > > Jackson automatically map my configuration objects to
> > > POJOs
> > > > > > that
> > > > > > > > I've
> > > > > > > > > > > > stopped using property files. They are lingua franca.
> > The
> > > > > only
> > > > > > > > > thought
> > > > > > > > > > > > might be to separate the config interface from the
> > > > > > implementation
> > > > > > > > to
> > > > > > > > > > > allow
> > > > > > > > > > > > for alternatives, but that might undermine your point
> > of
> > > > "do
> > > > > it
> > > > > > > > this
> > > > > > > > > > way
> > > > > > > > > > > so
> > > > > > > > > > > > that everyone can find it where they expect it".
> > > > > > > > > > > >
> > > > > > > > > > > > Serialization: Of the options, I like 1A the best,
> > though
> > > > > > > possibly
> > > > > > > > > with
> > > > > > > > > > > > either an option to specify a partition key rather
> than
> > > ID
> > > > > or a
> > > > > > > > > helper
> > > > > > > > > > to
> > > > > > > > > > > > translate an arbitrary byte[] or long into a
> partition
> > > > > number.
> > > > > > > > > > > >
> > > > > > > > > > > > Thanks
> > > > > > > > > > > > Clark
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > On Sun, Jan 26, 2014 at 9:13 PM, Jay Kreps <
> > > > > > jay.kreps@gmail.com>
> > > > > > > > > > wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > > Thanks for the detailed thoughts. Let me elaborate
> on
> > > the
> > > > > > > config
> > > > > > > > > > thing.
> > > > > > > > > > > > >
> > > > > > > > > > > > > I agree that at first glance key-value strings
> don't
> > > seem
> > > > > > like
> > > > > > > a
> > > > > > > > > very
> > > > > > > > > > > > good
> > > > > > > > > > > > > configuration api for a client. Surely a well-typed
> > > > config
> > > > > > > class
> > > > > > > > > > would
> > > > > > > > > > > be
> > > > > > > > > > > > > better! I actually disagree and let me see if I can
> > > > > convince
> > > > > > > you.
> > > > > > > > > > > > >
> > > > > > > > > > > > > My reasoning has nothing to do with the api and
> > > > everything
> > > > > to
> > > > > > > do
> > > > > > > > > with
> > > > > > > > > > > > > operations.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Clients are embedded in applications which are
> > > themselves
> > > > > > > > > configured.
> > > > > > > > > > > In
> > > > > > > > > > > > > any place that takes operations seriously the
> > > > configuration
> > > > > > for
> > > > > > > > > these
> > > > > > > > > > > > > applications will be version controlled and
> > maintained
> > > > > > through
> > > > > > > > some
> > > > > > > > > > > kind
> > > > > > > > > > > > of
> > > > > > > > > > > > > config management system. If we give a config class
> > > with
> > > > > > > getters
> > > > > > > > > and
> > > > > > > > > > > > > setters the application has to expose those
> > properties
> > > to
> > > > > its
> > > > > > > > > > > > > configuration. What invariably happens is that the
> > > > > > application
> > > > > > > > > > exposes
> > > > > > > > > > > > only
> > > > > > > > > > > > > a choice few properties that they thought they
> would
> > > > > change.
> > > > > > > > > > > Furthermore
> > > > > > > > > > > > > the application will make up a name for these
> configs
> > > > that
> > > > > > > seems
> > > > > > > > > > > > intuitive
> > > > > > > > > > > > > at the time in the 2 seconds the engineer spends
> > > thinking
> > > > > > about
> > > > > > > > it.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Now consider the result of this in the large. You
> end
> > > up
> > > > > with
> > > > > > > > > dozens
> > > > > > > > > > or
> > > > > > > > > > > > > hundreds of applications that have the client
> > embedded.
> > > > > Each
> > > > > > > > > exposes
> > > > > > > > > > a
> > > > > > > > > > > > > different, inadequate subset of the possible
> configs,
> > > > each
> > > > > > with
> > > > > > > > > > > different
> > > > > > > > > > > > > names. It is a nightmare.
> > > > > > > > > > > > >
> > > > > > > > > > > > > If you use a string-string map the config system
> can
> > > > > directly
> > > > > > > > get a
> > > > > > > > > > > > bundle
> > > > > > > > > > > > > of config key-value pairs and put them into the
> > client.
> > > > > This
> > > > > > > > means
> > > > > > > > > > that
> > > > > > > > > > > > all
> > > > > > > > > > > > > configuration is automatically available with the
> > name
> > > > > > > documented
> > > > > > > > > on
> > > > > > > > > > > the
> > > > > > > > > > > > > website in every application that does this. If you
> > > > upgrade
> > > > > > to
> > > > > > > a
> > > > > > > > > new
> > > > > > > > > > > > kafka
> > > > > > > > > > > > > version with more configs those will be exposed
> too.
> > If
> > > > you
> > > > > > > > realize
> > > > > > > > > > > that
> > > > > > > > > > > > > you need to change a default you can just go
> through
> > > your
> > > > > > > configs
> > > > > > > > > and
> > > > > > > > > > > > > change it everywhere as it will have the same name
> > > > > > everywhere.
> > > > > > > > > > > > >
> > > > > > > > > > > > > -Jay
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Sun, Jan 26, 2014 at 4:47 PM, Clark Breyman <
> > > > > > > > clark@breyman.com>
> > > > > > > > > > > > wrote:
> > > > > > > > > > > > >
> > > > > > > > > > > > > > Thanks Jay. I'll see if I can put together a more
> > > > > complete
> > > > > > > > > > response,
> > > > > > > > > > > > > > perhaps as separate threads so that topics don't
> > get
> > > > > > > entangled.
> > > > > > > > > In
> > > > > > > > > > > the
> > > > > > > > > > > > > mean
> > > > > > > > > > > > > > time, here's a couple responses:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Serialization: you've broken out a sub-thread so
> > i'll
> > > > > reply
> > > > > > > > > there.
> > > > > > > > > > My
> > > > > > > > > > > > > bias
> > > > > > > > > > > > > > is that I like generics (except for type-erasure)
> > and
> > > > in
> > > > > > > > > particular
> > > > > > > > > > > > they
> > > > > > > > > > > > > > make it easy to compose serializers for compound
> > > > payloads
> > > > > > > (e.g.
> > > > > > > > > > when
> > > > > > > > > > > a
> > > > > > > > > > > > > > common header wraps a payload of parameterized
> > type).
> > > > > I'll
> > > > > > > > > respond
> > > > > > > > > > to
> > > > > > > > > > > > > your
> > > > > > > > > > > > > > 4-options message with an example.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Build: I've seen a lot of "maven-compatible"
> build
> > > > > systems
> > > > > > > > > produce
> > > > > > > > > > > > > > "artifacts" that aren't really artifacts - no
> > > embedded
> > > > > POM
> > > > > > > or,
> > > > > > > > > > worst,
> > > > > > > > > > > > > > malformed POM. I know the sbt-generated artifacts
> > > were
> > > > > this
> > > > > > > > way -
> > > > > > > > > > > onus
> > > > > > > > > > > > is
> > > > > > > > > > > > > > on me to see what gradle is spitting out and
> what a
> > > > maven
> > > > > > > build
> > > > > > > > > > might
> > > > > > > > > > > > > look
> > > > > > > > > > > > > > like. Maven may be old and boring, but it gets
> out
> > of
> > > > the
> > > > > > way
> > > > > > > > and
> > > > > > > > > > > > > > integrates really seamlessly with a lot of IDEs.
> > When
> > > > > some
> > > > > > > > scala
> > > > > > > > > > > > > projects I
> > > > > > > > > > > > > > was working on in the fall of 2011 switched from
> > sbt
> > > to
> > > > > > > maven,
> > > > > > > > > > build
> > > > > > > > > > > > > became
> > > > > > > > > > > > > > a non-issue.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Config: Not a big deal  and no, I don't think a
> > > > > dropwizard
> > > > > > > > > > dependency
> > > > > > > > > > > > is
> > > > > > > > > > > > > > appropriate. I do like using simple entity beans
> > > > (POJO's
> > > > > > not
> > > > > > > > > j2EE)
> > > > > > > > > > > for
> > > > > > > > > > > > > > configuration, especially if they can be
> marshalled
> > > > > without
> > > > > > > > > > > annotation
> > > > > > > > > > > > by
> > > > > > > > > > > > > > Jackson. I only mentioned the dropwizard-extras
> > > >  because
> > > > > it
> > > > > > > has
> > > > > > > > > > some
> > > > > > > > > > > > > entity
> > > > > > > > > > > > > > bean versions of the ZK and Kafka configs.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Domain-packaging: Also not a big deal - it's
> what's
> > > > > > expected
> > > > > > > > and
> > > > > > > > > > it's
> > > > > > > > > > > > > > pretty free in most IDE's. The advantages I see
> is
> > > that
> > > > > it
> > > > > > is
> > > > > > > > > clear
> > > > > > > > > > > > > whether
> > > > > > > > > > > > > > something is from the Apache Kafka project and
> > > whether
> > > > > > > > something
> > > > > > > > > is
> > > > > > > > > > > > from
> > > > > > > > > > > > > > another org and related to Kafka. That said,
> > nothing
> > > > > really
> > > > > > > > > > enforces
> > > > > > > > > > > > it.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Futures: I'll see if I can create some examples
> to
> > > > > > > demonstrate
> > > > > > > > > > Future
> > > > > > > > > > > > > > making interop easier.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Regards,
> > > > > > > > > > > > > > C
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On Fri, Jan 24, 2014 at 4:36 PM, Jay Kreps <
> > > > > > > > jay.kreps@gmail.com>
> > > > > > > > > > > > wrote:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Hey Clark,
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > - Serialization: Yes I agree with these though
> I
> > > > don't
> > > > > > > > consider
> > > > > > > > > > the
> > > > > > > > > > > > > loss
> > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > generics a big issue. I'll try to summarize
> what
> > I
> > > > > would
> > > > > > > > > consider
> > > > > > > > > > > the
> > > > > > > > > > > > > > best
> > > > > > > > > > > > > > > alternative api with raw byte[].
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > - Maven: We had this debate a few months back
> and
> > > the
> > > > > > > > consensus
> > > > > > > > > > was
> > > > > > > > > > > > > > gradle.
> > > > > > > > > > > > > > > Is there a specific issue with the poms gradle
> > > > makes? I
> > > > > > am
> > > > > > > > > > > extremely
> > > > > > > > > > > > > > loath
> > > > > > > > > > > > > > > to revisit the issue as build issues are a
> > > recurring
> > > > > > thing
> > > > > > > > and
> > > > > > > > > no
> > > > > > > > > > > one
> > > > > > > > > > > > > > ever
> > > > > > > > > > > > > > > agrees and ultimately our build needs are very
> > > > simple.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > - Config: I'm not sure if I follow the point.
> Are
> > > you
> > > > > > > saying
> > > > > > > > we
> > > > > > > > > > > > should
> > > > > > > > > > > > > > use
> > > > > > > > > > > > > > > something in dropwizard for config? One
> principle
> > > > here
> > > > > is
> > > > > > > to
> > > > > > > > > try
> > > > > > > > > > to
> > > > > > > > > > > > > > remove
> > > > > > > > > > > > > > > as many client dependencies as possible as we
> > > > > inevitably
> > > > > > > run
> > > > > > > > > into
> > > > > > > > > > > > > > terrible
> > > > > > > > > > > > > > > compatibility issues with users who use the
> same
> > > > > library
> > > > > > or
> > > > > > > > its
> > > > > > > > > > > > > > > dependencies at different versions. Or are you
> > > > talking
> > > > > > > about
> > > > > > > > > > > > > maintaining
> > > > > > > > > > > > > > > compatibility with existing config parameters?
> I
> > > > think
> > > > > as
> > > > > > > > much
> > > > > > > > > > as a
> > > > > > > > > > > > > > config
> > > > > > > > > > > > > > > in the existing client makes sense it should
> have
> > > the
> > > > > > same
> > > > > > > > name
> > > > > > > > > > (I
> > > > > > > > > > > > was
> > > > > > > > > > > > > a
> > > > > > > > > > > > > > > bit sloppy about that so I'll fix any errors
> > > there).
> > > > > > There
> > > > > > > > are
> > > > > > > > > a
> > > > > > > > > > > few
> > > > > > > > > > > > > new
> > > > > > > > > > > > > > > things and we should give those reasonable
> > > defaults.
> > > > I
> > > > > > > think
> > > > > > > > > > config
> > > > > > > > > > > > is
> > > > > > > > > > > > > > > important so I'll start a thread on the config
> > > > package
> > > > > in
> > > > > > > > > there.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > - org.apache.kafka: We could do this. I always
> > > > > considered
> > > > > > > it
> > > > > > > > > kind
> > > > > > > > > > > of
> > > > > > > > > > > > an
> > > > > > > > > > > > > > odd
> > > > > > > > > > > > > > > thing Java programmers do that has no real
> > > motivation
> > > > > > (but
> > > > > > > I
> > > > > > > > > > could
> > > > > > > > > > > be
> > > > > > > > > > > > > > > re-educated!). I don't think it ends up
> reducing
> > > > naming
> > > > > > > > > conflicts
> > > > > > > > > > > in
> > > > > > > > > > > > > > > practice and it adds a lot of noise and nested
> > > > > > directories.
> > > > > > > > Is
> > > > > > > > > > > there
> > > > > > > > > > > > a
> > > > > > > > > > > > > > > reason you prefer this or just to be more
> > standard?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > - Future: Basically I didn't see any particular
> > > > > > advantage.
> > > > > > > > The
> > > > > > > > > > > > cancel()
> > > > > > > > > > > > > > > method doesn't really make sense so probably
> > > wouldn't
> > > > > > work.
> > > > > > > > > > > Likewise
> > > > > > > > > > > > I
> > > > > > > > > > > > > > > dislike the checked exceptions it requires.
> > > > Basically I
> > > > > > > just
> > > > > > > > > > wrote
> > > > > > > > > > > > out
> > > > > > > > > > > > > > some
> > > > > > > > > > > > > > > code examples and it seemed cleaner with a
> > special
> > > > > > purpose
> > > > > > > > > > object.
> > > > > > > > > > > I
> > > > > > > > > > > > > > wasn't
> > > > > > > > > > > > > > > actually aware of plans for improved futures in
> > > java
> > > > 8
> > > > > or
> > > > > > > the
> > > > > > > > > > other
> > > > > > > > > > > > > > > integrations. Maybe you could elaborate on
> this a
> > > bit
> > > > > and
> > > > > > > > show
> > > > > > > > > > how
> > > > > > > > > > > it
> > > > > > > > > > > > > > would
> > > > > > > > > > > > > > > be used? Sounds promising, I just don't know a
> > lot
> > > > > about
> > > > > > > it.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > -Jay
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > On Fri, Jan 24, 2014 at 3:30 PM, Clark Breyman
> <
> > > > > > > > > > clark@breyman.com>
> > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Jay - Thanks for the call for comments.
> Here's
> > > some
> > > > > > > initial
> > > > > > > > > > > input:
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > - Make message serialization a client
> > > > responsibility
> > > > > > > > (making
> > > > > > > > > > all
> > > > > > > > > > > > > > messages
> > > > > > > > > > > > > > > > byte[]). Reflection-based loading makes it
> > harder
> > > > to
> > > > > > use
> > > > > > > > > > generic
> > > > > > > > > > > > > codecs
> > > > > > > > > > > > > > > > (e.g.  Envelope<PREFIX, DATA, SUFFIX>) or
> build
> > > up
> > > > > > codec
> > > > > > > > > > > > > > > programmatically.
> > > > > > > > > > > > > > > > Non-default partitioning should require an
> > > explicit
> > > > > > > > partition
> > > > > > > > > > > key.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > - I really like the fact that it will be
> native
> > > > Java.
> > > > > > > > Please
> > > > > > > > > > > > consider
> > > > > > > > > > > > > > > using
> > > > > > > > > > > > > > > > native maven and not sbt, gradle, ivy, etc as
> > > they
> > > > > > don't
> > > > > > > > > > reliably
> > > > > > > > > > > > > play
> > > > > > > > > > > > > > > nice
> > > > > > > > > > > > > > > > in the maven ecosystem. A jar without a
> > > well-formed
> > > > > pom
> > > > > > > > > doesn't
> > > > > > > > > > > > feel
> > > > > > > > > > > > > > > like a
> > > > > > > > > > > > > > > > real artifact. The pom's generated by sbt et
> > al.
> > > > are
> > > > > > not
> > > > > > > > well
> > > > > > > > > > > > formed.
> > > > > > > > > > > > > > > Using
> > > > > > > > > > > > > > > > maven will make builds and IDE integration
> much
> > > > > > smoother.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > - Look at Nick Telford's dropwizard-extras
> > > package
> > > > in
> > > > > > > which
> > > > > > > > > he
> > > > > > > > > > > > > defines
> > > > > > > > > > > > > > > some
> > > > > > > > > > > > > > > > Jackson-compatible POJO's for loading
> > > > configuration.
> > > > > > > Seems
> > > > > > > > > like
> > > > > > > > > > > > your
> > > > > > > > > > > > > > > client
> > > > > > > > > > > > > > > > migration is similar. The config objects
> should
> > > > have
> > > > > > > > > > constructors
> > > > > > > > > > > > or
> > > > > > > > > > > > > > > > factories that accept Map<String, String> and
> > > > > > Properties
> > > > > > > > for
> > > > > > > > > > ease
> > > > > > > > > > > > of
> > > > > > > > > > > > > > > > migration.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > - Would you consider using the
> org.apache.kafka
> > > > > package
> > > > > > > for
> > > > > > > > > the
> > > > > > > > > > > new
> > > > > > > > > > > > > API
> > > > > > > > > > > > > > > > (quibble)
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > - Why create your own futures rather than use
> > > > > > > > > > > > > > > > java.util.concurrent.Future<Long> or similar?
> > > > > Standard
> > > > > > > > > futures
> > > > > > > > > > > will
> > > > > > > > > > > > > > play
> > > > > > > > > > > > > > > > nice with other reactive libs and things like
> > > J8's
> > > > > > > > > > > > ComposableFuture.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Thanks again,
> > > > > > > > > > > > > > > > C
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > On Fri, Jan 24, 2014 at 2:46 PM, Roger
> Hoover <
> > > > > > > > > > > > > roger.hoover@gmail.com
> > > > > > > > > > > > > > > > >wrote:
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > A couple comments:
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > 1) Why does the config use a broker list
> > > instead
> > > > of
> > > > > > > > > > discovering
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > brokers
> > > > > > > > > > > > > > > > > in ZooKeeper?  It doesn't match the
> > > > > HighLevelConsumer
> > > > > > > > API.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > 2) It looks like broker connections are
> > created
> > > > on
> > > > > > > > demand.
> > > > > > > > > >  I'm
> > > > > > > > > > > > > > > wondering
> > > > > > > > > > > > > > > > > if sometimes you might want to flush out
> > config
> > > > or
> > > > > > > > network
> > > > > > > > > > > > > > connectivity
> > > > > > > > > > > > > > > > > issues before pushing the first message
> > > through.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Should there also be a
> > KafkaProducer.connect()
> > > or
> > > > > > > .open()
> > > > > > > > > > > method
> > > > > > > > > > > > or
> > > > > > > > > > > > > > > > > connectAll()?  I guess it would try to
> > connect
> > > to
> > > > > all
> > > > > > > > > brokers
> > > > > > > > > > > in
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > BROKER_LIST_CONFIG
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > HTH,
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Roger
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > On Fri, Jan 24, 2014 at 11:54 AM, Jay
> Kreps <
> > > > > > > > > > > jay.kreps@gmail.com
> > > > > > > > > > > > >
> > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > As mentioned in a previous email we are
> > > working
> > > > > on
> > > > > > a
> > > > > > > > > > > > > > > re-implementation
> > > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > the producer. I would like to use this
> > email
> > > > > thread
> > > > > > > to
> > > > > > > > > > > discuss
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > details
> > > > > > > > > > > > > > > > > > of the public API and the configuration.
> I
> > > > would
> > > > > > love
> > > > > > > > for
> > > > > > > > > > us
> > > > > > > > > > > to
> > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > incredibly picky about this public api
> now
> > so
> > > > it
> > > > > is
> > > > > > > as
> > > > > > > > > good
> > > > > > > > > > > as
> > > > > > > > > > > > > > > possible
> > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > we don't need to break it in the future.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > The best way to get a feel for the API is
> > > > > actually
> > > > > > to
> > > > > > > > > take
> > > > > > > > > > a
> > > > > > > > > > > > look
> > > > > > > > > > > > > > at
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > javadoc, my hope is to get the api docs
> > good
> > > > > enough
> > > > > > > so
> > > > > > > > > that
> > > > > > > > > > > it
> > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > self-explanatory:
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> http://empathybox.com/kafka-javadoc/index.html?kafka/clients/producer/KafkaProducer.html
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Please take a look at this API and give
> me
> > > any
> > > > > > > thoughts
> > > > > > > > > you
> > > > > > > > > > > may
> > > > > > > > > > > > > > have!
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > It may also be reasonable to take a look
> at
> > > the
> > > > > > > > configs:
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> http://empathybox.com/kafka-javadoc/kafka/clients/producer/ProducerConfig.html
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > The actual code is posted here:
> > > > > > > > > > > > > > > > > >
> > > > https://issues.apache.org/jira/browse/KAFKA-1227
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > A few questions or comments to kick
> things
> > > off:
> > > > > > > > > > > > > > > > > > 1. We need to make a decision on whether
> > > > > > > serialization
> > > > > > > > of
> > > > > > > > > > the
> > > > > > > > > > > > > > user's
> > > > > > > > > > > > > > > > key
> > > > > > > > > > > > > > > > > > and value should be done by the user
> (with
> > > our
> > > > > api
> > > > > > > just
> > > > > > > > > > > taking
> > > > > > > > > > > > > > > byte[])
> > > > > > > > > > > > > > > > or
> > > > > > > > > > > > > > > > > > if we should take an object and allow the
> > > user
> > > > to
> > > > > > > > > > configure a
> > > > > > > > > > > > > > > > Serializer
> > > > > > > > > > > > > > > > > > class which we instantiate via
> reflection.
> > We
> > > > > take
> > > > > > > the
> > > > > > > > > > later
> > > > > > > > > > > > > > approach
> > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > the current producer, and I have carried
> > this
> > > > > > through
> > > > > > > > to
> > > > > > > > > > this
> > > > > > > > > > > > > > > > prototype.
> > > > > > > > > > > > > > > > > > The tradeoff I see is this: taking byte[]
> > is
> > > > > > actually
> > > > > > > > > > > simpler,
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > user
> > > > > > > > > > > > > > > > > can
> > > > > > > > > > > > > > > > > > directly do whatever serialization they
> > like.
> > > > The
> > > > > > > > > > > complication
> > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > actually
> > > > > > > > > > > > > > > > > > partitioning. Currently partitioning is
> > done
> > > > by a
> > > > > > > > similar
> > > > > > > > > > > > plug-in
> > > > > > > > > > > > > > api
> > > > > > > > > > > > > > > > > > (Partitioner) which the user can
> implement
> > > and
> > > > > > > > configure
> > > > > > > > > to
> > > > > > > > > > > > > > override
> > > > > > > > > > > > > > > > how
> > > > > > > > > > > > > > > > > > partitions are assigned. If we take
> byte[]
> > as
> > > > > input
> > > > > > > > then
> > > > > > > > > we
> > > > > > > > > > > > have
> > > > > > > > > > > > > no
> > > > > > > > > > > > > > > > > access
> > > > > > > > > > > > > > > > > > to the original object and partitioning
> > MUST
> > > be
> > > > > > done
> > > > > > > on
> > > > > > > > > the
> > > > > > > > > > > > > byte[].
> > > > > > > > > > > > > > > > This
> > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > fine for hash partitioning. However for
> > > various
> > > > > > types
> > > > > > > > of
> > > > > > > > > > > > semantic
> > > > > > > > > > > > > > > > > > partitioning (range partitioning, or
> > > whatever)
> > > > > you
> > > > > > > > would
> > > > > > > > > > want
> > > > > > > > > > > > > > access
> > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > original object. In the current approach
> a
> > > > > producer
> > > > > > > who
> > > > > > > > > > > wishes
> > > > > > > > > > > > to
> > > > > > > > > > > > > > > send
> > > > > > > > > > > > > > > > > > byte[] they have serialized in their own
> > code
> > > > can
> > > > > > > > > configure
> > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > BytesSerialization we supply which is
> just
> > a
> > > > "no
> > > > > > op"
> > > > > > > > > > > > > serialization.
> > > > > > > > > > > > > > > > > > 2. We should obsess over naming and make
> > sure
> > > > > each
> > > > > > of
> > > > > > > > the
> > > > > > > > > > > class
> > > > > > > > > > > > > > names
> > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > good.
> > > > > > > > > > > > > > > > > > 3. Jun has already pointed out that we
> need
> > > to
> > > > > > > include
> > > > > > > > > the
> > > > > > > > > > > > topic
> > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > partition in the response, which is
> > > absolutely
> > > > > > > right. I
> > > > > > > > > > > haven't
> > > > > > > > > > > > > > done
> > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > yet but that definitely needs to be
> there.
> > > > > > > > > > > > > > > > > > 4. Currently RecordSend.await will throw
> an
> > > > > > exception
> > > > > > > > if
> > > > > > > > > > the
> > > > > > > > > > > > > > request
> > > > > > > > > > > > > > > > > > failed. The intention here is that
> > > > > > > > > > > > producer.send(message).await()
> > > > > > > > > > > > > > > > exactly
> > > > > > > > > > > > > > > > > > simulates a synchronous call. Guozhang
> has
> > > > noted
> > > > > > that
> > > > > > > > > this
> > > > > > > > > > > is a
> > > > > > > > > > > > > > > little
> > > > > > > > > > > > > > > > > > annoying since the user must then catch
> > > > > exceptions.
> > > > > > > > > However
> > > > > > > > > > > if
> > > > > > > > > > > > we
> > > > > > > > > > > > > > > > remove
> > > > > > > > > > > > > > > > > > this then if the user doesn't check for
> > > errors
> > > > > they
> > > > > > > > won't
> > > > > > > > > > > know
> > > > > > > > > > > > > one
> > > > > > > > > > > > > > > has
> > > > > > > > > > > > > > > > > > occurred, which I predict will be a
> common
> > > > > mistake.
> > > > > > > > > > > > > > > > > > 5. Perhaps there is more we could do to
> > make
> > > > the
> > > > > > > async
> > > > > > > > > > > > callbacks
> > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > future
> > > > > > > > > > > > > > > > > > we give back intuitive and easy to
> program
> > > > > against?
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Some background info on implementation:
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > At a high level the primary difference in
> > > this
> > > > > > > producer
> > > > > > > > > is
> > > > > > > > > > > that
> > > > > > > > > > > > > it
> > > > > > > > > > > > > > > > > removes
> > > > > > > > > > > > > > > > > > the distinction between the "sync" and
> > > "async"
> > > > > > > > producer.
> > > > > > > > > > > > > > Effectively
> > > > > > > > > > > > > > > > all
> > > > > > > > > > > > > > > > > > requests are sent asynchronously but
> always
> > > > > return
> > > > > > a
> > > > > > > > > future
> > > > > > > > > > > > > > response
> > > > > > > > > > > > > > > > > object
> > > > > > > > > > > > > > > > > > that gives the offset as well as any
> error
> > > that
> > > > > may
> > > > > > > > have
> > > > > > > > > > > > occurred
> > > > > > > > > > > > > > > when
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > request is complete. The batching that is
> > > done
> > > > in
> > > > > > the
> > > > > > > > > async
> > > > > > > > > > > > > > producer
> > > > > > > > > > > > > > > > only
> > > > > > > > > > > > > > > > > > today is done whenever possible now. This
> > > means
> > > > > > that
> > > > > > > > the
> > > > > > > > > > sync
> > > > > > > > > > > > > > > producer,
> > > > > > > > > > > > > > > > > > under load, can get performance as good
> as
> > > the
> > > > > > async
> > > > > > > > > > producer
> > > > > > > > > > > > > > > > > (preliminary
> > > > > > > > > > > > > > > > > > results show the producer getting 1m
> > > > > messages/sec).
> > > > > > > > This
> > > > > > > > > > > works
> > > > > > > > > > > > > > > similar
> > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > group commit in databases but with
> respect
> > to
> > > > the
> > > > > > > > actual
> > > > > > > > > > > > network
> > > > > > > > > > > > > > > > > > transmission--any messages that arrive
> > while
> > > a
> > > > > send
> > > > > > > is
> > > > > > > > in
> > > > > > > > > > > > > progress
> > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > batched together. It is also possible to
> > > > > encourage
> > > > > > > > > batching
> > > > > > > > > > > > even
> > > > > > > > > > > > > > > under
> > > > > > > > > > > > > > > > > low
> > > > > > > > > > > > > > > > > > load to save server resources by
> > introducing
> > > a
> > > > > > delay
> > > > > > > on
> > > > > > > > > the
> > > > > > > > > > > > send
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > allow
> > > > > > > > > > > > > > > > > > more messages to accumulate; this is done
> > > using
> > > > > the
> > > > > > > > > > > > > linger.msconfig
> > > > > > > > > > > > > > > > > (this
> > > > > > > > > > > > > > > > > > is similar to Nagle's algorithm in TCP).
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > This producer does all network
> > communication
> > > > > > > > > asynchronously
> > > > > > > > > > > and
> > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > parallel
> > > > > > > > > > > > > > > > > > to all servers so the performance penalty
> > for
> > > > > > acks=-1
> > > > > > > > and
> > > > > > > > > > > > waiting
> > > > > > > > > > > > > > on
> > > > > > > > > > > > > > > > > > replication should be much reduced. I
> > haven't
> > > > > done
> > > > > > > much
> > > > > > > > > > > > > > benchmarking
> > > > > > > > > > > > > > > on
> > > > > > > > > > > > > > > > > > this yet, though.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > The high level design is described a
> little
> > > > here,
> > > > > > > > though
> > > > > > > > > > this
> > > > > > > > > > > > is
> > > > > > > > > > > > > > now
> > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > little out of date:
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > https://cwiki.apache.org/confluence/display/KAFKA/Client+Rewrite
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > -Jay
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: New Producer Public API

Posted by Tom Brown <to...@gmail.com>.
I strongly support the user of Future. In fact, the cancel method may not
be useless. Since the producer is meant to be used by N threads, it could
easily get overloaded such that a produce request could not be sent
immediately and had to be queued. In that case, cancelling should cause it
to not actually get sent.

--Tom


On Wed, Jan 29, 2014 at 11:06 AM, Jay Kreps <ja...@gmail.com> wrote:

> Hey Neha,
>
> Error handling in RecordSend works as in Future you will get the exception
> if there is one from any of the accessor methods or await().
>
> The purpose of hasError was that you can write things slightly more simply
> (which some people expressed preference for):
>   if(send.hasError())
>     // do something
>   long offset = send.offset();
>
> Instead of the more the slightly longer:
> try {
>    long offset = send.offset();
> } catch (KafkaException e) {
>    // do something
> }
>
>
> On Wed, Jan 29, 2014 at 10:01 AM, Neha Narkhede <neha.narkhede@gmail.com
> >wrote:
>
> > Regarding the use of Futures -
> >
> > Agree that there are some downsides to using Futures but both approaches
> > have some tradeoffs.
> >
> > - Standardization and usability
> > Future is a widely used and understood Java API and given that the
> > functionality that RecordSend hopes to provide is essentially that of
> > Future, I think it makes sense to expose a widely understood public API
> for
> > our clients. RecordSend, on the other hand, seems to provide some APIs
> that
> > are very similar to that of Future, in addition to exposing a bunch of
> APIs
> > that belong to ProduceRequestResult. As a user, I would've really
> preferred
> > to deal with ProduceRequestResult directly -
> > Future<ProduceRequestResult> send(...)
> >
> > - Error handling
> > RecordSend's error handling is quite unintuitive where the user has to
> > remember to invoke hasError and error, instead of just throwing the
> > exception. Now there are
> > some downsides regarding error handling with the Future as well, where
> the
> > user has to catch InterruptedException when we would never run into it.
> > However, it seems like a price worth paying for supporting a standard API
> > and error handling
> >
> > - Unused APIs
> > This is a downside of using Future, where the cancel() operation would
> > always return false and mean nothing. But we can mention that caveat in
> our
> > Java docs.
> >
> > To summarize, I would prefer to expose a well understood and widely
> adopted
> > Java API and put up with the overhead of catching one unnecessary checked
> > exception, rather than wrap the useful ProduceRequestResult in a custom
> > async object (RecordSend) and explain that to our many users.
> >
> > Thanks,
> > Neha
> >
> >
> >
> >
> > On Tue, Jan 28, 2014 at 8:10 PM, Jay Kreps <ja...@gmail.com> wrote:
> >
> > > Hey Neha,
> > >
> > > Can you elaborate on why you prefer using Java's Future? The downside
> in
> > my
> > > mind is the use of the checked InterruptedException and
> > ExecutionException.
> > > ExecutionException is arguable, but forcing you to catch
> > > InterruptedException, often in code that can't be interrupted, seems
> > > perverse. It also leaves us with the cancel() method which I don't
> think
> > we
> > > really can implement.
> > >
> > > Option 1A, to recap/elaborate, was the following. There is no
> Serializer
> > or
> > > Partitioner api. We take a byte[] key and value and an optional integer
> > > partition. If you specify the integer partition it will be used. If you
> > do
> > > not specify a key or a partition the partition will be chosen in a
> round
> > > robin fashion. If you specify a key but no partition we will chose a
> > > partition based on a hash of the key. In order to let the user find the
> > > partition we will need to given them access to the Cluster instance
> > > directly from the producer.
> > >
> > > -Jay
> > >
> > >
> > > On Tue, Jan 28, 2014 at 6:25 PM, Neha Narkhede <
> neha.narkhede@gmail.com
> > > >wrote:
> > >
> > > > Here are more thoughts on the public APIs -
> > > >
> > > > - I suggest we use java's Future instead of custom Future especially
> > > since
> > > > it is part of the public API
> > > >
> > > > - Serialization: I like the simplicity of the producer APIs with the
> > > > absence of serialization where we just deal with byte arrays for keys
> > and
> > > > values. What I don't like about this is the performance overhead on
> the
> > > > Partitioner for any kind of custom partitioning based on the
> > > partitionKey.
> > > > Since the only purpose of partitionKey is to do custom partitioning,
> > why
> > > > can't we take it in directly as an integer and let the user figure
> out
> > > the
> > > > mapping from partition_key -> partition_id using the getCluster()
> API?
> > > If I
> > > > understand correctly, this is similar to what you suggested as part
> of
> > > > option 1A. I like this approach since it maintains the simplicity of
> > APIs
> > > > by allowing us to deal with bytes and does not compromise performance
> > in
> > > > the custom partitioning case.
> > > >
> > > > Thanks,
> > > > Neha
> > > >
> > > >
> > > >
> > > > On Tue, Jan 28, 2014 at 5:42 PM, Jay Kreps <ja...@gmail.com>
> > wrote:
> > > >
> > > > > Hey Tom,
> > > > >
> > > > > That sounds cool. How did you end up handling parallel I/O if you
> > wrap
> > > > the
> > > > > individual connections? Don't you need some selector that selects
> > over
> > > > all
> > > > > the connections?
> > > > >
> > > > > -Jay
> > > > >
> > > > >
> > > > > On Tue, Jan 28, 2014 at 2:31 PM, Tom Brown <to...@gmail.com>
> > > wrote:
> > > > >
> > > > > > I implemented a 0.7 client in pure java, and its API very closely
> > > > > resembled
> > > > > > this. (When multiple people independently engineer the same
> > solution,
> > > > > it's
> > > > > > probably good... right?). However, there were a few architectural
> > > > > > differences with my client:
> > > > > >
> > > > > > 1. The basic client itself was just an asynchronous layer around
> > the
> > > > > > different server functions. In and of itself it had no knowledge
> of
> > > > > > partitions, only servers (and maintained TCP connections to
> them).
> > > > > >
> > > > > > 2. The main producer was an additional layer that provided a
> > > high-level
> > > > > > interface that could batch individual messages based on
> partition.
> > > > > >
> > > > > > 3. Knowledge of partitioning was done via an interface so that
> > > > different
> > > > > > strategies could be used.
> > > > > >
> > > > > > 4. Partitioning was done by the user, with knowledge of the
> > available
> > > > > > partitions provided by #3.
> > > > > >
> > > > > > 5. Serialization was done by the user to simplify the API.
> > > > > >
> > > > > > 6. Futures were used to make asynchronous emulate synchronous
> > calls.
> > > > > >
> > > > > >
> > > > > > The main benefit of this approach is flexibility. For example,
> > since
> > > > the
> > > > > > base client was just a managed connection (and not inherently a
> > > > > producer),
> > > > > > it was easy to composite a produce request and an offsets request
> > > > > together
> > > > > > into a confirmed produce request (officially not available in
> 0.7).
> > > > > >
> > > > > > Decoupling the basic client from partition management allowed the
> > me
> > > to
> > > > > > implement zk discovery as a separate project so that the main
> > project
> > > > had
> > > > > > no complex dependencies. The same was true of decoupling
> > > serialization.
> > > > > > It's trivial to build an optional layer that adds those features
> > in,
> > > > > while
> > > > > > allowing access to the base APIs for those that need it.
> > > > > >
> > > > > > Using standard Future objects was also beneficial, since I could
> > > > combine
> > > > > > them with existing tools (such as guava).
> > > > > >
> > > > > > It may be too late to be of use, but I have been working with my
> > > > > company's
> > > > > > legal department to release the implementation I described above.
> > If
> > > > > you're
> > > > > > interested in it, let me know.
> > > > > >
> > > > > >
> > > > > > To sum up my thoughts regarding the new API, I think it's a great
> > > > start.
> > > > > I
> > > > > > would like to see a more layered approach so I can use the parts
> I
> > > > want,
> > > > > > and adapt the other parts as needed. I would also like to see
> > > standard
> > > > > > interfaces (especially Future) used where they makes sense.
> > > > > >
> > > > > > --Tom
> > > > > >
> > > > > >
> > > > > >
> > > > > >
> > > > > >
> > > > > > On Tue, Jan 28, 2014 at 1:33 PM, Roger Hoover <
> > > roger.hoover@gmail.com
> > > > > > >wrote:
> > > > > >
> > > > > > > +1 ListenableFuture: If this works similar to Deferreds in
> > Twisted
> > > > > Python
> > > > > > > or Promised IO in Javascript, I think this is a great pattern
> for
> > > > > > > decoupling your callback logic from the place where the Future
> is
> > > > > > > generated.  You can register as many callbacks as you like,
> each
> > in
> > > > the
> > > > > > > appropriate layer of the code and have each observer get
> notified
> > > > when
> > > > > > the
> > > > > > > promised i/o is complete without any of them knowing about each
> > > > other.
> > > > > > >
> > > > > > >
> > > > > > > On Tue, Jan 28, 2014 at 11:32 AM, Jay Kreps <
> jay.kreps@gmail.com
> > >
> > > > > wrote:
> > > > > > >
> > > > > > > > Hey Ross,
> > > > > > > >
> > > > > > > > - ListenableFuture: Interesting. That would be an alternative
> > to
> > > > the
> > > > > > > direct
> > > > > > > > callback support we provide. There could be pros to this, let
> > me
> > > > > think
> > > > > > > > about it.
> > > > > > > > - We could provide layering, but I feel that the
> serialization
> > is
> > > > > such
> > > > > > a
> > > > > > > > small thing we should just make a decision and chose one, it
> > > > doesn't
> > > > > > seem
> > > > > > > > to me to justify a whole public facing layer.
> > > > > > > > - Yes, this is fairly esoteric, essentially I think it is
> > fairly
> > > > > > similar
> > > > > > > to
> > > > > > > > databases like DynamoDB that allow you to specify two
> partition
> > > > keys
> > > > > (I
> > > > > > > > think DynamoDB does this...). The reasoning is that in fact
> > there
> > > > are
> > > > > > > > several things you can use the key field for: (1) to compute
> > the
> > > > > > > partition
> > > > > > > > to store the data in, (2) as a unique identifier to
> deduplicate
> > > > that
> > > > > > > > partition's records within a log. These two things are almost
> > > > always
> > > > > > the
> > > > > > > > same, but occationally may differ when you want to group data
> > in
> > > a
> > > > > more
> > > > > > > > sophisticated way then just a hash of the primary key but
> still
> > > > > retain
> > > > > > > the
> > > > > > > > proper primary key for delivery to the consumer and log
> > > compaction.
> > > > > > > >
> > > > > > > > -Jay
> > > > > > > >
> > > > > > > >
> > > > > > > > On Tue, Jan 28, 2014 at 3:24 AM, Ross Black <
> > > > ross.w.black@gmail.com>
> > > > > > > > wrote:
> > > > > > > >
> > > > > > > > > Hi Jay,
> > > > > > > > >
> > > > > > > > > - Just to add some more info/confusion about possibly using
> > > > Future
> > > > > > ...
> > > > > > > > >   If Kafka uses a JDK future, it plays nicely with other
> > > > frameworks
> > > > > > as
> > > > > > > > > well.
> > > > > > > > >   Google Guava has a ListenableFuture that allows callback
> > > > handling
> > > > > > to
> > > > > > > be
> > > > > > > > > added via the returned future, and allows the callbacks to
> be
> > > > > passed
> > > > > > > off
> > > > > > > > to
> > > > > > > > > a specified executor.
> > > > > > > > >
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> http://docs.guava-libraries.googlecode.com/git-history/release/javadoc/com/google/common/util/concurrent/ListenableFuture.html
> > > > > > > > >   The JDK future can easily be converted to a listenable
> > > future.
> > > > > > > > >
> > > > > > > > > - On the question of byte[] vs Object, could this be solved
> > by
> > > > > > layering
> > > > > > > > the
> > > > > > > > > API?  eg. a raw producer (use byte[] and specify the
> > partition
> > > > > > number)
> > > > > > > > and
> > > > > > > > > a normal producer (use generic object and specify a
> > > Partitioner)?
> > > > > > > > >
> > > > > > > > > - I am confused by the keys in ProducerRecord and
> > Partitioner.
> > > > >  What
> > > > > > is
> > > > > > > > the
> > > > > > > > > usage for both a key and a partition key? (I am not yet
> using
> > > > 0.8)
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > Thanks,
> > > > > > > > > Ross
> > > > > > > > >
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > On 28 January 2014 05:00, Xavier Stevens <
> xavier@gaikai.com>
> > > > > wrote:
> > > > > > > > >
> > > > > > > > > > AutoCloseable would be nice for us as most of our code is
> > > using
> > > > > > Java
> > > > > > > 7
> > > > > > > > at
> > > > > > > > > > this point.
> > > > > > > > > >
> > > > > > > > > > I like Dropwizard's configuration mapping to POJOs via
> > > Jackson,
> > > > > but
> > > > > > > if
> > > > > > > > > you
> > > > > > > > > > wanted to stick with property maps I don't care enough to
> > > > object.
> > > > > > > > > >
> > > > > > > > > > If the producer only dealt with bytes, is there a way we
> > > could
> > > > > > still
> > > > > > > > due
> > > > > > > > > > partition plugins without specifying the number
> > explicitly? I
> > > > > would
> > > > > > > > > prefer
> > > > > > > > > > to be able to pass in field(s) that would be used by the
> > > > > > partitioner.
> > > > > > > > > > Obviously if this wasn't possible you could always
> > > deserialize
> > > > > the
> > > > > > > > object
> > > > > > > > > > in the partitioner and grab the fields you want, but that
> > > seems
> > > > > > > really
> > > > > > > > > > expensive to do on every message.
> > > > > > > > > >
> > > > > > > > > > It would also be nice to have a Java API Encoder
> > constructor
> > > > > taking
> > > > > > > in
> > > > > > > > > > VerifiableProperties. Scala understands how to handle
> > "props:
> > > > > > > > > > VerifiableProperties = null", but Java doesn't. So you
> > don't
> > > > run
> > > > > > into
> > > > > > > > > this
> > > > > > > > > > problem until runtime.
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > -Xavier
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > On Mon, Jan 27, 2014 at 9:37 AM, Clark Breyman <
> > > > > clark@breyman.com>
> > > > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > Jay -
> > > > > > > > > > >
> > > > > > > > > > > Config - your explanation makes sense. I'm just so
> > > accustomed
> > > > > to
> > > > > > > > having
> > > > > > > > > > > Jackson automatically map my configuration objects to
> > POJOs
> > > > > that
> > > > > > > I've
> > > > > > > > > > > stopped using property files. They are lingua franca.
> The
> > > > only
> > > > > > > > thought
> > > > > > > > > > > might be to separate the config interface from the
> > > > > implementation
> > > > > > > to
> > > > > > > > > > allow
> > > > > > > > > > > for alternatives, but that might undermine your point
> of
> > > "do
> > > > it
> > > > > > > this
> > > > > > > > > way
> > > > > > > > > > so
> > > > > > > > > > > that everyone can find it where they expect it".
> > > > > > > > > > >
> > > > > > > > > > > Serialization: Of the options, I like 1A the best,
> though
> > > > > > possibly
> > > > > > > > with
> > > > > > > > > > > either an option to specify a partition key rather than
> > ID
> > > > or a
> > > > > > > > helper
> > > > > > > > > to
> > > > > > > > > > > translate an arbitrary byte[] or long into a partition
> > > > number.
> > > > > > > > > > >
> > > > > > > > > > > Thanks
> > > > > > > > > > > Clark
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > On Sun, Jan 26, 2014 at 9:13 PM, Jay Kreps <
> > > > > jay.kreps@gmail.com>
> > > > > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > Thanks for the detailed thoughts. Let me elaborate on
> > the
> > > > > > config
> > > > > > > > > thing.
> > > > > > > > > > > >
> > > > > > > > > > > > I agree that at first glance key-value strings don't
> > seem
> > > > > like
> > > > > > a
> > > > > > > > very
> > > > > > > > > > > good
> > > > > > > > > > > > configuration api for a client. Surely a well-typed
> > > config
> > > > > > class
> > > > > > > > > would
> > > > > > > > > > be
> > > > > > > > > > > > better! I actually disagree and let me see if I can
> > > > convince
> > > > > > you.
> > > > > > > > > > > >
> > > > > > > > > > > > My reasoning has nothing to do with the api and
> > > everything
> > > > to
> > > > > > do
> > > > > > > > with
> > > > > > > > > > > > operations.
> > > > > > > > > > > >
> > > > > > > > > > > > Clients are embedded in applications which are
> > themselves
> > > > > > > > configured.
> > > > > > > > > > In
> > > > > > > > > > > > any place that takes operations seriously the
> > > configuration
> > > > > for
> > > > > > > > these
> > > > > > > > > > > > applications will be version controlled and
> maintained
> > > > > through
> > > > > > > some
> > > > > > > > > > kind
> > > > > > > > > > > of
> > > > > > > > > > > > config management system. If we give a config class
> > with
> > > > > > getters
> > > > > > > > and
> > > > > > > > > > > > setters the application has to expose those
> properties
> > to
> > > > its
> > > > > > > > > > > > configuration. What invariably happens is that the
> > > > > application
> > > > > > > > > exposes
> > > > > > > > > > > only
> > > > > > > > > > > > a choice few properties that they thought they would
> > > > change.
> > > > > > > > > > Furthermore
> > > > > > > > > > > > the application will make up a name for these configs
> > > that
> > > > > > seems
> > > > > > > > > > > intuitive
> > > > > > > > > > > > at the time in the 2 seconds the engineer spends
> > thinking
> > > > > about
> > > > > > > it.
> > > > > > > > > > > >
> > > > > > > > > > > > Now consider the result of this in the large. You end
> > up
> > > > with
> > > > > > > > dozens
> > > > > > > > > or
> > > > > > > > > > > > hundreds of applications that have the client
> embedded.
> > > > Each
> > > > > > > > exposes
> > > > > > > > > a
> > > > > > > > > > > > different, inadequate subset of the possible configs,
> > > each
> > > > > with
> > > > > > > > > > different
> > > > > > > > > > > > names. It is a nightmare.
> > > > > > > > > > > >
> > > > > > > > > > > > If you use a string-string map the config system can
> > > > directly
> > > > > > > get a
> > > > > > > > > > > bundle
> > > > > > > > > > > > of config key-value pairs and put them into the
> client.
> > > > This
> > > > > > > means
> > > > > > > > > that
> > > > > > > > > > > all
> > > > > > > > > > > > configuration is automatically available with the
> name
> > > > > > documented
> > > > > > > > on
> > > > > > > > > > the
> > > > > > > > > > > > website in every application that does this. If you
> > > upgrade
> > > > > to
> > > > > > a
> > > > > > > > new
> > > > > > > > > > > kafka
> > > > > > > > > > > > version with more configs those will be exposed too.
> If
> > > you
> > > > > > > realize
> > > > > > > > > > that
> > > > > > > > > > > > you need to change a default you can just go through
> > your
> > > > > > configs
> > > > > > > > and
> > > > > > > > > > > > change it everywhere as it will have the same name
> > > > > everywhere.
> > > > > > > > > > > >
> > > > > > > > > > > > -Jay
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > On Sun, Jan 26, 2014 at 4:47 PM, Clark Breyman <
> > > > > > > clark@breyman.com>
> > > > > > > > > > > wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > > Thanks Jay. I'll see if I can put together a more
> > > > complete
> > > > > > > > > response,
> > > > > > > > > > > > > perhaps as separate threads so that topics don't
> get
> > > > > > entangled.
> > > > > > > > In
> > > > > > > > > > the
> > > > > > > > > > > > mean
> > > > > > > > > > > > > time, here's a couple responses:
> > > > > > > > > > > > >
> > > > > > > > > > > > > Serialization: you've broken out a sub-thread so
> i'll
> > > > reply
> > > > > > > > there.
> > > > > > > > > My
> > > > > > > > > > > > bias
> > > > > > > > > > > > > is that I like generics (except for type-erasure)
> and
> > > in
> > > > > > > > particular
> > > > > > > > > > > they
> > > > > > > > > > > > > make it easy to compose serializers for compound
> > > payloads
> > > > > > (e.g.
> > > > > > > > > when
> > > > > > > > > > a
> > > > > > > > > > > > > common header wraps a payload of parameterized
> type).
> > > > I'll
> > > > > > > > respond
> > > > > > > > > to
> > > > > > > > > > > > your
> > > > > > > > > > > > > 4-options message with an example.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Build: I've seen a lot of "maven-compatible" build
> > > > systems
> > > > > > > > produce
> > > > > > > > > > > > > "artifacts" that aren't really artifacts - no
> > embedded
> > > > POM
> > > > > > or,
> > > > > > > > > worst,
> > > > > > > > > > > > > malformed POM. I know the sbt-generated artifacts
> > were
> > > > this
> > > > > > > way -
> > > > > > > > > > onus
> > > > > > > > > > > is
> > > > > > > > > > > > > on me to see what gradle is spitting out and what a
> > > maven
> > > > > > build
> > > > > > > > > might
> > > > > > > > > > > > look
> > > > > > > > > > > > > like. Maven may be old and boring, but it gets out
> of
> > > the
> > > > > way
> > > > > > > and
> > > > > > > > > > > > > integrates really seamlessly with a lot of IDEs.
> When
> > > > some
> > > > > > > scala
> > > > > > > > > > > > projects I
> > > > > > > > > > > > > was working on in the fall of 2011 switched from
> sbt
> > to
> > > > > > maven,
> > > > > > > > > build
> > > > > > > > > > > > became
> > > > > > > > > > > > > a non-issue.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Config: Not a big deal  and no, I don't think a
> > > > dropwizard
> > > > > > > > > dependency
> > > > > > > > > > > is
> > > > > > > > > > > > > appropriate. I do like using simple entity beans
> > > (POJO's
> > > > > not
> > > > > > > > j2EE)
> > > > > > > > > > for
> > > > > > > > > > > > > configuration, especially if they can be marshalled
> > > > without
> > > > > > > > > > annotation
> > > > > > > > > > > by
> > > > > > > > > > > > > Jackson. I only mentioned the dropwizard-extras
> > >  because
> > > > it
> > > > > > has
> > > > > > > > > some
> > > > > > > > > > > > entity
> > > > > > > > > > > > > bean versions of the ZK and Kafka configs.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Domain-packaging: Also not a big deal - it's what's
> > > > > expected
> > > > > > > and
> > > > > > > > > it's
> > > > > > > > > > > > > pretty free in most IDE's. The advantages I see is
> > that
> > > > it
> > > > > is
> > > > > > > > clear
> > > > > > > > > > > > whether
> > > > > > > > > > > > > something is from the Apache Kafka project and
> > whether
> > > > > > > something
> > > > > > > > is
> > > > > > > > > > > from
> > > > > > > > > > > > > another org and related to Kafka. That said,
> nothing
> > > > really
> > > > > > > > > enforces
> > > > > > > > > > > it.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Futures: I'll see if I can create some examples to
> > > > > > demonstrate
> > > > > > > > > Future
> > > > > > > > > > > > > making interop easier.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Regards,
> > > > > > > > > > > > > C
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Fri, Jan 24, 2014 at 4:36 PM, Jay Kreps <
> > > > > > > jay.kreps@gmail.com>
> > > > > > > > > > > wrote:
> > > > > > > > > > > > >
> > > > > > > > > > > > > > Hey Clark,
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > - Serialization: Yes I agree with these though I
> > > don't
> > > > > > > consider
> > > > > > > > > the
> > > > > > > > > > > > loss
> > > > > > > > > > > > > of
> > > > > > > > > > > > > > generics a big issue. I'll try to summarize what
> I
> > > > would
> > > > > > > > consider
> > > > > > > > > > the
> > > > > > > > > > > > > best
> > > > > > > > > > > > > > alternative api with raw byte[].
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > - Maven: We had this debate a few months back and
> > the
> > > > > > > consensus
> > > > > > > > > was
> > > > > > > > > > > > > gradle.
> > > > > > > > > > > > > > Is there a specific issue with the poms gradle
> > > makes? I
> > > > > am
> > > > > > > > > > extremely
> > > > > > > > > > > > > loath
> > > > > > > > > > > > > > to revisit the issue as build issues are a
> > recurring
> > > > > thing
> > > > > > > and
> > > > > > > > no
> > > > > > > > > > one
> > > > > > > > > > > > > ever
> > > > > > > > > > > > > > agrees and ultimately our build needs are very
> > > simple.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > - Config: I'm not sure if I follow the point. Are
> > you
> > > > > > saying
> > > > > > > we
> > > > > > > > > > > should
> > > > > > > > > > > > > use
> > > > > > > > > > > > > > something in dropwizard for config? One principle
> > > here
> > > > is
> > > > > > to
> > > > > > > > try
> > > > > > > > > to
> > > > > > > > > > > > > remove
> > > > > > > > > > > > > > as many client dependencies as possible as we
> > > > inevitably
> > > > > > run
> > > > > > > > into
> > > > > > > > > > > > > terrible
> > > > > > > > > > > > > > compatibility issues with users who use the same
> > > > library
> > > > > or
> > > > > > > its
> > > > > > > > > > > > > > dependencies at different versions. Or are you
> > > talking
> > > > > > about
> > > > > > > > > > > > maintaining
> > > > > > > > > > > > > > compatibility with existing config parameters? I
> > > think
> > > > as
> > > > > > > much
> > > > > > > > > as a
> > > > > > > > > > > > > config
> > > > > > > > > > > > > > in the existing client makes sense it should have
> > the
> > > > > same
> > > > > > > name
> > > > > > > > > (I
> > > > > > > > > > > was
> > > > > > > > > > > > a
> > > > > > > > > > > > > > bit sloppy about that so I'll fix any errors
> > there).
> > > > > There
> > > > > > > are
> > > > > > > > a
> > > > > > > > > > few
> > > > > > > > > > > > new
> > > > > > > > > > > > > > things and we should give those reasonable
> > defaults.
> > > I
> > > > > > think
> > > > > > > > > config
> > > > > > > > > > > is
> > > > > > > > > > > > > > important so I'll start a thread on the config
> > > package
> > > > in
> > > > > > > > there.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > - org.apache.kafka: We could do this. I always
> > > > considered
> > > > > > it
> > > > > > > > kind
> > > > > > > > > > of
> > > > > > > > > > > an
> > > > > > > > > > > > > odd
> > > > > > > > > > > > > > thing Java programmers do that has no real
> > motivation
> > > > > (but
> > > > > > I
> > > > > > > > > could
> > > > > > > > > > be
> > > > > > > > > > > > > > re-educated!). I don't think it ends up reducing
> > > naming
> > > > > > > > conflicts
> > > > > > > > > > in
> > > > > > > > > > > > > > practice and it adds a lot of noise and nested
> > > > > directories.
> > > > > > > Is
> > > > > > > > > > there
> > > > > > > > > > > a
> > > > > > > > > > > > > > reason you prefer this or just to be more
> standard?
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > - Future: Basically I didn't see any particular
> > > > > advantage.
> > > > > > > The
> > > > > > > > > > > cancel()
> > > > > > > > > > > > > > method doesn't really make sense so probably
> > wouldn't
> > > > > work.
> > > > > > > > > > Likewise
> > > > > > > > > > > I
> > > > > > > > > > > > > > dislike the checked exceptions it requires.
> > > Basically I
> > > > > > just
> > > > > > > > > wrote
> > > > > > > > > > > out
> > > > > > > > > > > > > some
> > > > > > > > > > > > > > code examples and it seemed cleaner with a
> special
> > > > > purpose
> > > > > > > > > object.
> > > > > > > > > > I
> > > > > > > > > > > > > wasn't
> > > > > > > > > > > > > > actually aware of plans for improved futures in
> > java
> > > 8
> > > > or
> > > > > > the
> > > > > > > > > other
> > > > > > > > > > > > > > integrations. Maybe you could elaborate on this a
> > bit
> > > > and
> > > > > > > show
> > > > > > > > > how
> > > > > > > > > > it
> > > > > > > > > > > > > would
> > > > > > > > > > > > > > be used? Sounds promising, I just don't know a
> lot
> > > > about
> > > > > > it.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > -Jay
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On Fri, Jan 24, 2014 at 3:30 PM, Clark Breyman <
> > > > > > > > > clark@breyman.com>
> > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Jay - Thanks for the call for comments. Here's
> > some
> > > > > > initial
> > > > > > > > > > input:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > - Make message serialization a client
> > > responsibility
> > > > > > > (making
> > > > > > > > > all
> > > > > > > > > > > > > messages
> > > > > > > > > > > > > > > byte[]). Reflection-based loading makes it
> harder
> > > to
> > > > > use
> > > > > > > > > generic
> > > > > > > > > > > > codecs
> > > > > > > > > > > > > > > (e.g.  Envelope<PREFIX, DATA, SUFFIX>) or build
> > up
> > > > > codec
> > > > > > > > > > > > > > programmatically.
> > > > > > > > > > > > > > > Non-default partitioning should require an
> > explicit
> > > > > > > partition
> > > > > > > > > > key.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > - I really like the fact that it will be native
> > > Java.
> > > > > > > Please
> > > > > > > > > > > consider
> > > > > > > > > > > > > > using
> > > > > > > > > > > > > > > native maven and not sbt, gradle, ivy, etc as
> > they
> > > > > don't
> > > > > > > > > reliably
> > > > > > > > > > > > play
> > > > > > > > > > > > > > nice
> > > > > > > > > > > > > > > in the maven ecosystem. A jar without a
> > well-formed
> > > > pom
> > > > > > > > doesn't
> > > > > > > > > > > feel
> > > > > > > > > > > > > > like a
> > > > > > > > > > > > > > > real artifact. The pom's generated by sbt et
> al.
> > > are
> > > > > not
> > > > > > > well
> > > > > > > > > > > formed.
> > > > > > > > > > > > > > Using
> > > > > > > > > > > > > > > maven will make builds and IDE integration much
> > > > > smoother.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > - Look at Nick Telford's dropwizard-extras
> > package
> > > in
> > > > > > which
> > > > > > > > he
> > > > > > > > > > > > defines
> > > > > > > > > > > > > > some
> > > > > > > > > > > > > > > Jackson-compatible POJO's for loading
> > > configuration.
> > > > > > Seems
> > > > > > > > like
> > > > > > > > > > > your
> > > > > > > > > > > > > > client
> > > > > > > > > > > > > > > migration is similar. The config objects should
> > > have
> > > > > > > > > constructors
> > > > > > > > > > > or
> > > > > > > > > > > > > > > factories that accept Map<String, String> and
> > > > > Properties
> > > > > > > for
> > > > > > > > > ease
> > > > > > > > > > > of
> > > > > > > > > > > > > > > migration.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > - Would you consider using the org.apache.kafka
> > > > package
> > > > > > for
> > > > > > > > the
> > > > > > > > > > new
> > > > > > > > > > > > API
> > > > > > > > > > > > > > > (quibble)
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > - Why create your own futures rather than use
> > > > > > > > > > > > > > > java.util.concurrent.Future<Long> or similar?
> > > > Standard
> > > > > > > > futures
> > > > > > > > > > will
> > > > > > > > > > > > > play
> > > > > > > > > > > > > > > nice with other reactive libs and things like
> > J8's
> > > > > > > > > > > ComposableFuture.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Thanks again,
> > > > > > > > > > > > > > > C
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > On Fri, Jan 24, 2014 at 2:46 PM, Roger Hoover <
> > > > > > > > > > > > roger.hoover@gmail.com
> > > > > > > > > > > > > > > >wrote:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > A couple comments:
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > 1) Why does the config use a broker list
> > instead
> > > of
> > > > > > > > > discovering
> > > > > > > > > > > the
> > > > > > > > > > > > > > > brokers
> > > > > > > > > > > > > > > > in ZooKeeper?  It doesn't match the
> > > > HighLevelConsumer
> > > > > > > API.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > 2) It looks like broker connections are
> created
> > > on
> > > > > > > demand.
> > > > > > > > >  I'm
> > > > > > > > > > > > > > wondering
> > > > > > > > > > > > > > > > if sometimes you might want to flush out
> config
> > > or
> > > > > > > network
> > > > > > > > > > > > > connectivity
> > > > > > > > > > > > > > > > issues before pushing the first message
> > through.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Should there also be a
> KafkaProducer.connect()
> > or
> > > > > > .open()
> > > > > > > > > > method
> > > > > > > > > > > or
> > > > > > > > > > > > > > > > connectAll()?  I guess it would try to
> connect
> > to
> > > > all
> > > > > > > > brokers
> > > > > > > > > > in
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > BROKER_LIST_CONFIG
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > HTH,
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Roger
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > On Fri, Jan 24, 2014 at 11:54 AM, Jay Kreps <
> > > > > > > > > > jay.kreps@gmail.com
> > > > > > > > > > > >
> > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > As mentioned in a previous email we are
> > working
> > > > on
> > > > > a
> > > > > > > > > > > > > > re-implementation
> > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > the producer. I would like to use this
> email
> > > > thread
> > > > > > to
> > > > > > > > > > discuss
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > details
> > > > > > > > > > > > > > > > > of the public API and the configuration. I
> > > would
> > > > > love
> > > > > > > for
> > > > > > > > > us
> > > > > > > > > > to
> > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > incredibly picky about this public api now
> so
> > > it
> > > > is
> > > > > > as
> > > > > > > > good
> > > > > > > > > > as
> > > > > > > > > > > > > > possible
> > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > we don't need to break it in the future.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > The best way to get a feel for the API is
> > > > actually
> > > > > to
> > > > > > > > take
> > > > > > > > > a
> > > > > > > > > > > look
> > > > > > > > > > > > > at
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > javadoc, my hope is to get the api docs
> good
> > > > enough
> > > > > > so
> > > > > > > > that
> > > > > > > > > > it
> > > > > > > > > > > is
> > > > > > > > > > > > > > > > > self-explanatory:
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> http://empathybox.com/kafka-javadoc/index.html?kafka/clients/producer/KafkaProducer.html
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Please take a look at this API and give me
> > any
> > > > > > thoughts
> > > > > > > > you
> > > > > > > > > > may
> > > > > > > > > > > > > have!
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > It may also be reasonable to take a look at
> > the
> > > > > > > configs:
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> http://empathybox.com/kafka-javadoc/kafka/clients/producer/ProducerConfig.html
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > The actual code is posted here:
> > > > > > > > > > > > > > > > >
> > > https://issues.apache.org/jira/browse/KAFKA-1227
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > A few questions or comments to kick things
> > off:
> > > > > > > > > > > > > > > > > 1. We need to make a decision on whether
> > > > > > serialization
> > > > > > > of
> > > > > > > > > the
> > > > > > > > > > > > > user's
> > > > > > > > > > > > > > > key
> > > > > > > > > > > > > > > > > and value should be done by the user (with
> > our
> > > > api
> > > > > > just
> > > > > > > > > > taking
> > > > > > > > > > > > > > byte[])
> > > > > > > > > > > > > > > or
> > > > > > > > > > > > > > > > > if we should take an object and allow the
> > user
> > > to
> > > > > > > > > configure a
> > > > > > > > > > > > > > > Serializer
> > > > > > > > > > > > > > > > > class which we instantiate via reflection.
> We
> > > > take
> > > > > > the
> > > > > > > > > later
> > > > > > > > > > > > > approach
> > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > the current producer, and I have carried
> this
> > > > > through
> > > > > > > to
> > > > > > > > > this
> > > > > > > > > > > > > > > prototype.
> > > > > > > > > > > > > > > > > The tradeoff I see is this: taking byte[]
> is
> > > > > actually
> > > > > > > > > > simpler,
> > > > > > > > > > > > the
> > > > > > > > > > > > > > user
> > > > > > > > > > > > > > > > can
> > > > > > > > > > > > > > > > > directly do whatever serialization they
> like.
> > > The
> > > > > > > > > > complication
> > > > > > > > > > > is
> > > > > > > > > > > > > > > > actually
> > > > > > > > > > > > > > > > > partitioning. Currently partitioning is
> done
> > > by a
> > > > > > > similar
> > > > > > > > > > > plug-in
> > > > > > > > > > > > > api
> > > > > > > > > > > > > > > > > (Partitioner) which the user can implement
> > and
> > > > > > > configure
> > > > > > > > to
> > > > > > > > > > > > > override
> > > > > > > > > > > > > > > how
> > > > > > > > > > > > > > > > > partitions are assigned. If we take byte[]
> as
> > > > input
> > > > > > > then
> > > > > > > > we
> > > > > > > > > > > have
> > > > > > > > > > > > no
> > > > > > > > > > > > > > > > access
> > > > > > > > > > > > > > > > > to the original object and partitioning
> MUST
> > be
> > > > > done
> > > > > > on
> > > > > > > > the
> > > > > > > > > > > > byte[].
> > > > > > > > > > > > > > > This
> > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > fine for hash partitioning. However for
> > various
> > > > > types
> > > > > > > of
> > > > > > > > > > > semantic
> > > > > > > > > > > > > > > > > partitioning (range partitioning, or
> > whatever)
> > > > you
> > > > > > > would
> > > > > > > > > want
> > > > > > > > > > > > > access
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > original object. In the current approach a
> > > > producer
> > > > > > who
> > > > > > > > > > wishes
> > > > > > > > > > > to
> > > > > > > > > > > > > > send
> > > > > > > > > > > > > > > > > byte[] they have serialized in their own
> code
> > > can
> > > > > > > > configure
> > > > > > > > > > the
> > > > > > > > > > > > > > > > > BytesSerialization we supply which is just
> a
> > > "no
> > > > > op"
> > > > > > > > > > > > serialization.
> > > > > > > > > > > > > > > > > 2. We should obsess over naming and make
> sure
> > > > each
> > > > > of
> > > > > > > the
> > > > > > > > > > class
> > > > > > > > > > > > > names
> > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > good.
> > > > > > > > > > > > > > > > > 3. Jun has already pointed out that we need
> > to
> > > > > > include
> > > > > > > > the
> > > > > > > > > > > topic
> > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > partition in the response, which is
> > absolutely
> > > > > > right. I
> > > > > > > > > > haven't
> > > > > > > > > > > > > done
> > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > yet but that definitely needs to be there.
> > > > > > > > > > > > > > > > > 4. Currently RecordSend.await will throw an
> > > > > exception
> > > > > > > if
> > > > > > > > > the
> > > > > > > > > > > > > request
> > > > > > > > > > > > > > > > > failed. The intention here is that
> > > > > > > > > > > producer.send(message).await()
> > > > > > > > > > > > > > > exactly
> > > > > > > > > > > > > > > > > simulates a synchronous call. Guozhang has
> > > noted
> > > > > that
> > > > > > > > this
> > > > > > > > > > is a
> > > > > > > > > > > > > > little
> > > > > > > > > > > > > > > > > annoying since the user must then catch
> > > > exceptions.
> > > > > > > > However
> > > > > > > > > > if
> > > > > > > > > > > we
> > > > > > > > > > > > > > > remove
> > > > > > > > > > > > > > > > > this then if the user doesn't check for
> > errors
> > > > they
> > > > > > > won't
> > > > > > > > > > know
> > > > > > > > > > > > one
> > > > > > > > > > > > > > has
> > > > > > > > > > > > > > > > > occurred, which I predict will be a common
> > > > mistake.
> > > > > > > > > > > > > > > > > 5. Perhaps there is more we could do to
> make
> > > the
> > > > > > async
> > > > > > > > > > > callbacks
> > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > future
> > > > > > > > > > > > > > > > > we give back intuitive and easy to program
> > > > against?
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Some background info on implementation:
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > At a high level the primary difference in
> > this
> > > > > > producer
> > > > > > > > is
> > > > > > > > > > that
> > > > > > > > > > > > it
> > > > > > > > > > > > > > > > removes
> > > > > > > > > > > > > > > > > the distinction between the "sync" and
> > "async"
> > > > > > > producer.
> > > > > > > > > > > > > Effectively
> > > > > > > > > > > > > > > all
> > > > > > > > > > > > > > > > > requests are sent asynchronously but always
> > > > return
> > > > > a
> > > > > > > > future
> > > > > > > > > > > > > response
> > > > > > > > > > > > > > > > object
> > > > > > > > > > > > > > > > > that gives the offset as well as any error
> > that
> > > > may
> > > > > > > have
> > > > > > > > > > > occurred
> > > > > > > > > > > > > > when
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > request is complete. The batching that is
> > done
> > > in
> > > > > the
> > > > > > > > async
> > > > > > > > > > > > > producer
> > > > > > > > > > > > > > > only
> > > > > > > > > > > > > > > > > today is done whenever possible now. This
> > means
> > > > > that
> > > > > > > the
> > > > > > > > > sync
> > > > > > > > > > > > > > producer,
> > > > > > > > > > > > > > > > > under load, can get performance as good as
> > the
> > > > > async
> > > > > > > > > producer
> > > > > > > > > > > > > > > > (preliminary
> > > > > > > > > > > > > > > > > results show the producer getting 1m
> > > > messages/sec).
> > > > > > > This
> > > > > > > > > > works
> > > > > > > > > > > > > > similar
> > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > group commit in databases but with respect
> to
> > > the
> > > > > > > actual
> > > > > > > > > > > network
> > > > > > > > > > > > > > > > > transmission--any messages that arrive
> while
> > a
> > > > send
> > > > > > is
> > > > > > > in
> > > > > > > > > > > > progress
> > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > batched together. It is also possible to
> > > > encourage
> > > > > > > > batching
> > > > > > > > > > > even
> > > > > > > > > > > > > > under
> > > > > > > > > > > > > > > > low
> > > > > > > > > > > > > > > > > load to save server resources by
> introducing
> > a
> > > > > delay
> > > > > > on
> > > > > > > > the
> > > > > > > > > > > send
> > > > > > > > > > > > to
> > > > > > > > > > > > > > > allow
> > > > > > > > > > > > > > > > > more messages to accumulate; this is done
> > using
> > > > the
> > > > > > > > > > > > linger.msconfig
> > > > > > > > > > > > > > > > (this
> > > > > > > > > > > > > > > > > is similar to Nagle's algorithm in TCP).
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > This producer does all network
> communication
> > > > > > > > asynchronously
> > > > > > > > > > and
> > > > > > > > > > > > in
> > > > > > > > > > > > > > > > parallel
> > > > > > > > > > > > > > > > > to all servers so the performance penalty
> for
> > > > > acks=-1
> > > > > > > and
> > > > > > > > > > > waiting
> > > > > > > > > > > > > on
> > > > > > > > > > > > > > > > > replication should be much reduced. I
> haven't
> > > > done
> > > > > > much
> > > > > > > > > > > > > benchmarking
> > > > > > > > > > > > > > on
> > > > > > > > > > > > > > > > > this yet, though.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > The high level design is described a little
> > > here,
> > > > > > > though
> > > > > > > > > this
> > > > > > > > > > > is
> > > > > > > > > > > > > now
> > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > little out of date:
> > > > > > > > > > > > > > > > >
> > > > > > > > > > >
> > > > > https://cwiki.apache.org/confluence/display/KAFKA/Client+Rewrite
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > -Jay
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: New Producer Public API

Posted by Jay Kreps <ja...@gmail.com>.
Yes, we will absolutely retain protocol compatibility with 0.8 though the
java api will change. The prototype code I posted works with 0.8.

-Jay


On Wed, Jan 29, 2014 at 10:19 AM, Steve Morin <st...@gmail.com> wrote:

> Is the new producer API going to maintain protocol compatibility with old
> version if the API under the hood?
>
> > On Jan 29, 2014, at 10:15, Jay Kreps <ja...@gmail.com> wrote:
> >
> > The challenge of directly exposing ProduceRequestResult is that the
> offset
> > provided is just the base offset and there is no way to know for a
> > particular message where it was in relation to that base offset because
> the
> > batching is transparent and non-deterministic. So I think we do need some
> > kind of per-message result.
> >
> > I started with Future<RequestResult>, I think for the same reason you
> > prefer it but then when I actually looked at some code samples it wasn't
> > too great--checked exceptions, methods that we can't easily implement,
> etc.
> > I moved away from that for two reasons:
> > 1. When I actually wrote out some code samples of usage they were a
> little
> > ugly for the reasons I described--checked exceptions, methods we can't
> > implement, no helper methods, etc.
> > 2. I originally intended to make the result send work like a
> > ListenableFuture so that you would register the callback on the result
> > rather than as part of the call. I moved away from this primarily because
> > the implementation complexity was a little higher.
> >
> > Whether or not the code prettiness on its own outweighs the familiarity
> of
> > a normal Future I don't know, but that was the evolution of my thinking.
> >
> > -Jay
> >
> >
> >> On Wed, Jan 29, 2014 at 10:06 AM, Jay Kreps <ja...@gmail.com>
> wrote:
> >>
> >> Hey Neha,
> >>
> >> Error handling in RecordSend works as in Future you will get the
> exception
> >> if there is one from any of the accessor methods or await().
> >>
> >> The purpose of hasError was that you can write things slightly more
> simply
> >> (which some people expressed preference for):
> >>  if(send.hasError())
> >>    // do something
> >>  long offset = send.offset();
> >>
> >> Instead of the more the slightly longer:
> >> try {
> >>   long offset = send.offset();
> >> } catch (KafkaException e) {
> >>   // do something
> >> }
> >>
> >>
> >> On Wed, Jan 29, 2014 at 10:01 AM, Neha Narkhede <
> neha.narkhede@gmail.com>wrote:
> >>
> >>> Regarding the use of Futures -
> >>>
> >>> Agree that there are some downsides to using Futures but both
> approaches
> >>> have some tradeoffs.
> >>>
> >>> - Standardization and usability
> >>> Future is a widely used and understood Java API and given that the
> >>> functionality that RecordSend hopes to provide is essentially that of
> >>> Future, I think it makes sense to expose a widely understood public API
> >>> for
> >>> our clients. RecordSend, on the other hand, seems to provide some APIs
> >>> that
> >>> are very similar to that of Future, in addition to exposing a bunch of
> >>> APIs
> >>> that belong to ProduceRequestResult. As a user, I would've really
> >>> preferred
> >>> to deal with ProduceRequestResult directly -
> >>> Future<ProduceRequestResult> send(...)
> >>>
> >>> - Error handling
> >>> RecordSend's error handling is quite unintuitive where the user has to
> >>> remember to invoke hasError and error, instead of just throwing the
> >>> exception. Now there are
> >>> some downsides regarding error handling with the Future as well, where
> the
> >>> user has to catch InterruptedException when we would never run into it.
> >>> However, it seems like a price worth paying for supporting a standard
> API
> >>> and error handling
> >>>
> >>> - Unused APIs
> >>> This is a downside of using Future, where the cancel() operation would
> >>> always return false and mean nothing. But we can mention that caveat in
> >>> our
> >>> Java docs.
> >>>
> >>> To summarize, I would prefer to expose a well understood and widely
> >>> adopted
> >>> Java API and put up with the overhead of catching one unnecessary
> checked
> >>> exception, rather than wrap the useful ProduceRequestResult in a custom
> >>> async object (RecordSend) and explain that to our many users.
> >>>
> >>> Thanks,
> >>> Neha
> >>>
> >>>
> >>>
> >>>
> >>>> On Tue, Jan 28, 2014 at 8:10 PM, Jay Kreps <ja...@gmail.com>
> wrote:
> >>>>
> >>>> Hey Neha,
> >>>>
> >>>> Can you elaborate on why you prefer using Java's Future? The downside
> >>> in my
> >>>> mind is the use of the checked InterruptedException and
> >>> ExecutionException.
> >>>> ExecutionException is arguable, but forcing you to catch
> >>>> InterruptedException, often in code that can't be interrupted, seems
> >>>> perverse. It also leaves us with the cancel() method which I don't
> >>> think we
> >>>> really can implement.
> >>>>
> >>>> Option 1A, to recap/elaborate, was the following. There is no
> >>> Serializer or
> >>>> Partitioner api. We take a byte[] key and value and an optional
> integer
> >>>> partition. If you specify the integer partition it will be used. If
> you
> >>> do
> >>>> not specify a key or a partition the partition will be chosen in a
> round
> >>>> robin fashion. If you specify a key but no partition we will chose a
> >>>> partition based on a hash of the key. In order to let the user find
> the
> >>>> partition we will need to given them access to the Cluster instance
> >>>> directly from the producer.
> >>>>
> >>>> -Jay
> >>>>
> >>>>
> >>>> On Tue, Jan 28, 2014 at 6:25 PM, Neha Narkhede <
> neha.narkhede@gmail.com
> >>>>> wrote:
> >>>>
> >>>>> Here are more thoughts on the public APIs -
> >>>>>
> >>>>> - I suggest we use java's Future instead of custom Future especially
> >>>> since
> >>>>> it is part of the public API
> >>>>>
> >>>>> - Serialization: I like the simplicity of the producer APIs with the
> >>>>> absence of serialization where we just deal with byte arrays for keys
> >>> and
> >>>>> values. What I don't like about this is the performance overhead on
> >>> the
> >>>>> Partitioner for any kind of custom partitioning based on the
> >>>> partitionKey.
> >>>>> Since the only purpose of partitionKey is to do custom partitioning,
> >>> why
> >>>>> can't we take it in directly as an integer and let the user figure
> out
> >>>> the
> >>>>> mapping from partition_key -> partition_id using the getCluster()
> API?
> >>>> If I
> >>>>> understand correctly, this is similar to what you suggested as part
> of
> >>>>> option 1A. I like this approach since it maintains the simplicity of
> >>> APIs
> >>>>> by allowing us to deal with bytes and does not compromise performance
> >>> in
> >>>>> the custom partitioning case.
> >>>>>
> >>>>> Thanks,
> >>>>> Neha
> >>>>>
> >>>>>
> >>>>>
> >>>>> On Tue, Jan 28, 2014 at 5:42 PM, Jay Kreps <ja...@gmail.com>
> >>> wrote:
> >>>>>
> >>>>>> Hey Tom,
> >>>>>>
> >>>>>> That sounds cool. How did you end up handling parallel I/O if you
> >>> wrap
> >>>>> the
> >>>>>> individual connections? Don't you need some selector that selects
> >>> over
> >>>>> all
> >>>>>> the connections?
> >>>>>>
> >>>>>> -Jay
> >>>>>>
> >>>>>>
> >>>>>> On Tue, Jan 28, 2014 at 2:31 PM, Tom Brown <to...@gmail.com>
> >>>> wrote:
> >>>>>>
> >>>>>>> I implemented a 0.7 client in pure java, and its API very closely
> >>>>>> resembled
> >>>>>>> this. (When multiple people independently engineer the same
> >>> solution,
> >>>>>> it's
> >>>>>>> probably good... right?). However, there were a few architectural
> >>>>>>> differences with my client:
> >>>>>>>
> >>>>>>> 1. The basic client itself was just an asynchronous layer around
> >>> the
> >>>>>>> different server functions. In and of itself it had no knowledge
> >>> of
> >>>>>>> partitions, only servers (and maintained TCP connections to them).
> >>>>>>>
> >>>>>>> 2. The main producer was an additional layer that provided a
> >>>> high-level
> >>>>>>> interface that could batch individual messages based on partition.
> >>>>>>>
> >>>>>>> 3. Knowledge of partitioning was done via an interface so that
> >>>>> different
> >>>>>>> strategies could be used.
> >>>>>>>
> >>>>>>> 4. Partitioning was done by the user, with knowledge of the
> >>> available
> >>>>>>> partitions provided by #3.
> >>>>>>>
> >>>>>>> 5. Serialization was done by the user to simplify the API.
> >>>>>>>
> >>>>>>> 6. Futures were used to make asynchronous emulate synchronous
> >>> calls.
> >>>>>>>
> >>>>>>>
> >>>>>>> The main benefit of this approach is flexibility. For example,
> >>> since
> >>>>> the
> >>>>>>> base client was just a managed connection (and not inherently a
> >>>>>> producer),
> >>>>>>> it was easy to composite a produce request and an offsets request
> >>>>>> together
> >>>>>>> into a confirmed produce request (officially not available in
> >>> 0.7).
> >>>>>>>
> >>>>>>> Decoupling the basic client from partition management allowed the
> >>> me
> >>>> to
> >>>>>>> implement zk discovery as a separate project so that the main
> >>> project
> >>>>> had
> >>>>>>> no complex dependencies. The same was true of decoupling
> >>>> serialization.
> >>>>>>> It's trivial to build an optional layer that adds those features
> >>> in,
> >>>>>> while
> >>>>>>> allowing access to the base APIs for those that need it.
> >>>>>>>
> >>>>>>> Using standard Future objects was also beneficial, since I could
> >>>>> combine
> >>>>>>> them with existing tools (such as guava).
> >>>>>>>
> >>>>>>> It may be too late to be of use, but I have been working with my
> >>>>>> company's
> >>>>>>> legal department to release the implementation I described above.
> >>> If
> >>>>>> you're
> >>>>>>> interested in it, let me know.
> >>>>>>>
> >>>>>>>
> >>>>>>> To sum up my thoughts regarding the new API, I think it's a great
> >>>>> start.
> >>>>>> I
> >>>>>>> would like to see a more layered approach so I can use the parts I
> >>>>> want,
> >>>>>>> and adapt the other parts as needed. I would also like to see
> >>>> standard
> >>>>>>> interfaces (especially Future) used where they makes sense.
> >>>>>>>
> >>>>>>> --Tom
> >>>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>> On Tue, Jan 28, 2014 at 1:33 PM, Roger Hoover <
> >>>> roger.hoover@gmail.com
> >>>>>>>> wrote:
> >>>>>>>
> >>>>>>>> +1 ListenableFuture: If this works similar to Deferreds in
> >>> Twisted
> >>>>>> Python
> >>>>>>>> or Promised IO in Javascript, I think this is a great pattern
> >>> for
> >>>>>>>> decoupling your callback logic from the place where the Future
> >>> is
> >>>>>>>> generated.  You can register as many callbacks as you like,
> >>> each in
> >>>>> the
> >>>>>>>> appropriate layer of the code and have each observer get
> >>> notified
> >>>>> when
> >>>>>>> the
> >>>>>>>> promised i/o is complete without any of them knowing about each
> >>>>> other.
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> On Tue, Jan 28, 2014 at 11:32 AM, Jay Kreps <
> >>> jay.kreps@gmail.com>
> >>>>>> wrote:
> >>>>>>>>
> >>>>>>>>> Hey Ross,
> >>>>>>>>>
> >>>>>>>>> - ListenableFuture: Interesting. That would be an alternative
> >>> to
> >>>>> the
> >>>>>>>> direct
> >>>>>>>>> callback support we provide. There could be pros to this, let
> >>> me
> >>>>>> think
> >>>>>>>>> about it.
> >>>>>>>>> - We could provide layering, but I feel that the
> >>> serialization is
> >>>>>> such
> >>>>>>> a
> >>>>>>>>> small thing we should just make a decision and chose one, it
> >>>>> doesn't
> >>>>>>> seem
> >>>>>>>>> to me to justify a whole public facing layer.
> >>>>>>>>> - Yes, this is fairly esoteric, essentially I think it is
> >>> fairly
> >>>>>>> similar
> >>>>>>>> to
> >>>>>>>>> databases like DynamoDB that allow you to specify two
> >>> partition
> >>>>> keys
> >>>>>> (I
> >>>>>>>>> think DynamoDB does this...). The reasoning is that in fact
> >>> there
> >>>>> are
> >>>>>>>>> several things you can use the key field for: (1) to compute
> >>> the
> >>>>>>>> partition
> >>>>>>>>> to store the data in, (2) as a unique identifier to
> >>> deduplicate
> >>>>> that
> >>>>>>>>> partition's records within a log. These two things are almost
> >>>>> always
> >>>>>>> the
> >>>>>>>>> same, but occationally may differ when you want to group data
> >>> in
> >>>> a
> >>>>>> more
> >>>>>>>>> sophisticated way then just a hash of the primary key but
> >>> still
> >>>>>> retain
> >>>>>>>> the
> >>>>>>>>> proper primary key for delivery to the consumer and log
> >>>> compaction.
> >>>>>>>>>
> >>>>>>>>> -Jay
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> On Tue, Jan 28, 2014 at 3:24 AM, Ross Black <
> >>>>> ross.w.black@gmail.com>
> >>>>>>>>> wrote:
> >>>>>>>>>
> >>>>>>>>>> Hi Jay,
> >>>>>>>>>>
> >>>>>>>>>> - Just to add some more info/confusion about possibly using
> >>>>> Future
> >>>>>>> ...
> >>>>>>>>>>  If Kafka uses a JDK future, it plays nicely with other
> >>>>> frameworks
> >>>>>>> as
> >>>>>>>>>> well.
> >>>>>>>>>>  Google Guava has a ListenableFuture that allows callback
> >>>>> handling
> >>>>>>> to
> >>>>>>>> be
> >>>>>>>>>> added via the returned future, and allows the callbacks to
> >>> be
> >>>>>> passed
> >>>>>>>> off
> >>>>>>>>> to
> >>>>>>>>>> a specified executor.
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>
> >>>>
> >>>
> http://docs.guava-libraries.googlecode.com/git-history/release/javadoc/com/google/common/util/concurrent/ListenableFuture.html
> >>>>>>>>>>  The JDK future can easily be converted to a listenable
> >>>> future.
> >>>>>>>>>>
> >>>>>>>>>> - On the question of byte[] vs Object, could this be solved
> >>> by
> >>>>>>> layering
> >>>>>>>>> the
> >>>>>>>>>> API?  eg. a raw producer (use byte[] and specify the
> >>> partition
> >>>>>>> number)
> >>>>>>>>> and
> >>>>>>>>>> a normal producer (use generic object and specify a
> >>>> Partitioner)?
> >>>>>>>>>>
> >>>>>>>>>> - I am confused by the keys in ProducerRecord and
> >>> Partitioner.
> >>>>>> What
> >>>>>>> is
> >>>>>>>>> the
> >>>>>>>>>> usage for both a key and a partition key? (I am not yet
> >>> using
> >>>>> 0.8)
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> Thanks,
> >>>>>>>>>> Ross
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> On 28 January 2014 05:00, Xavier Stevens <xavier@gaikai.com
> >>>>
> >>>>>> wrote:
> >>>>>>>>>>
> >>>>>>>>>>> AutoCloseable would be nice for us as most of our code is
> >>>> using
> >>>>>>> Java
> >>>>>>>> 7
> >>>>>>>>> at
> >>>>>>>>>>> this point.
> >>>>>>>>>>>
> >>>>>>>>>>> I like Dropwizard's configuration mapping to POJOs via
> >>>> Jackson,
> >>>>>> but
> >>>>>>>> if
> >>>>>>>>>> you
> >>>>>>>>>>> wanted to stick with property maps I don't care enough to
> >>>>> object.
> >>>>>>>>>>>
> >>>>>>>>>>> If the producer only dealt with bytes, is there a way we
> >>>> could
> >>>>>>> still
> >>>>>>>>> due
> >>>>>>>>>>> partition plugins without specifying the number
> >>> explicitly? I
> >>>>>> would
> >>>>>>>>>> prefer
> >>>>>>>>>>> to be able to pass in field(s) that would be used by the
> >>>>>>> partitioner.
> >>>>>>>>>>> Obviously if this wasn't possible you could always
> >>>> deserialize
> >>>>>> the
> >>>>>>>>> object
> >>>>>>>>>>> in the partitioner and grab the fields you want, but that
> >>>> seems
> >>>>>>>> really
> >>>>>>>>>>> expensive to do on every message.
> >>>>>>>>>>>
> >>>>>>>>>>> It would also be nice to have a Java API Encoder
> >>> constructor
> >>>>>> taking
> >>>>>>>> in
> >>>>>>>>>>> VerifiableProperties. Scala understands how to handle
> >>> "props:
> >>>>>>>>>>> VerifiableProperties = null", but Java doesn't. So you
> >>> don't
> >>>>> run
> >>>>>>> into
> >>>>>>>>>> this
> >>>>>>>>>>> problem until runtime.
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> -Xavier
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> On Mon, Jan 27, 2014 at 9:37 AM, Clark Breyman <
> >>>>>> clark@breyman.com>
> >>>>>>>>>> wrote:
> >>>>>>>>>>>
> >>>>>>>>>>>> Jay -
> >>>>>>>>>>>>
> >>>>>>>>>>>> Config - your explanation makes sense. I'm just so
> >>>> accustomed
> >>>>>> to
> >>>>>>>>> having
> >>>>>>>>>>>> Jackson automatically map my configuration objects to
> >>> POJOs
> >>>>>> that
> >>>>>>>> I've
> >>>>>>>>>>>> stopped using property files. They are lingua franca.
> >>> The
> >>>>> only
> >>>>>>>>> thought
> >>>>>>>>>>>> might be to separate the config interface from the
> >>>>>> implementation
> >>>>>>>> to
> >>>>>>>>>>> allow
> >>>>>>>>>>>> for alternatives, but that might undermine your point of
> >>>> "do
> >>>>> it
> >>>>>>>> this
> >>>>>>>>>> way
> >>>>>>>>>>> so
> >>>>>>>>>>>> that everyone can find it where they expect it".
> >>>>>>>>>>>>
> >>>>>>>>>>>> Serialization: Of the options, I like 1A the best,
> >>> though
> >>>>>>> possibly
> >>>>>>>>> with
> >>>>>>>>>>>> either an option to specify a partition key rather than
> >>> ID
> >>>>> or a
> >>>>>>>>> helper
> >>>>>>>>>> to
> >>>>>>>>>>>> translate an arbitrary byte[] or long into a partition
> >>>>> number.
> >>>>>>>>>>>>
> >>>>>>>>>>>> Thanks
> >>>>>>>>>>>> Clark
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> On Sun, Jan 26, 2014 at 9:13 PM, Jay Kreps <
> >>>>>> jay.kreps@gmail.com>
> >>>>>>>>>> wrote:
> >>>>>>>>>>>>
> >>>>>>>>>>>>> Thanks for the detailed thoughts. Let me elaborate on
> >>> the
> >>>>>>> config
> >>>>>>>>>> thing.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> I agree that at first glance key-value strings don't
> >>> seem
> >>>>>> like
> >>>>>>> a
> >>>>>>>>> very
> >>>>>>>>>>>> good
> >>>>>>>>>>>>> configuration api for a client. Surely a well-typed
> >>>> config
> >>>>>>> class
> >>>>>>>>>> would
> >>>>>>>>>>> be
> >>>>>>>>>>>>> better! I actually disagree and let me see if I can
> >>>>> convince
> >>>>>>> you.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> My reasoning has nothing to do with the api and
> >>>> everything
> >>>>> to
> >>>>>>> do
> >>>>>>>>> with
> >>>>>>>>>>>>> operations.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Clients are embedded in applications which are
> >>> themselves
> >>>>>>>>> configured.
> >>>>>>>>>>> In
> >>>>>>>>>>>>> any place that takes operations seriously the
> >>>> configuration
> >>>>>> for
> >>>>>>>>> these
> >>>>>>>>>>>>> applications will be version controlled and maintained
> >>>>>> through
> >>>>>>>> some
> >>>>>>>>>>> kind
> >>>>>>>>>>>> of
> >>>>>>>>>>>>> config management system. If we give a config class
> >>> with
> >>>>>>> getters
> >>>>>>>>> and
> >>>>>>>>>>>>> setters the application has to expose those
> >>> properties to
> >>>>> its
> >>>>>>>>>>>>> configuration. What invariably happens is that the
> >>>>>> application
> >>>>>>>>>> exposes
> >>>>>>>>>>>> only
> >>>>>>>>>>>>> a choice few properties that they thought they would
> >>>>> change.
> >>>>>>>>>>> Furthermore
> >>>>>>>>>>>>> the application will make up a name for these configs
> >>>> that
> >>>>>>> seems
> >>>>>>>>>>>> intuitive
> >>>>>>>>>>>>> at the time in the 2 seconds the engineer spends
> >>> thinking
> >>>>>> about
> >>>>>>>> it.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Now consider the result of this in the large. You end
> >>> up
> >>>>> with
> >>>>>>>>> dozens
> >>>>>>>>>> or
> >>>>>>>>>>>>> hundreds of applications that have the client
> >>> embedded.
> >>>>> Each
> >>>>>>>>> exposes
> >>>>>>>>>> a
> >>>>>>>>>>>>> different, inadequate subset of the possible configs,
> >>>> each
> >>>>>> with
> >>>>>>>>>>> different
> >>>>>>>>>>>>> names. It is a nightmare.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> If you use a string-string map the config system can
> >>>>> directly
> >>>>>>>> get a
> >>>>>>>>>>>> bundle
> >>>>>>>>>>>>> of config key-value pairs and put them into the
> >>> client.
> >>>>> This
> >>>>>>>> means
> >>>>>>>>>> that
> >>>>>>>>>>>> all
> >>>>>>>>>>>>> configuration is automatically available with the name
> >>>>>>> documented
> >>>>>>>>> on
> >>>>>>>>>>> the
> >>>>>>>>>>>>> website in every application that does this. If you
> >>>> upgrade
> >>>>>> to
> >>>>>>> a
> >>>>>>>>> new
> >>>>>>>>>>>> kafka
> >>>>>>>>>>>>> version with more configs those will be exposed too.
> >>> If
> >>>> you
> >>>>>>>> realize
> >>>>>>>>>>> that
> >>>>>>>>>>>>> you need to change a default you can just go through
> >>> your
> >>>>>>> configs
> >>>>>>>>> and
> >>>>>>>>>>>>> change it everywhere as it will have the same name
> >>>>>> everywhere.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> -Jay
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> On Sun, Jan 26, 2014 at 4:47 PM, Clark Breyman <
> >>>>>>>> clark@breyman.com>
> >>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>> Thanks Jay. I'll see if I can put together a more
> >>>>> complete
> >>>>>>>>>> response,
> >>>>>>>>>>>>>> perhaps as separate threads so that topics don't get
> >>>>>>> entangled.
> >>>>>>>>> In
> >>>>>>>>>>> the
> >>>>>>>>>>>>> mean
> >>>>>>>>>>>>>> time, here's a couple responses:
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Serialization: you've broken out a sub-thread so
> >>> i'll
> >>>>> reply
> >>>>>>>>> there.
> >>>>>>>>>> My
> >>>>>>>>>>>>> bias
> >>>>>>>>>>>>>> is that I like generics (except for type-erasure)
> >>> and
> >>>> in
> >>>>>>>>> particular
> >>>>>>>>>>>> they
> >>>>>>>>>>>>>> make it easy to compose serializers for compound
> >>>> payloads
> >>>>>>> (e.g.
> >>>>>>>>>> when
> >>>>>>>>>>> a
> >>>>>>>>>>>>>> common header wraps a payload of parameterized
> >>> type).
> >>>>> I'll
> >>>>>>>>> respond
> >>>>>>>>>> to
> >>>>>>>>>>>>> your
> >>>>>>>>>>>>>> 4-options message with an example.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Build: I've seen a lot of "maven-compatible" build
> >>>>> systems
> >>>>>>>>> produce
> >>>>>>>>>>>>>> "artifacts" that aren't really artifacts - no
> >>> embedded
> >>>>> POM
> >>>>>>> or,
> >>>>>>>>>> worst,
> >>>>>>>>>>>>>> malformed POM. I know the sbt-generated artifacts
> >>> were
> >>>>> this
> >>>>>>>> way -
> >>>>>>>>>>> onus
> >>>>>>>>>>>> is
> >>>>>>>>>>>>>> on me to see what gradle is spitting out and what a
> >>>> maven
> >>>>>>> build
> >>>>>>>>>> might
> >>>>>>>>>>>>> look
> >>>>>>>>>>>>>> like. Maven may be old and boring, but it gets out
> >>> of
> >>>> the
> >>>>>> way
> >>>>>>>> and
> >>>>>>>>>>>>>> integrates really seamlessly with a lot of IDEs.
> >>> When
> >>>>> some
> >>>>>>>> scala
> >>>>>>>>>>>>> projects I
> >>>>>>>>>>>>>> was working on in the fall of 2011 switched from
> >>> sbt to
> >>>>>>> maven,
> >>>>>>>>>> build
> >>>>>>>>>>>>> became
> >>>>>>>>>>>>>> a non-issue.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Config: Not a big deal  and no, I don't think a
> >>>>> dropwizard
> >>>>>>>>>> dependency
> >>>>>>>>>>>> is
> >>>>>>>>>>>>>> appropriate. I do like using simple entity beans
> >>>> (POJO's
> >>>>>> not
> >>>>>>>>> j2EE)
> >>>>>>>>>>> for
> >>>>>>>>>>>>>> configuration, especially if they can be marshalled
> >>>>> without
> >>>>>>>>>>> annotation
> >>>>>>>>>>>> by
> >>>>>>>>>>>>>> Jackson. I only mentioned the dropwizard-extras
> >>>> because
> >>>>> it
> >>>>>>> has
> >>>>>>>>>> some
> >>>>>>>>>>>>> entity
> >>>>>>>>>>>>>> bean versions of the ZK and Kafka configs.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Domain-packaging: Also not a big deal - it's what's
> >>>>>> expected
> >>>>>>>> and
> >>>>>>>>>> it's
> >>>>>>>>>>>>>> pretty free in most IDE's. The advantages I see is
> >>> that
> >>>>> it
> >>>>>> is
> >>>>>>>>> clear
> >>>>>>>>>>>>> whether
> >>>>>>>>>>>>>> something is from the Apache Kafka project and
> >>> whether
> >>>>>>>> something
> >>>>>>>>> is
> >>>>>>>>>>>> from
> >>>>>>>>>>>>>> another org and related to Kafka. That said, nothing
> >>>>> really
> >>>>>>>>>> enforces
> >>>>>>>>>>>> it.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Futures: I'll see if I can create some examples to
> >>>>>>> demonstrate
> >>>>>>>>>> Future
> >>>>>>>>>>>>>> making interop easier.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>> C
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> On Fri, Jan 24, 2014 at 4:36 PM, Jay Kreps <
> >>>>>>>> jay.kreps@gmail.com>
> >>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Hey Clark,
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> - Serialization: Yes I agree with these though I
> >>>> don't
> >>>>>>>> consider
> >>>>>>>>>> the
> >>>>>>>>>>>>> loss
> >>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>> generics a big issue. I'll try to summarize what I
> >>>>> would
> >>>>>>>>> consider
> >>>>>>>>>>> the
> >>>>>>>>>>>>>> best
> >>>>>>>>>>>>>>> alternative api with raw byte[].
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> - Maven: We had this debate a few months back and
> >>> the
> >>>>>>>> consensus
> >>>>>>>>>> was
> >>>>>>>>>>>>>> gradle.
> >>>>>>>>>>>>>>> Is there a specific issue with the poms gradle
> >>>> makes? I
> >>>>>> am
> >>>>>>>>>>> extremely
> >>>>>>>>>>>>>> loath
> >>>>>>>>>>>>>>> to revisit the issue as build issues are a
> >>> recurring
> >>>>>> thing
> >>>>>>>> and
> >>>>>>>>> no
> >>>>>>>>>>> one
> >>>>>>>>>>>>>> ever
> >>>>>>>>>>>>>>> agrees and ultimately our build needs are very
> >>>> simple.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> - Config: I'm not sure if I follow the point. Are
> >>> you
> >>>>>>> saying
> >>>>>>>> we
> >>>>>>>>>>>> should
> >>>>>>>>>>>>>> use
> >>>>>>>>>>>>>>> something in dropwizard for config? One principle
> >>>> here
> >>>>> is
> >>>>>>> to
> >>>>>>>>> try
> >>>>>>>>>> to
> >>>>>>>>>>>>>> remove
> >>>>>>>>>>>>>>> as many client dependencies as possible as we
> >>>>> inevitably
> >>>>>>> run
> >>>>>>>>> into
> >>>>>>>>>>>>>> terrible
> >>>>>>>>>>>>>>> compatibility issues with users who use the same
> >>>>> library
> >>>>>> or
> >>>>>>>> its
> >>>>>>>>>>>>>>> dependencies at different versions. Or are you
> >>>> talking
> >>>>>>> about
> >>>>>>>>>>>>> maintaining
> >>>>>>>>>>>>>>> compatibility with existing config parameters? I
> >>>> think
> >>>>> as
> >>>>>>>> much
> >>>>>>>>>> as a
> >>>>>>>>>>>>>> config
> >>>>>>>>>>>>>>> in the existing client makes sense it should have
> >>> the
> >>>>>> same
> >>>>>>>> name
> >>>>>>>>>> (I
> >>>>>>>>>>>> was
> >>>>>>>>>>>>> a
> >>>>>>>>>>>>>>> bit sloppy about that so I'll fix any errors
> >>> there).
> >>>>>> There
> >>>>>>>> are
> >>>>>>>>> a
> >>>>>>>>>>> few
> >>>>>>>>>>>>> new
> >>>>>>>>>>>>>>> things and we should give those reasonable
> >>> defaults.
> >>>> I
> >>>>>>> think
> >>>>>>>>>> config
> >>>>>>>>>>>> is
> >>>>>>>>>>>>>>> important so I'll start a thread on the config
> >>>> package
> >>>>> in
> >>>>>>>>> there.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> - org.apache.kafka: We could do this. I always
> >>>>> considered
> >>>>>>> it
> >>>>>>>>> kind
> >>>>>>>>>>> of
> >>>>>>>>>>>> an
> >>>>>>>>>>>>>> odd
> >>>>>>>>>>>>>>> thing Java programmers do that has no real
> >>> motivation
> >>>>>> (but
> >>>>>>> I
> >>>>>>>>>> could
> >>>>>>>>>>> be
> >>>>>>>>>>>>>>> re-educated!). I don't think it ends up reducing
> >>>> naming
> >>>>>>>>> conflicts
> >>>>>>>>>>> in
> >>>>>>>>>>>>>>> practice and it adds a lot of noise and nested
> >>>>>> directories.
> >>>>>>>> Is
> >>>>>>>>>>> there
> >>>>>>>>>>>> a
> >>>>>>>>>>>>>>> reason you prefer this or just to be more
> >>> standard?
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> - Future: Basically I didn't see any particular
> >>>>>> advantage.
> >>>>>>>> The
> >>>>>>>>>>>> cancel()
> >>>>>>>>>>>>>>> method doesn't really make sense so probably
> >>> wouldn't
> >>>>>> work.
> >>>>>>>>>>> Likewise
> >>>>>>>>>>>> I
> >>>>>>>>>>>>>>> dislike the checked exceptions it requires.
> >>>> Basically I
> >>>>>>> just
> >>>>>>>>>> wrote
> >>>>>>>>>>>> out
> >>>>>>>>>>>>>> some
> >>>>>>>>>>>>>>> code examples and it seemed cleaner with a special
> >>>>>> purpose
> >>>>>>>>>> object.
> >>>>>>>>>>> I
> >>>>>>>>>>>>>> wasn't
> >>>>>>>>>>>>>>> actually aware of plans for improved futures in
> >>> java
> >>>> 8
> >>>>> or
> >>>>>>> the
> >>>>>>>>>> other
> >>>>>>>>>>>>>>> integrations. Maybe you could elaborate on this a
> >>> bit
> >>>>> and
> >>>>>>>> show
> >>>>>>>>>> how
> >>>>>>>>>>> it
> >>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>> be used? Sounds promising, I just don't know a lot
> >>>>> about
> >>>>>>> it.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> -Jay
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> On Fri, Jan 24, 2014 at 3:30 PM, Clark Breyman <
> >>>>>>>>>> clark@breyman.com>
> >>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Jay - Thanks for the call for comments. Here's
> >>> some
> >>>>>>> initial
> >>>>>>>>>>> input:
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> - Make message serialization a client
> >>>> responsibility
> >>>>>>>> (making
> >>>>>>>>>> all
> >>>>>>>>>>>>>> messages
> >>>>>>>>>>>>>>>> byte[]). Reflection-based loading makes it
> >>> harder
> >>>> to
> >>>>>> use
> >>>>>>>>>> generic
> >>>>>>>>>>>>> codecs
> >>>>>>>>>>>>>>>> (e.g.  Envelope<PREFIX, DATA, SUFFIX>) or build
> >>> up
> >>>>>> codec
> >>>>>>>>>>>>>>> programmatically.
> >>>>>>>>>>>>>>>> Non-default partitioning should require an
> >>> explicit
> >>>>>>>> partition
> >>>>>>>>>>> key.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> - I really like the fact that it will be native
> >>>> Java.
> >>>>>>>> Please
> >>>>>>>>>>>> consider
> >>>>>>>>>>>>>>> using
> >>>>>>>>>>>>>>>> native maven and not sbt, gradle, ivy, etc as
> >>> they
> >>>>>> don't
> >>>>>>>>>> reliably
> >>>>>>>>>>>>> play
> >>>>>>>>>>>>>>> nice
> >>>>>>>>>>>>>>>> in the maven ecosystem. A jar without a
> >>> well-formed
> >>>>> pom
> >>>>>>>>> doesn't
> >>>>>>>>>>>> feel
> >>>>>>>>>>>>>>> like a
> >>>>>>>>>>>>>>>> real artifact. The pom's generated by sbt et al.
> >>>> are
> >>>>>> not
> >>>>>>>> well
> >>>>>>>>>>>> formed.
> >>>>>>>>>>>>>>> Using
> >>>>>>>>>>>>>>>> maven will make builds and IDE integration much
> >>>>>> smoother.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> - Look at Nick Telford's dropwizard-extras
> >>> package
> >>>> in
> >>>>>>> which
> >>>>>>>>> he
> >>>>>>>>>>>>> defines
> >>>>>>>>>>>>>>> some
> >>>>>>>>>>>>>>>> Jackson-compatible POJO's for loading
> >>>> configuration.
> >>>>>>> Seems
> >>>>>>>>> like
> >>>>>>>>>>>> your
> >>>>>>>>>>>>>>> client
> >>>>>>>>>>>>>>>> migration is similar. The config objects should
> >>>> have
> >>>>>>>>>> constructors
> >>>>>>>>>>>> or
> >>>>>>>>>>>>>>>> factories that accept Map<String, String> and
> >>>>>> Properties
> >>>>>>>> for
> >>>>>>>>>> ease
> >>>>>>>>>>>> of
> >>>>>>>>>>>>>>>> migration.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> - Would you consider using the org.apache.kafka
> >>>>> package
> >>>>>>> for
> >>>>>>>>> the
> >>>>>>>>>>> new
> >>>>>>>>>>>>> API
> >>>>>>>>>>>>>>>> (quibble)
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> - Why create your own futures rather than use
> >>>>>>>>>>>>>>>> java.util.concurrent.Future<Long> or similar?
> >>>>> Standard
> >>>>>>>>> futures
> >>>>>>>>>>> will
> >>>>>>>>>>>>>> play
> >>>>>>>>>>>>>>>> nice with other reactive libs and things like
> >>> J8's
> >>>>>>>>>>>> ComposableFuture.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Thanks again,
> >>>>>>>>>>>>>>>> C
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> On Fri, Jan 24, 2014 at 2:46 PM, Roger Hoover <
> >>>>>>>>>>>>> roger.hoover@gmail.com
> >>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> A couple comments:
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> 1) Why does the config use a broker list
> >>> instead
> >>>> of
> >>>>>>>>>> discovering
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>> brokers
> >>>>>>>>>>>>>>>>> in ZooKeeper?  It doesn't match the
> >>>>> HighLevelConsumer
> >>>>>>>> API.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> 2) It looks like broker connections are
> >>> created
> >>>> on
> >>>>>>>> demand.
> >>>>>>>>>> I'm
> >>>>>>>>>>>>>>> wondering
> >>>>>>>>>>>>>>>>> if sometimes you might want to flush out
> >>> config
> >>>> or
> >>>>>>>> network
> >>>>>>>>>>>>>> connectivity
> >>>>>>>>>>>>>>>>> issues before pushing the first message
> >>> through.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Should there also be a
> >>> KafkaProducer.connect() or
> >>>>>>> .open()
> >>>>>>>>>>> method
> >>>>>>>>>>>> or
> >>>>>>>>>>>>>>>>> connectAll()?  I guess it would try to
> >>> connect to
> >>>>> all
> >>>>>>>>> brokers
> >>>>>>>>>>> in
> >>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>> BROKER_LIST_CONFIG
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> HTH,
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Roger
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> On Fri, Jan 24, 2014 at 11:54 AM, Jay Kreps <
> >>>>>>>>>>> jay.kreps@gmail.com
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> As mentioned in a previous email we are
> >>> working
> >>>>> on
> >>>>>> a
> >>>>>>>>>>>>>>> re-implementation
> >>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>> the producer. I would like to use this email
> >>>>> thread
> >>>>>>> to
> >>>>>>>>>>> discuss
> >>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>> details
> >>>>>>>>>>>>>>>>>> of the public API and the configuration. I
> >>>> would
> >>>>>> love
> >>>>>>>> for
> >>>>>>>>>> us
> >>>>>>>>>>> to
> >>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>> incredibly picky about this public api now
> >>> so
> >>>> it
> >>>>> is
> >>>>>>> as
> >>>>>>>>> good
> >>>>>>>>>>> as
> >>>>>>>>>>>>>>> possible
> >>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>> we don't need to break it in the future.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> The best way to get a feel for the API is
> >>>>> actually
> >>>>>> to
> >>>>>>>>> take
> >>>>>>>>>> a
> >>>>>>>>>>>> look
> >>>>>>>>>>>>>> at
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>> javadoc, my hope is to get the api docs good
> >>>>> enough
> >>>>>>> so
> >>>>>>>>> that
> >>>>>>>>>>> it
> >>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>> self-explanatory:
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>
> >>>>
> >>>
> http://empathybox.com/kafka-javadoc/index.html?kafka/clients/producer/KafkaProducer.html
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Please take a look at this API and give me
> >>> any
> >>>>>>> thoughts
> >>>>>>>>> you
> >>>>>>>>>>> may
> >>>>>>>>>>>>>> have!
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> It may also be reasonable to take a look at
> >>> the
> >>>>>>>> configs:
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>
> >>>>
> >>>
> http://empathybox.com/kafka-javadoc/kafka/clients/producer/ProducerConfig.html
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> The actual code is posted here:
> >>>>>>>>>>>>>>>>>>
> >>>> https://issues.apache.org/jira/browse/KAFKA-1227
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> A few questions or comments to kick things
> >>> off:
> >>>>>>>>>>>>>>>>>> 1. We need to make a decision on whether
> >>>>>>> serialization
> >>>>>>>> of
> >>>>>>>>>> the
> >>>>>>>>>>>>>> user's
> >>>>>>>>>>>>>>>> key
> >>>>>>>>>>>>>>>>>> and value should be done by the user (with
> >>> our
> >>>>> api
> >>>>>>> just
> >>>>>>>>>>> taking
> >>>>>>>>>>>>>>> byte[])
> >>>>>>>>>>>>>>>> or
> >>>>>>>>>>>>>>>>>> if we should take an object and allow the
> >>> user
> >>>> to
> >>>>>>>>>> configure a
> >>>>>>>>>>>>>>>> Serializer
> >>>>>>>>>>>>>>>>>> class which we instantiate via reflection.
> >>> We
> >>>>> take
> >>>>>>> the
> >>>>>>>>>> later
> >>>>>>>>>>>>>> approach
> >>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>> the current producer, and I have carried
> >>> this
> >>>>>> through
> >>>>>>>> to
> >>>>>>>>>> this
> >>>>>>>>>>>>>>>> prototype.
> >>>>>>>>>>>>>>>>>> The tradeoff I see is this: taking byte[] is
> >>>>>> actually
> >>>>>>>>>>> simpler,
> >>>>>>>>>>>>> the
> >>>>>>>>>>>>>>> user
> >>>>>>>>>>>>>>>>> can
> >>>>>>>>>>>>>>>>>> directly do whatever serialization they
> >>> like.
> >>>> The
> >>>>>>>>>>> complication
> >>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>> actually
> >>>>>>>>>>>>>>>>>> partitioning. Currently partitioning is done
> >>>> by a
> >>>>>>>> similar
> >>>>>>>>>>>> plug-in
> >>>>>>>>>>>>>> api
> >>>>>>>>>>>>>>>>>> (Partitioner) which the user can implement
> >>> and
> >>>>>>>> configure
> >>>>>>>>> to
> >>>>>>>>>>>>>> override
> >>>>>>>>>>>>>>>> how
> >>>>>>>>>>>>>>>>>> partitions are assigned. If we take byte[]
> >>> as
> >>>>> input
> >>>>>>>> then
> >>>>>>>>> we
> >>>>>>>>>>>> have
> >>>>>>>>>>>>> no
> >>>>>>>>>>>>>>>>> access
> >>>>>>>>>>>>>>>>>> to the original object and partitioning
> >>> MUST be
> >>>>>> done
> >>>>>>> on
> >>>>>>>>> the
> >>>>>>>>>>>>> byte[].
> >>>>>>>>>>>>>>>> This
> >>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>> fine for hash partitioning. However for
> >>> various
> >>>>>> types
> >>>>>>>> of
> >>>>>>>>>>>> semantic
> >>>>>>>>>>>>>>>>>> partitioning (range partitioning, or
> >>> whatever)
> >>>>> you
> >>>>>>>> would
> >>>>>>>>>> want
> >>>>>>>>>>>>>> access
> >>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>> original object. In the current approach a
> >>>>> producer
> >>>>>>> who
> >>>>>>>>>>> wishes
> >>>>>>>>>>>> to
> >>>>>>>>>>>>>>> send
> >>>>>>>>>>>>>>>>>> byte[] they have serialized in their own
> >>> code
> >>>> can
> >>>>>>>>> configure
> >>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>> BytesSerialization we supply which is just a
> >>>> "no
> >>>>>> op"
> >>>>>>>>>>>>> serialization.
> >>>>>>>>>>>>>>>>>> 2. We should obsess over naming and make
> >>> sure
> >>>>> each
> >>>>>> of
> >>>>>>>> the
> >>>>>>>>>>> class
> >>>>>>>>>>>>>> names
> >>>>>>>>>>>>>>>> are
> >>>>>>>>>>>>>>>>>> good.
> >>>>>>>>>>>>>>>>>> 3. Jun has already pointed out that we need
> >>> to
> >>>>>>> include
> >>>>>>>>> the
> >>>>>>>>>>>> topic
> >>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>> partition in the response, which is
> >>> absolutely
> >>>>>>> right. I
> >>>>>>>>>>> haven't
> >>>>>>>>>>>>>> done
> >>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>> yet but that definitely needs to be there.
> >>>>>>>>>>>>>>>>>> 4. Currently RecordSend.await will throw an
> >>>>>> exception
> >>>>>>>> if
> >>>>>>>>>> the
> >>>>>>>>>>>>>> request
> >>>>>>>>>>>>>>>>>> failed. The intention here is that
> >>>>>>>>>>>> producer.send(message).await()
> >>>>>>>>>>>>>>>> exactly
> >>>>>>>>>>>>>>>>>> simulates a synchronous call. Guozhang has
> >>>> noted
> >>>>>> that
> >>>>>>>>> this
> >>>>>>>>>>> is a
> >>>>>>>>>>>>>>> little
> >>>>>>>>>>>>>>>>>> annoying since the user must then catch
> >>>>> exceptions.
> >>>>>>>>> However
> >>>>>>>>>>> if
> >>>>>>>>>>>> we
> >>>>>>>>>>>>>>>> remove
> >>>>>>>>>>>>>>>>>> this then if the user doesn't check for
> >>> errors
> >>>>> they
> >>>>>>>> won't
> >>>>>>>>>>> know
> >>>>>>>>>>>>> one
> >>>>>>>>>>>>>>> has
> >>>>>>>>>>>>>>>>>> occurred, which I predict will be a common
> >>>>> mistake.
> >>>>>>>>>>>>>>>>>> 5. Perhaps there is more we could do to make
> >>>> the
> >>>>>>> async
> >>>>>>>>>>>> callbacks
> >>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>> future
> >>>>>>>>>>>>>>>>>> we give back intuitive and easy to program
> >>>>> against?
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Some background info on implementation:
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> At a high level the primary difference in
> >>> this
> >>>>>>> producer
> >>>>>>>>> is
> >>>>>>>>>>> that
> >>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>> removes
> >>>>>>>>>>>>>>>>>> the distinction between the "sync" and
> >>> "async"
> >>>>>>>> producer.
> >>>>>>>>>>>>>> Effectively
> >>>>>>>>>>>>>>>> all
> >>>>>>>>>>>>>>>>>> requests are sent asynchronously but always
> >>>>> return
> >>>>>> a
> >>>>>>>>> future
> >>>>>>>>>>>>>> response
> >>>>>>>>>>>>>>>>> object
> >>>>>>>>>>>>>>>>>> that gives the offset as well as any error
> >>> that
> >>>>> may
> >>>>>>>> have
> >>>>>>>>>>>> occurred
> >>>>>>>>>>>>>>> when
> >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>> request is complete. The batching that is
> >>> done
> >>>> in
> >>>>>> the
> >>>>>>>>> async
> >>>>>>>>>>>>>> producer
> >>>>>>>>>>>>>>>> only
> >>>>>>>>>>>>>>>>>> today is done whenever possible now. This
> >>> means
> >>>>>> that
> >>>>>>>> the
> >>>>>>>>>> sync
> >>>>>>>>>>>>>>> producer,
> >>>>>>>>>>>>>>>>>> under load, can get performance as good as
> >>> the
> >>>>>> async
> >>>>>>>>>> producer
> >>>>>>>>>>>>>>>>> (preliminary
> >>>>>>>>>>>>>>>>>> results show the producer getting 1m
> >>>>> messages/sec).
> >>>>>>>> This
> >>>>>>>>>>> works
> >>>>>>>>>>>>>>> similar
> >>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>> group commit in databases but with respect
> >>> to
> >>>> the
> >>>>>>>> actual
> >>>>>>>>>>>> network
> >>>>>>>>>>>>>>>>>> transmission--any messages that arrive
> >>> while a
> >>>>> send
> >>>>>>> is
> >>>>>>>> in
> >>>>>>>>>>>>> progress
> >>>>>>>>>>>>>>> are
> >>>>>>>>>>>>>>>>>> batched together. It is also possible to
> >>>>> encourage
> >>>>>>>>> batching
> >>>>>>>>>>>> even
> >>>>>>>>>>>>>>> under
> >>>>>>>>>>>>>>>>> low
> >>>>>>>>>>>>>>>>>> load to save server resources by
> >>> introducing a
> >>>>>> delay
> >>>>>>> on
> >>>>>>>>> the
> >>>>>>>>>>>> send
> >>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>> allow
> >>>>>>>>>>>>>>>>>> more messages to accumulate; this is done
> >>> using
> >>>>> the
> >>>>>>>>>>>>> linger.msconfig
> >>>>>>>>>>>>>>>>> (this
> >>>>>>>>>>>>>>>>>> is similar to Nagle's algorithm in TCP).
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> This producer does all network communication
> >>>>>>>>> asynchronously
> >>>>>>>>>>> and
> >>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>> parallel
> >>>>>>>>>>>>>>>>>> to all servers so the performance penalty
> >>> for
> >>>>>> acks=-1
> >>>>>>>> and
> >>>>>>>>>>>> waiting
> >>>>>>>>>>>>>> on
> >>>>>>>>>>>>>>>>>> replication should be much reduced. I
> >>> haven't
> >>>>> done
> >>>>>>> much
> >>>>>>>>>>>>>> benchmarking
> >>>>>>>>>>>>>>> on
> >>>>>>>>>>>>>>>>>> this yet, though.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> The high level design is described a little
> >>>> here,
> >>>>>>>> though
> >>>>>>>>>> this
> >>>>>>>>>>>> is
> >>>>>>>>>>>>>> now
> >>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>> little out of date:
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>> https://cwiki.apache.org/confluence/display/KAFKA/Client+Rewrite
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> -Jay
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>
> >>>>
> >>>
> >>
> >>
>

Re: New Producer Public API

Posted by Steve Morin <st...@gmail.com>.
Is the new producer API going to maintain protocol compatibility with old version if the API under the hood?

> On Jan 29, 2014, at 10:15, Jay Kreps <ja...@gmail.com> wrote:
> 
> The challenge of directly exposing ProduceRequestResult is that the offset
> provided is just the base offset and there is no way to know for a
> particular message where it was in relation to that base offset because the
> batching is transparent and non-deterministic. So I think we do need some
> kind of per-message result.
> 
> I started with Future<RequestResult>, I think for the same reason you
> prefer it but then when I actually looked at some code samples it wasn't
> too great--checked exceptions, methods that we can't easily implement, etc.
> I moved away from that for two reasons:
> 1. When I actually wrote out some code samples of usage they were a little
> ugly for the reasons I described--checked exceptions, methods we can't
> implement, no helper methods, etc.
> 2. I originally intended to make the result send work like a
> ListenableFuture so that you would register the callback on the result
> rather than as part of the call. I moved away from this primarily because
> the implementation complexity was a little higher.
> 
> Whether or not the code prettiness on its own outweighs the familiarity of
> a normal Future I don't know, but that was the evolution of my thinking.
> 
> -Jay
> 
> 
>> On Wed, Jan 29, 2014 at 10:06 AM, Jay Kreps <ja...@gmail.com> wrote:
>> 
>> Hey Neha,
>> 
>> Error handling in RecordSend works as in Future you will get the exception
>> if there is one from any of the accessor methods or await().
>> 
>> The purpose of hasError was that you can write things slightly more simply
>> (which some people expressed preference for):
>>  if(send.hasError())
>>    // do something
>>  long offset = send.offset();
>> 
>> Instead of the more the slightly longer:
>> try {
>>   long offset = send.offset();
>> } catch (KafkaException e) {
>>   // do something
>> }
>> 
>> 
>> On Wed, Jan 29, 2014 at 10:01 AM, Neha Narkhede <ne...@gmail.com>wrote:
>> 
>>> Regarding the use of Futures -
>>> 
>>> Agree that there are some downsides to using Futures but both approaches
>>> have some tradeoffs.
>>> 
>>> - Standardization and usability
>>> Future is a widely used and understood Java API and given that the
>>> functionality that RecordSend hopes to provide is essentially that of
>>> Future, I think it makes sense to expose a widely understood public API
>>> for
>>> our clients. RecordSend, on the other hand, seems to provide some APIs
>>> that
>>> are very similar to that of Future, in addition to exposing a bunch of
>>> APIs
>>> that belong to ProduceRequestResult. As a user, I would've really
>>> preferred
>>> to deal with ProduceRequestResult directly -
>>> Future<ProduceRequestResult> send(...)
>>> 
>>> - Error handling
>>> RecordSend's error handling is quite unintuitive where the user has to
>>> remember to invoke hasError and error, instead of just throwing the
>>> exception. Now there are
>>> some downsides regarding error handling with the Future as well, where the
>>> user has to catch InterruptedException when we would never run into it.
>>> However, it seems like a price worth paying for supporting a standard API
>>> and error handling
>>> 
>>> - Unused APIs
>>> This is a downside of using Future, where the cancel() operation would
>>> always return false and mean nothing. But we can mention that caveat in
>>> our
>>> Java docs.
>>> 
>>> To summarize, I would prefer to expose a well understood and widely
>>> adopted
>>> Java API and put up with the overhead of catching one unnecessary checked
>>> exception, rather than wrap the useful ProduceRequestResult in a custom
>>> async object (RecordSend) and explain that to our many users.
>>> 
>>> Thanks,
>>> Neha
>>> 
>>> 
>>> 
>>> 
>>>> On Tue, Jan 28, 2014 at 8:10 PM, Jay Kreps <ja...@gmail.com> wrote:
>>>> 
>>>> Hey Neha,
>>>> 
>>>> Can you elaborate on why you prefer using Java's Future? The downside
>>> in my
>>>> mind is the use of the checked InterruptedException and
>>> ExecutionException.
>>>> ExecutionException is arguable, but forcing you to catch
>>>> InterruptedException, often in code that can't be interrupted, seems
>>>> perverse. It also leaves us with the cancel() method which I don't
>>> think we
>>>> really can implement.
>>>> 
>>>> Option 1A, to recap/elaborate, was the following. There is no
>>> Serializer or
>>>> Partitioner api. We take a byte[] key and value and an optional integer
>>>> partition. If you specify the integer partition it will be used. If you
>>> do
>>>> not specify a key or a partition the partition will be chosen in a round
>>>> robin fashion. If you specify a key but no partition we will chose a
>>>> partition based on a hash of the key. In order to let the user find the
>>>> partition we will need to given them access to the Cluster instance
>>>> directly from the producer.
>>>> 
>>>> -Jay
>>>> 
>>>> 
>>>> On Tue, Jan 28, 2014 at 6:25 PM, Neha Narkhede <neha.narkhede@gmail.com
>>>>> wrote:
>>>> 
>>>>> Here are more thoughts on the public APIs -
>>>>> 
>>>>> - I suggest we use java's Future instead of custom Future especially
>>>> since
>>>>> it is part of the public API
>>>>> 
>>>>> - Serialization: I like the simplicity of the producer APIs with the
>>>>> absence of serialization where we just deal with byte arrays for keys
>>> and
>>>>> values. What I don't like about this is the performance overhead on
>>> the
>>>>> Partitioner for any kind of custom partitioning based on the
>>>> partitionKey.
>>>>> Since the only purpose of partitionKey is to do custom partitioning,
>>> why
>>>>> can't we take it in directly as an integer and let the user figure out
>>>> the
>>>>> mapping from partition_key -> partition_id using the getCluster() API?
>>>> If I
>>>>> understand correctly, this is similar to what you suggested as part of
>>>>> option 1A. I like this approach since it maintains the simplicity of
>>> APIs
>>>>> by allowing us to deal with bytes and does not compromise performance
>>> in
>>>>> the custom partitioning case.
>>>>> 
>>>>> Thanks,
>>>>> Neha
>>>>> 
>>>>> 
>>>>> 
>>>>> On Tue, Jan 28, 2014 at 5:42 PM, Jay Kreps <ja...@gmail.com>
>>> wrote:
>>>>> 
>>>>>> Hey Tom,
>>>>>> 
>>>>>> That sounds cool. How did you end up handling parallel I/O if you
>>> wrap
>>>>> the
>>>>>> individual connections? Don't you need some selector that selects
>>> over
>>>>> all
>>>>>> the connections?
>>>>>> 
>>>>>> -Jay
>>>>>> 
>>>>>> 
>>>>>> On Tue, Jan 28, 2014 at 2:31 PM, Tom Brown <to...@gmail.com>
>>>> wrote:
>>>>>> 
>>>>>>> I implemented a 0.7 client in pure java, and its API very closely
>>>>>> resembled
>>>>>>> this. (When multiple people independently engineer the same
>>> solution,
>>>>>> it's
>>>>>>> probably good... right?). However, there were a few architectural
>>>>>>> differences with my client:
>>>>>>> 
>>>>>>> 1. The basic client itself was just an asynchronous layer around
>>> the
>>>>>>> different server functions. In and of itself it had no knowledge
>>> of
>>>>>>> partitions, only servers (and maintained TCP connections to them).
>>>>>>> 
>>>>>>> 2. The main producer was an additional layer that provided a
>>>> high-level
>>>>>>> interface that could batch individual messages based on partition.
>>>>>>> 
>>>>>>> 3. Knowledge of partitioning was done via an interface so that
>>>>> different
>>>>>>> strategies could be used.
>>>>>>> 
>>>>>>> 4. Partitioning was done by the user, with knowledge of the
>>> available
>>>>>>> partitions provided by #3.
>>>>>>> 
>>>>>>> 5. Serialization was done by the user to simplify the API.
>>>>>>> 
>>>>>>> 6. Futures were used to make asynchronous emulate synchronous
>>> calls.
>>>>>>> 
>>>>>>> 
>>>>>>> The main benefit of this approach is flexibility. For example,
>>> since
>>>>> the
>>>>>>> base client was just a managed connection (and not inherently a
>>>>>> producer),
>>>>>>> it was easy to composite a produce request and an offsets request
>>>>>> together
>>>>>>> into a confirmed produce request (officially not available in
>>> 0.7).
>>>>>>> 
>>>>>>> Decoupling the basic client from partition management allowed the
>>> me
>>>> to
>>>>>>> implement zk discovery as a separate project so that the main
>>> project
>>>>> had
>>>>>>> no complex dependencies. The same was true of decoupling
>>>> serialization.
>>>>>>> It's trivial to build an optional layer that adds those features
>>> in,
>>>>>> while
>>>>>>> allowing access to the base APIs for those that need it.
>>>>>>> 
>>>>>>> Using standard Future objects was also beneficial, since I could
>>>>> combine
>>>>>>> them with existing tools (such as guava).
>>>>>>> 
>>>>>>> It may be too late to be of use, but I have been working with my
>>>>>> company's
>>>>>>> legal department to release the implementation I described above.
>>> If
>>>>>> you're
>>>>>>> interested in it, let me know.
>>>>>>> 
>>>>>>> 
>>>>>>> To sum up my thoughts regarding the new API, I think it's a great
>>>>> start.
>>>>>> I
>>>>>>> would like to see a more layered approach so I can use the parts I
>>>>> want,
>>>>>>> and adapt the other parts as needed. I would also like to see
>>>> standard
>>>>>>> interfaces (especially Future) used where they makes sense.
>>>>>>> 
>>>>>>> --Tom
>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>>> On Tue, Jan 28, 2014 at 1:33 PM, Roger Hoover <
>>>> roger.hoover@gmail.com
>>>>>>>> wrote:
>>>>>>> 
>>>>>>>> +1 ListenableFuture: If this works similar to Deferreds in
>>> Twisted
>>>>>> Python
>>>>>>>> or Promised IO in Javascript, I think this is a great pattern
>>> for
>>>>>>>> decoupling your callback logic from the place where the Future
>>> is
>>>>>>>> generated.  You can register as many callbacks as you like,
>>> each in
>>>>> the
>>>>>>>> appropriate layer of the code and have each observer get
>>> notified
>>>>> when
>>>>>>> the
>>>>>>>> promised i/o is complete without any of them knowing about each
>>>>> other.
>>>>>>>> 
>>>>>>>> 
>>>>>>>> On Tue, Jan 28, 2014 at 11:32 AM, Jay Kreps <
>>> jay.kreps@gmail.com>
>>>>>> wrote:
>>>>>>>> 
>>>>>>>>> Hey Ross,
>>>>>>>>> 
>>>>>>>>> - ListenableFuture: Interesting. That would be an alternative
>>> to
>>>>> the
>>>>>>>> direct
>>>>>>>>> callback support we provide. There could be pros to this, let
>>> me
>>>>>> think
>>>>>>>>> about it.
>>>>>>>>> - We could provide layering, but I feel that the
>>> serialization is
>>>>>> such
>>>>>>> a
>>>>>>>>> small thing we should just make a decision and chose one, it
>>>>> doesn't
>>>>>>> seem
>>>>>>>>> to me to justify a whole public facing layer.
>>>>>>>>> - Yes, this is fairly esoteric, essentially I think it is
>>> fairly
>>>>>>> similar
>>>>>>>> to
>>>>>>>>> databases like DynamoDB that allow you to specify two
>>> partition
>>>>> keys
>>>>>> (I
>>>>>>>>> think DynamoDB does this...). The reasoning is that in fact
>>> there
>>>>> are
>>>>>>>>> several things you can use the key field for: (1) to compute
>>> the
>>>>>>>> partition
>>>>>>>>> to store the data in, (2) as a unique identifier to
>>> deduplicate
>>>>> that
>>>>>>>>> partition's records within a log. These two things are almost
>>>>> always
>>>>>>> the
>>>>>>>>> same, but occationally may differ when you want to group data
>>> in
>>>> a
>>>>>> more
>>>>>>>>> sophisticated way then just a hash of the primary key but
>>> still
>>>>>> retain
>>>>>>>> the
>>>>>>>>> proper primary key for delivery to the consumer and log
>>>> compaction.
>>>>>>>>> 
>>>>>>>>> -Jay
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> On Tue, Jan 28, 2014 at 3:24 AM, Ross Black <
>>>>> ross.w.black@gmail.com>
>>>>>>>>> wrote:
>>>>>>>>> 
>>>>>>>>>> Hi Jay,
>>>>>>>>>> 
>>>>>>>>>> - Just to add some more info/confusion about possibly using
>>>>> Future
>>>>>>> ...
>>>>>>>>>>  If Kafka uses a JDK future, it plays nicely with other
>>>>> frameworks
>>>>>>> as
>>>>>>>>>> well.
>>>>>>>>>>  Google Guava has a ListenableFuture that allows callback
>>>>> handling
>>>>>>> to
>>>>>>>> be
>>>>>>>>>> added via the returned future, and allows the callbacks to
>>> be
>>>>>> passed
>>>>>>>> off
>>>>>>>>> to
>>>>>>>>>> a specified executor.
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>> 
>>>> 
>>> http://docs.guava-libraries.googlecode.com/git-history/release/javadoc/com/google/common/util/concurrent/ListenableFuture.html
>>>>>>>>>>  The JDK future can easily be converted to a listenable
>>>> future.
>>>>>>>>>> 
>>>>>>>>>> - On the question of byte[] vs Object, could this be solved
>>> by
>>>>>>> layering
>>>>>>>>> the
>>>>>>>>>> API?  eg. a raw producer (use byte[] and specify the
>>> partition
>>>>>>> number)
>>>>>>>>> and
>>>>>>>>>> a normal producer (use generic object and specify a
>>>> Partitioner)?
>>>>>>>>>> 
>>>>>>>>>> - I am confused by the keys in ProducerRecord and
>>> Partitioner.
>>>>>> What
>>>>>>> is
>>>>>>>>> the
>>>>>>>>>> usage for both a key and a partition key? (I am not yet
>>> using
>>>>> 0.8)
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> Thanks,
>>>>>>>>>> Ross
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> On 28 January 2014 05:00, Xavier Stevens <xavier@gaikai.com
>>>> 
>>>>>> wrote:
>>>>>>>>>> 
>>>>>>>>>>> AutoCloseable would be nice for us as most of our code is
>>>> using
>>>>>>> Java
>>>>>>>> 7
>>>>>>>>> at
>>>>>>>>>>> this point.
>>>>>>>>>>> 
>>>>>>>>>>> I like Dropwizard's configuration mapping to POJOs via
>>>> Jackson,
>>>>>> but
>>>>>>>> if
>>>>>>>>>> you
>>>>>>>>>>> wanted to stick with property maps I don't care enough to
>>>>> object.
>>>>>>>>>>> 
>>>>>>>>>>> If the producer only dealt with bytes, is there a way we
>>>> could
>>>>>>> still
>>>>>>>>> due
>>>>>>>>>>> partition plugins without specifying the number
>>> explicitly? I
>>>>>> would
>>>>>>>>>> prefer
>>>>>>>>>>> to be able to pass in field(s) that would be used by the
>>>>>>> partitioner.
>>>>>>>>>>> Obviously if this wasn't possible you could always
>>>> deserialize
>>>>>> the
>>>>>>>>> object
>>>>>>>>>>> in the partitioner and grab the fields you want, but that
>>>> seems
>>>>>>>> really
>>>>>>>>>>> expensive to do on every message.
>>>>>>>>>>> 
>>>>>>>>>>> It would also be nice to have a Java API Encoder
>>> constructor
>>>>>> taking
>>>>>>>> in
>>>>>>>>>>> VerifiableProperties. Scala understands how to handle
>>> "props:
>>>>>>>>>>> VerifiableProperties = null", but Java doesn't. So you
>>> don't
>>>>> run
>>>>>>> into
>>>>>>>>>> this
>>>>>>>>>>> problem until runtime.
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> -Xavier
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> On Mon, Jan 27, 2014 at 9:37 AM, Clark Breyman <
>>>>>> clark@breyman.com>
>>>>>>>>>> wrote:
>>>>>>>>>>> 
>>>>>>>>>>>> Jay -
>>>>>>>>>>>> 
>>>>>>>>>>>> Config - your explanation makes sense. I'm just so
>>>> accustomed
>>>>>> to
>>>>>>>>> having
>>>>>>>>>>>> Jackson automatically map my configuration objects to
>>> POJOs
>>>>>> that
>>>>>>>> I've
>>>>>>>>>>>> stopped using property files. They are lingua franca.
>>> The
>>>>> only
>>>>>>>>> thought
>>>>>>>>>>>> might be to separate the config interface from the
>>>>>> implementation
>>>>>>>> to
>>>>>>>>>>> allow
>>>>>>>>>>>> for alternatives, but that might undermine your point of
>>>> "do
>>>>> it
>>>>>>>> this
>>>>>>>>>> way
>>>>>>>>>>> so
>>>>>>>>>>>> that everyone can find it where they expect it".
>>>>>>>>>>>> 
>>>>>>>>>>>> Serialization: Of the options, I like 1A the best,
>>> though
>>>>>>> possibly
>>>>>>>>> with
>>>>>>>>>>>> either an option to specify a partition key rather than
>>> ID
>>>>> or a
>>>>>>>>> helper
>>>>>>>>>> to
>>>>>>>>>>>> translate an arbitrary byte[] or long into a partition
>>>>> number.
>>>>>>>>>>>> 
>>>>>>>>>>>> Thanks
>>>>>>>>>>>> Clark
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> On Sun, Jan 26, 2014 at 9:13 PM, Jay Kreps <
>>>>>> jay.kreps@gmail.com>
>>>>>>>>>> wrote:
>>>>>>>>>>>> 
>>>>>>>>>>>>> Thanks for the detailed thoughts. Let me elaborate on
>>> the
>>>>>>> config
>>>>>>>>>> thing.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> I agree that at first glance key-value strings don't
>>> seem
>>>>>> like
>>>>>>> a
>>>>>>>>> very
>>>>>>>>>>>> good
>>>>>>>>>>>>> configuration api for a client. Surely a well-typed
>>>> config
>>>>>>> class
>>>>>>>>>> would
>>>>>>>>>>> be
>>>>>>>>>>>>> better! I actually disagree and let me see if I can
>>>>> convince
>>>>>>> you.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> My reasoning has nothing to do with the api and
>>>> everything
>>>>> to
>>>>>>> do
>>>>>>>>> with
>>>>>>>>>>>>> operations.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> Clients are embedded in applications which are
>>> themselves
>>>>>>>>> configured.
>>>>>>>>>>> In
>>>>>>>>>>>>> any place that takes operations seriously the
>>>> configuration
>>>>>> for
>>>>>>>>> these
>>>>>>>>>>>>> applications will be version controlled and maintained
>>>>>> through
>>>>>>>> some
>>>>>>>>>>> kind
>>>>>>>>>>>> of
>>>>>>>>>>>>> config management system. If we give a config class
>>> with
>>>>>>> getters
>>>>>>>>> and
>>>>>>>>>>>>> setters the application has to expose those
>>> properties to
>>>>> its
>>>>>>>>>>>>> configuration. What invariably happens is that the
>>>>>> application
>>>>>>>>>> exposes
>>>>>>>>>>>> only
>>>>>>>>>>>>> a choice few properties that they thought they would
>>>>> change.
>>>>>>>>>>> Furthermore
>>>>>>>>>>>>> the application will make up a name for these configs
>>>> that
>>>>>>> seems
>>>>>>>>>>>> intuitive
>>>>>>>>>>>>> at the time in the 2 seconds the engineer spends
>>> thinking
>>>>>> about
>>>>>>>> it.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> Now consider the result of this in the large. You end
>>> up
>>>>> with
>>>>>>>>> dozens
>>>>>>>>>> or
>>>>>>>>>>>>> hundreds of applications that have the client
>>> embedded.
>>>>> Each
>>>>>>>>> exposes
>>>>>>>>>> a
>>>>>>>>>>>>> different, inadequate subset of the possible configs,
>>>> each
>>>>>> with
>>>>>>>>>>> different
>>>>>>>>>>>>> names. It is a nightmare.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> If you use a string-string map the config system can
>>>>> directly
>>>>>>>> get a
>>>>>>>>>>>> bundle
>>>>>>>>>>>>> of config key-value pairs and put them into the
>>> client.
>>>>> This
>>>>>>>> means
>>>>>>>>>> that
>>>>>>>>>>>> all
>>>>>>>>>>>>> configuration is automatically available with the name
>>>>>>> documented
>>>>>>>>> on
>>>>>>>>>>> the
>>>>>>>>>>>>> website in every application that does this. If you
>>>> upgrade
>>>>>> to
>>>>>>> a
>>>>>>>>> new
>>>>>>>>>>>> kafka
>>>>>>>>>>>>> version with more configs those will be exposed too.
>>> If
>>>> you
>>>>>>>> realize
>>>>>>>>>>> that
>>>>>>>>>>>>> you need to change a default you can just go through
>>> your
>>>>>>> configs
>>>>>>>>> and
>>>>>>>>>>>>> change it everywhere as it will have the same name
>>>>>> everywhere.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> -Jay
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> On Sun, Jan 26, 2014 at 4:47 PM, Clark Breyman <
>>>>>>>> clark@breyman.com>
>>>>>>>>>>>> wrote:
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> Thanks Jay. I'll see if I can put together a more
>>>>> complete
>>>>>>>>>> response,
>>>>>>>>>>>>>> perhaps as separate threads so that topics don't get
>>>>>>> entangled.
>>>>>>>>> In
>>>>>>>>>>> the
>>>>>>>>>>>>> mean
>>>>>>>>>>>>>> time, here's a couple responses:
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> Serialization: you've broken out a sub-thread so
>>> i'll
>>>>> reply
>>>>>>>>> there.
>>>>>>>>>> My
>>>>>>>>>>>>> bias
>>>>>>>>>>>>>> is that I like generics (except for type-erasure)
>>> and
>>>> in
>>>>>>>>> particular
>>>>>>>>>>>> they
>>>>>>>>>>>>>> make it easy to compose serializers for compound
>>>> payloads
>>>>>>> (e.g.
>>>>>>>>>> when
>>>>>>>>>>> a
>>>>>>>>>>>>>> common header wraps a payload of parameterized
>>> type).
>>>>> I'll
>>>>>>>>> respond
>>>>>>>>>> to
>>>>>>>>>>>>> your
>>>>>>>>>>>>>> 4-options message with an example.
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> Build: I've seen a lot of "maven-compatible" build
>>>>> systems
>>>>>>>>> produce
>>>>>>>>>>>>>> "artifacts" that aren't really artifacts - no
>>> embedded
>>>>> POM
>>>>>>> or,
>>>>>>>>>> worst,
>>>>>>>>>>>>>> malformed POM. I know the sbt-generated artifacts
>>> were
>>>>> this
>>>>>>>> way -
>>>>>>>>>>> onus
>>>>>>>>>>>> is
>>>>>>>>>>>>>> on me to see what gradle is spitting out and what a
>>>> maven
>>>>>>> build
>>>>>>>>>> might
>>>>>>>>>>>>> look
>>>>>>>>>>>>>> like. Maven may be old and boring, but it gets out
>>> of
>>>> the
>>>>>> way
>>>>>>>> and
>>>>>>>>>>>>>> integrates really seamlessly with a lot of IDEs.
>>> When
>>>>> some
>>>>>>>> scala
>>>>>>>>>>>>> projects I
>>>>>>>>>>>>>> was working on in the fall of 2011 switched from
>>> sbt to
>>>>>>> maven,
>>>>>>>>>> build
>>>>>>>>>>>>> became
>>>>>>>>>>>>>> a non-issue.
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> Config: Not a big deal  and no, I don't think a
>>>>> dropwizard
>>>>>>>>>> dependency
>>>>>>>>>>>> is
>>>>>>>>>>>>>> appropriate. I do like using simple entity beans
>>>> (POJO's
>>>>>> not
>>>>>>>>> j2EE)
>>>>>>>>>>> for
>>>>>>>>>>>>>> configuration, especially if they can be marshalled
>>>>> without
>>>>>>>>>>> annotation
>>>>>>>>>>>> by
>>>>>>>>>>>>>> Jackson. I only mentioned the dropwizard-extras
>>>> because
>>>>> it
>>>>>>> has
>>>>>>>>>> some
>>>>>>>>>>>>> entity
>>>>>>>>>>>>>> bean versions of the ZK and Kafka configs.
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> Domain-packaging: Also not a big deal - it's what's
>>>>>> expected
>>>>>>>> and
>>>>>>>>>> it's
>>>>>>>>>>>>>> pretty free in most IDE's. The advantages I see is
>>> that
>>>>> it
>>>>>> is
>>>>>>>>> clear
>>>>>>>>>>>>> whether
>>>>>>>>>>>>>> something is from the Apache Kafka project and
>>> whether
>>>>>>>> something
>>>>>>>>> is
>>>>>>>>>>>> from
>>>>>>>>>>>>>> another org and related to Kafka. That said, nothing
>>>>> really
>>>>>>>>>> enforces
>>>>>>>>>>>> it.
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> Futures: I'll see if I can create some examples to
>>>>>>> demonstrate
>>>>>>>>>> Future
>>>>>>>>>>>>>> making interop easier.
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>> C
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> On Fri, Jan 24, 2014 at 4:36 PM, Jay Kreps <
>>>>>>>> jay.kreps@gmail.com>
>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> Hey Clark,
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> - Serialization: Yes I agree with these though I
>>>> don't
>>>>>>>> consider
>>>>>>>>>> the
>>>>>>>>>>>>> loss
>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>> generics a big issue. I'll try to summarize what I
>>>>> would
>>>>>>>>> consider
>>>>>>>>>>> the
>>>>>>>>>>>>>> best
>>>>>>>>>>>>>>> alternative api with raw byte[].
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> - Maven: We had this debate a few months back and
>>> the
>>>>>>>> consensus
>>>>>>>>>> was
>>>>>>>>>>>>>> gradle.
>>>>>>>>>>>>>>> Is there a specific issue with the poms gradle
>>>> makes? I
>>>>>> am
>>>>>>>>>>> extremely
>>>>>>>>>>>>>> loath
>>>>>>>>>>>>>>> to revisit the issue as build issues are a
>>> recurring
>>>>>> thing
>>>>>>>> and
>>>>>>>>> no
>>>>>>>>>>> one
>>>>>>>>>>>>>> ever
>>>>>>>>>>>>>>> agrees and ultimately our build needs are very
>>>> simple.
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> - Config: I'm not sure if I follow the point. Are
>>> you
>>>>>>> saying
>>>>>>>> we
>>>>>>>>>>>> should
>>>>>>>>>>>>>> use
>>>>>>>>>>>>>>> something in dropwizard for config? One principle
>>>> here
>>>>> is
>>>>>>> to
>>>>>>>>> try
>>>>>>>>>> to
>>>>>>>>>>>>>> remove
>>>>>>>>>>>>>>> as many client dependencies as possible as we
>>>>> inevitably
>>>>>>> run
>>>>>>>>> into
>>>>>>>>>>>>>> terrible
>>>>>>>>>>>>>>> compatibility issues with users who use the same
>>>>> library
>>>>>> or
>>>>>>>> its
>>>>>>>>>>>>>>> dependencies at different versions. Or are you
>>>> talking
>>>>>>> about
>>>>>>>>>>>>> maintaining
>>>>>>>>>>>>>>> compatibility with existing config parameters? I
>>>> think
>>>>> as
>>>>>>>> much
>>>>>>>>>> as a
>>>>>>>>>>>>>> config
>>>>>>>>>>>>>>> in the existing client makes sense it should have
>>> the
>>>>>> same
>>>>>>>> name
>>>>>>>>>> (I
>>>>>>>>>>>> was
>>>>>>>>>>>>> a
>>>>>>>>>>>>>>> bit sloppy about that so I'll fix any errors
>>> there).
>>>>>> There
>>>>>>>> are
>>>>>>>>> a
>>>>>>>>>>> few
>>>>>>>>>>>>> new
>>>>>>>>>>>>>>> things and we should give those reasonable
>>> defaults.
>>>> I
>>>>>>> think
>>>>>>>>>> config
>>>>>>>>>>>> is
>>>>>>>>>>>>>>> important so I'll start a thread on the config
>>>> package
>>>>> in
>>>>>>>>> there.
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> - org.apache.kafka: We could do this. I always
>>>>> considered
>>>>>>> it
>>>>>>>>> kind
>>>>>>>>>>> of
>>>>>>>>>>>> an
>>>>>>>>>>>>>> odd
>>>>>>>>>>>>>>> thing Java programmers do that has no real
>>> motivation
>>>>>> (but
>>>>>>> I
>>>>>>>>>> could
>>>>>>>>>>> be
>>>>>>>>>>>>>>> re-educated!). I don't think it ends up reducing
>>>> naming
>>>>>>>>> conflicts
>>>>>>>>>>> in
>>>>>>>>>>>>>>> practice and it adds a lot of noise and nested
>>>>>> directories.
>>>>>>>> Is
>>>>>>>>>>> there
>>>>>>>>>>>> a
>>>>>>>>>>>>>>> reason you prefer this or just to be more
>>> standard?
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> - Future: Basically I didn't see any particular
>>>>>> advantage.
>>>>>>>> The
>>>>>>>>>>>> cancel()
>>>>>>>>>>>>>>> method doesn't really make sense so probably
>>> wouldn't
>>>>>> work.
>>>>>>>>>>> Likewise
>>>>>>>>>>>> I
>>>>>>>>>>>>>>> dislike the checked exceptions it requires.
>>>> Basically I
>>>>>>> just
>>>>>>>>>> wrote
>>>>>>>>>>>> out
>>>>>>>>>>>>>> some
>>>>>>>>>>>>>>> code examples and it seemed cleaner with a special
>>>>>> purpose
>>>>>>>>>> object.
>>>>>>>>>>> I
>>>>>>>>>>>>>> wasn't
>>>>>>>>>>>>>>> actually aware of plans for improved futures in
>>> java
>>>> 8
>>>>> or
>>>>>>> the
>>>>>>>>>> other
>>>>>>>>>>>>>>> integrations. Maybe you could elaborate on this a
>>> bit
>>>>> and
>>>>>>>> show
>>>>>>>>>> how
>>>>>>>>>>> it
>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>> be used? Sounds promising, I just don't know a lot
>>>>> about
>>>>>>> it.
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> -Jay
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> On Fri, Jan 24, 2014 at 3:30 PM, Clark Breyman <
>>>>>>>>>> clark@breyman.com>
>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> Jay - Thanks for the call for comments. Here's
>>> some
>>>>>>> initial
>>>>>>>>>>> input:
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> - Make message serialization a client
>>>> responsibility
>>>>>>>> (making
>>>>>>>>>> all
>>>>>>>>>>>>>> messages
>>>>>>>>>>>>>>>> byte[]). Reflection-based loading makes it
>>> harder
>>>> to
>>>>>> use
>>>>>>>>>> generic
>>>>>>>>>>>>> codecs
>>>>>>>>>>>>>>>> (e.g.  Envelope<PREFIX, DATA, SUFFIX>) or build
>>> up
>>>>>> codec
>>>>>>>>>>>>>>> programmatically.
>>>>>>>>>>>>>>>> Non-default partitioning should require an
>>> explicit
>>>>>>>> partition
>>>>>>>>>>> key.
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> - I really like the fact that it will be native
>>>> Java.
>>>>>>>> Please
>>>>>>>>>>>> consider
>>>>>>>>>>>>>>> using
>>>>>>>>>>>>>>>> native maven and not sbt, gradle, ivy, etc as
>>> they
>>>>>> don't
>>>>>>>>>> reliably
>>>>>>>>>>>>> play
>>>>>>>>>>>>>>> nice
>>>>>>>>>>>>>>>> in the maven ecosystem. A jar without a
>>> well-formed
>>>>> pom
>>>>>>>>> doesn't
>>>>>>>>>>>> feel
>>>>>>>>>>>>>>> like a
>>>>>>>>>>>>>>>> real artifact. The pom's generated by sbt et al.
>>>> are
>>>>>> not
>>>>>>>> well
>>>>>>>>>>>> formed.
>>>>>>>>>>>>>>> Using
>>>>>>>>>>>>>>>> maven will make builds and IDE integration much
>>>>>> smoother.
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> - Look at Nick Telford's dropwizard-extras
>>> package
>>>> in
>>>>>>> which
>>>>>>>>> he
>>>>>>>>>>>>> defines
>>>>>>>>>>>>>>> some
>>>>>>>>>>>>>>>> Jackson-compatible POJO's for loading
>>>> configuration.
>>>>>>> Seems
>>>>>>>>> like
>>>>>>>>>>>> your
>>>>>>>>>>>>>>> client
>>>>>>>>>>>>>>>> migration is similar. The config objects should
>>>> have
>>>>>>>>>> constructors
>>>>>>>>>>>> or
>>>>>>>>>>>>>>>> factories that accept Map<String, String> and
>>>>>> Properties
>>>>>>>> for
>>>>>>>>>> ease
>>>>>>>>>>>> of
>>>>>>>>>>>>>>>> migration.
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> - Would you consider using the org.apache.kafka
>>>>> package
>>>>>>> for
>>>>>>>>> the
>>>>>>>>>>> new
>>>>>>>>>>>>> API
>>>>>>>>>>>>>>>> (quibble)
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> - Why create your own futures rather than use
>>>>>>>>>>>>>>>> java.util.concurrent.Future<Long> or similar?
>>>>> Standard
>>>>>>>>> futures
>>>>>>>>>>> will
>>>>>>>>>>>>>> play
>>>>>>>>>>>>>>>> nice with other reactive libs and things like
>>> J8's
>>>>>>>>>>>> ComposableFuture.
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> Thanks again,
>>>>>>>>>>>>>>>> C
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> On Fri, Jan 24, 2014 at 2:46 PM, Roger Hoover <
>>>>>>>>>>>>> roger.hoover@gmail.com
>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> A couple comments:
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 1) Why does the config use a broker list
>>> instead
>>>> of
>>>>>>>>>> discovering
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>> brokers
>>>>>>>>>>>>>>>>> in ZooKeeper?  It doesn't match the
>>>>> HighLevelConsumer
>>>>>>>> API.
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 2) It looks like broker connections are
>>> created
>>>> on
>>>>>>>> demand.
>>>>>>>>>> I'm
>>>>>>>>>>>>>>> wondering
>>>>>>>>>>>>>>>>> if sometimes you might want to flush out
>>> config
>>>> or
>>>>>>>> network
>>>>>>>>>>>>>> connectivity
>>>>>>>>>>>>>>>>> issues before pushing the first message
>>> through.
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> Should there also be a
>>> KafkaProducer.connect() or
>>>>>>> .open()
>>>>>>>>>>> method
>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>> connectAll()?  I guess it would try to
>>> connect to
>>>>> all
>>>>>>>>> brokers
>>>>>>>>>>> in
>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> BROKER_LIST_CONFIG
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> HTH,
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> Roger
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> On Fri, Jan 24, 2014 at 11:54 AM, Jay Kreps <
>>>>>>>>>>> jay.kreps@gmail.com
>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> As mentioned in a previous email we are
>>> working
>>>>> on
>>>>>> a
>>>>>>>>>>>>>>> re-implementation
>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>> the producer. I would like to use this email
>>>>> thread
>>>>>>> to
>>>>>>>>>>> discuss
>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> details
>>>>>>>>>>>>>>>>>> of the public API and the configuration. I
>>>> would
>>>>>> love
>>>>>>>> for
>>>>>>>>>> us
>>>>>>>>>>> to
>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>> incredibly picky about this public api now
>>> so
>>>> it
>>>>> is
>>>>>>> as
>>>>>>>>> good
>>>>>>>>>>> as
>>>>>>>>>>>>>>> possible
>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>> we don't need to break it in the future.
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> The best way to get a feel for the API is
>>>>> actually
>>>>>> to
>>>>>>>>> take
>>>>>>>>>> a
>>>>>>>>>>>> look
>>>>>>>>>>>>>> at
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> javadoc, my hope is to get the api docs good
>>>>> enough
>>>>>>> so
>>>>>>>>> that
>>>>>>>>>>> it
>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>> self-explanatory:
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>> 
>>>> 
>>> http://empathybox.com/kafka-javadoc/index.html?kafka/clients/producer/KafkaProducer.html
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> Please take a look at this API and give me
>>> any
>>>>>>> thoughts
>>>>>>>>> you
>>>>>>>>>>> may
>>>>>>>>>>>>>> have!
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> It may also be reasonable to take a look at
>>> the
>>>>>>>> configs:
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>> 
>>>> 
>>> http://empathybox.com/kafka-javadoc/kafka/clients/producer/ProducerConfig.html
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> The actual code is posted here:
>>>>>>>>>>>>>>>>>> 
>>>> https://issues.apache.org/jira/browse/KAFKA-1227
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> A few questions or comments to kick things
>>> off:
>>>>>>>>>>>>>>>>>> 1. We need to make a decision on whether
>>>>>>> serialization
>>>>>>>> of
>>>>>>>>>> the
>>>>>>>>>>>>>> user's
>>>>>>>>>>>>>>>> key
>>>>>>>>>>>>>>>>>> and value should be done by the user (with
>>> our
>>>>> api
>>>>>>> just
>>>>>>>>>>> taking
>>>>>>>>>>>>>>> byte[])
>>>>>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>>> if we should take an object and allow the
>>> user
>>>> to
>>>>>>>>>> configure a
>>>>>>>>>>>>>>>> Serializer
>>>>>>>>>>>>>>>>>> class which we instantiate via reflection.
>>> We
>>>>> take
>>>>>>> the
>>>>>>>>>> later
>>>>>>>>>>>>>> approach
>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>> the current producer, and I have carried
>>> this
>>>>>> through
>>>>>>>> to
>>>>>>>>>> this
>>>>>>>>>>>>>>>> prototype.
>>>>>>>>>>>>>>>>>> The tradeoff I see is this: taking byte[] is
>>>>>> actually
>>>>>>>>>>> simpler,
>>>>>>>>>>>>> the
>>>>>>>>>>>>>>> user
>>>>>>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>> directly do whatever serialization they
>>> like.
>>>> The
>>>>>>>>>>> complication
>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>> actually
>>>>>>>>>>>>>>>>>> partitioning. Currently partitioning is done
>>>> by a
>>>>>>>> similar
>>>>>>>>>>>> plug-in
>>>>>>>>>>>>>> api
>>>>>>>>>>>>>>>>>> (Partitioner) which the user can implement
>>> and
>>>>>>>> configure
>>>>>>>>> to
>>>>>>>>>>>>>> override
>>>>>>>>>>>>>>>> how
>>>>>>>>>>>>>>>>>> partitions are assigned. If we take byte[]
>>> as
>>>>> input
>>>>>>>> then
>>>>>>>>> we
>>>>>>>>>>>> have
>>>>>>>>>>>>> no
>>>>>>>>>>>>>>>>> access
>>>>>>>>>>>>>>>>>> to the original object and partitioning
>>> MUST be
>>>>>> done
>>>>>>> on
>>>>>>>>> the
>>>>>>>>>>>>> byte[].
>>>>>>>>>>>>>>>> This
>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>> fine for hash partitioning. However for
>>> various
>>>>>> types
>>>>>>>> of
>>>>>>>>>>>> semantic
>>>>>>>>>>>>>>>>>> partitioning (range partitioning, or
>>> whatever)
>>>>> you
>>>>>>>> would
>>>>>>>>>> want
>>>>>>>>>>>>>> access
>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> original object. In the current approach a
>>>>> producer
>>>>>>> who
>>>>>>>>>>> wishes
>>>>>>>>>>>> to
>>>>>>>>>>>>>>> send
>>>>>>>>>>>>>>>>>> byte[] they have serialized in their own
>>> code
>>>> can
>>>>>>>>> configure
>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> BytesSerialization we supply which is just a
>>>> "no
>>>>>> op"
>>>>>>>>>>>>> serialization.
>>>>>>>>>>>>>>>>>> 2. We should obsess over naming and make
>>> sure
>>>>> each
>>>>>> of
>>>>>>>> the
>>>>>>>>>>> class
>>>>>>>>>>>>>> names
>>>>>>>>>>>>>>>> are
>>>>>>>>>>>>>>>>>> good.
>>>>>>>>>>>>>>>>>> 3. Jun has already pointed out that we need
>>> to
>>>>>>> include
>>>>>>>>> the
>>>>>>>>>>>> topic
>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>> partition in the response, which is
>>> absolutely
>>>>>>> right. I
>>>>>>>>>>> haven't
>>>>>>>>>>>>>> done
>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>> yet but that definitely needs to be there.
>>>>>>>>>>>>>>>>>> 4. Currently RecordSend.await will throw an
>>>>>> exception
>>>>>>>> if
>>>>>>>>>> the
>>>>>>>>>>>>>> request
>>>>>>>>>>>>>>>>>> failed. The intention here is that
>>>>>>>>>>>> producer.send(message).await()
>>>>>>>>>>>>>>>> exactly
>>>>>>>>>>>>>>>>>> simulates a synchronous call. Guozhang has
>>>> noted
>>>>>> that
>>>>>>>>> this
>>>>>>>>>>> is a
>>>>>>>>>>>>>>> little
>>>>>>>>>>>>>>>>>> annoying since the user must then catch
>>>>> exceptions.
>>>>>>>>> However
>>>>>>>>>>> if
>>>>>>>>>>>> we
>>>>>>>>>>>>>>>> remove
>>>>>>>>>>>>>>>>>> this then if the user doesn't check for
>>> errors
>>>>> they
>>>>>>>> won't
>>>>>>>>>>> know
>>>>>>>>>>>>> one
>>>>>>>>>>>>>>> has
>>>>>>>>>>>>>>>>>> occurred, which I predict will be a common
>>>>> mistake.
>>>>>>>>>>>>>>>>>> 5. Perhaps there is more we could do to make
>>>> the
>>>>>>> async
>>>>>>>>>>>> callbacks
>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>> future
>>>>>>>>>>>>>>>>>> we give back intuitive and easy to program
>>>>> against?
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> Some background info on implementation:
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> At a high level the primary difference in
>>> this
>>>>>>> producer
>>>>>>>>> is
>>>>>>>>>>> that
>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>> removes
>>>>>>>>>>>>>>>>>> the distinction between the "sync" and
>>> "async"
>>>>>>>> producer.
>>>>>>>>>>>>>> Effectively
>>>>>>>>>>>>>>>> all
>>>>>>>>>>>>>>>>>> requests are sent asynchronously but always
>>>>> return
>>>>>> a
>>>>>>>>> future
>>>>>>>>>>>>>> response
>>>>>>>>>>>>>>>>> object
>>>>>>>>>>>>>>>>>> that gives the offset as well as any error
>>> that
>>>>> may
>>>>>>>> have
>>>>>>>>>>>> occurred
>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> request is complete. The batching that is
>>> done
>>>> in
>>>>>> the
>>>>>>>>> async
>>>>>>>>>>>>>> producer
>>>>>>>>>>>>>>>> only
>>>>>>>>>>>>>>>>>> today is done whenever possible now. This
>>> means
>>>>>> that
>>>>>>>> the
>>>>>>>>>> sync
>>>>>>>>>>>>>>> producer,
>>>>>>>>>>>>>>>>>> under load, can get performance as good as
>>> the
>>>>>> async
>>>>>>>>>> producer
>>>>>>>>>>>>>>>>> (preliminary
>>>>>>>>>>>>>>>>>> results show the producer getting 1m
>>>>> messages/sec).
>>>>>>>> This
>>>>>>>>>>> works
>>>>>>>>>>>>>>> similar
>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>> group commit in databases but with respect
>>> to
>>>> the
>>>>>>>> actual
>>>>>>>>>>>> network
>>>>>>>>>>>>>>>>>> transmission--any messages that arrive
>>> while a
>>>>> send
>>>>>>> is
>>>>>>>> in
>>>>>>>>>>>>> progress
>>>>>>>>>>>>>>> are
>>>>>>>>>>>>>>>>>> batched together. It is also possible to
>>>>> encourage
>>>>>>>>> batching
>>>>>>>>>>>> even
>>>>>>>>>>>>>>> under
>>>>>>>>>>>>>>>>> low
>>>>>>>>>>>>>>>>>> load to save server resources by
>>> introducing a
>>>>>> delay
>>>>>>> on
>>>>>>>>> the
>>>>>>>>>>>> send
>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>> allow
>>>>>>>>>>>>>>>>>> more messages to accumulate; this is done
>>> using
>>>>> the
>>>>>>>>>>>>> linger.msconfig
>>>>>>>>>>>>>>>>> (this
>>>>>>>>>>>>>>>>>> is similar to Nagle's algorithm in TCP).
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> This producer does all network communication
>>>>>>>>> asynchronously
>>>>>>>>>>> and
>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>> parallel
>>>>>>>>>>>>>>>>>> to all servers so the performance penalty
>>> for
>>>>>> acks=-1
>>>>>>>> and
>>>>>>>>>>>> waiting
>>>>>>>>>>>>>> on
>>>>>>>>>>>>>>>>>> replication should be much reduced. I
>>> haven't
>>>>> done
>>>>>>> much
>>>>>>>>>>>>>> benchmarking
>>>>>>>>>>>>>>> on
>>>>>>>>>>>>>>>>>> this yet, though.
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> The high level design is described a little
>>>> here,
>>>>>>>> though
>>>>>>>>>> this
>>>>>>>>>>>> is
>>>>>>>>>>>>>> now
>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>> little out of date:
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/Client+Rewrite
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> -Jay
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>> 
>>>> 
>>> 
>> 
>> 

Re: New Producer Public API

Posted by Jay Kreps <ja...@gmail.com>.
The challenge of directly exposing ProduceRequestResult is that the offset
provided is just the base offset and there is no way to know for a
particular message where it was in relation to that base offset because the
batching is transparent and non-deterministic. So I think we do need some
kind of per-message result.

I started with Future<RequestResult>, I think for the same reason you
prefer it but then when I actually looked at some code samples it wasn't
too great--checked exceptions, methods that we can't easily implement, etc.
I moved away from that for two reasons:
1. When I actually wrote out some code samples of usage they were a little
ugly for the reasons I described--checked exceptions, methods we can't
implement, no helper methods, etc.
2. I originally intended to make the result send work like a
ListenableFuture so that you would register the callback on the result
rather than as part of the call. I moved away from this primarily because
the implementation complexity was a little higher.

Whether or not the code prettiness on its own outweighs the familiarity of
a normal Future I don't know, but that was the evolution of my thinking.

-Jay


On Wed, Jan 29, 2014 at 10:06 AM, Jay Kreps <ja...@gmail.com> wrote:

> Hey Neha,
>
> Error handling in RecordSend works as in Future you will get the exception
> if there is one from any of the accessor methods or await().
>
> The purpose of hasError was that you can write things slightly more simply
> (which some people expressed preference for):
>   if(send.hasError())
>     // do something
>   long offset = send.offset();
>
> Instead of the more the slightly longer:
> try {
>    long offset = send.offset();
> } catch (KafkaException e) {
>    // do something
> }
>
>
> On Wed, Jan 29, 2014 at 10:01 AM, Neha Narkhede <ne...@gmail.com>wrote:
>
>> Regarding the use of Futures -
>>
>> Agree that there are some downsides to using Futures but both approaches
>> have some tradeoffs.
>>
>> - Standardization and usability
>> Future is a widely used and understood Java API and given that the
>> functionality that RecordSend hopes to provide is essentially that of
>> Future, I think it makes sense to expose a widely understood public API
>> for
>> our clients. RecordSend, on the other hand, seems to provide some APIs
>> that
>> are very similar to that of Future, in addition to exposing a bunch of
>> APIs
>> that belong to ProduceRequestResult. As a user, I would've really
>> preferred
>> to deal with ProduceRequestResult directly -
>> Future<ProduceRequestResult> send(...)
>>
>> - Error handling
>> RecordSend's error handling is quite unintuitive where the user has to
>> remember to invoke hasError and error, instead of just throwing the
>> exception. Now there are
>> some downsides regarding error handling with the Future as well, where the
>> user has to catch InterruptedException when we would never run into it.
>> However, it seems like a price worth paying for supporting a standard API
>> and error handling
>>
>> - Unused APIs
>> This is a downside of using Future, where the cancel() operation would
>> always return false and mean nothing. But we can mention that caveat in
>> our
>> Java docs.
>>
>> To summarize, I would prefer to expose a well understood and widely
>> adopted
>> Java API and put up with the overhead of catching one unnecessary checked
>> exception, rather than wrap the useful ProduceRequestResult in a custom
>> async object (RecordSend) and explain that to our many users.
>>
>> Thanks,
>> Neha
>>
>>
>>
>>
>> On Tue, Jan 28, 2014 at 8:10 PM, Jay Kreps <ja...@gmail.com> wrote:
>>
>> > Hey Neha,
>> >
>> > Can you elaborate on why you prefer using Java's Future? The downside
>> in my
>> > mind is the use of the checked InterruptedException and
>> ExecutionException.
>> > ExecutionException is arguable, but forcing you to catch
>> > InterruptedException, often in code that can't be interrupted, seems
>> > perverse. It also leaves us with the cancel() method which I don't
>> think we
>> > really can implement.
>> >
>> > Option 1A, to recap/elaborate, was the following. There is no
>> Serializer or
>> > Partitioner api. We take a byte[] key and value and an optional integer
>> > partition. If you specify the integer partition it will be used. If you
>> do
>> > not specify a key or a partition the partition will be chosen in a round
>> > robin fashion. If you specify a key but no partition we will chose a
>> > partition based on a hash of the key. In order to let the user find the
>> > partition we will need to given them access to the Cluster instance
>> > directly from the producer.
>> >
>> > -Jay
>> >
>> >
>> > On Tue, Jan 28, 2014 at 6:25 PM, Neha Narkhede <neha.narkhede@gmail.com
>> > >wrote:
>> >
>> > > Here are more thoughts on the public APIs -
>> > >
>> > > - I suggest we use java's Future instead of custom Future especially
>> > since
>> > > it is part of the public API
>> > >
>> > > - Serialization: I like the simplicity of the producer APIs with the
>> > > absence of serialization where we just deal with byte arrays for keys
>> and
>> > > values. What I don't like about this is the performance overhead on
>> the
>> > > Partitioner for any kind of custom partitioning based on the
>> > partitionKey.
>> > > Since the only purpose of partitionKey is to do custom partitioning,
>> why
>> > > can't we take it in directly as an integer and let the user figure out
>> > the
>> > > mapping from partition_key -> partition_id using the getCluster() API?
>> > If I
>> > > understand correctly, this is similar to what you suggested as part of
>> > > option 1A. I like this approach since it maintains the simplicity of
>> APIs
>> > > by allowing us to deal with bytes and does not compromise performance
>> in
>> > > the custom partitioning case.
>> > >
>> > > Thanks,
>> > > Neha
>> > >
>> > >
>> > >
>> > > On Tue, Jan 28, 2014 at 5:42 PM, Jay Kreps <ja...@gmail.com>
>> wrote:
>> > >
>> > > > Hey Tom,
>> > > >
>> > > > That sounds cool. How did you end up handling parallel I/O if you
>> wrap
>> > > the
>> > > > individual connections? Don't you need some selector that selects
>> over
>> > > all
>> > > > the connections?
>> > > >
>> > > > -Jay
>> > > >
>> > > >
>> > > > On Tue, Jan 28, 2014 at 2:31 PM, Tom Brown <to...@gmail.com>
>> > wrote:
>> > > >
>> > > > > I implemented a 0.7 client in pure java, and its API very closely
>> > > > resembled
>> > > > > this. (When multiple people independently engineer the same
>> solution,
>> > > > it's
>> > > > > probably good... right?). However, there were a few architectural
>> > > > > differences with my client:
>> > > > >
>> > > > > 1. The basic client itself was just an asynchronous layer around
>> the
>> > > > > different server functions. In and of itself it had no knowledge
>> of
>> > > > > partitions, only servers (and maintained TCP connections to them).
>> > > > >
>> > > > > 2. The main producer was an additional layer that provided a
>> > high-level
>> > > > > interface that could batch individual messages based on partition.
>> > > > >
>> > > > > 3. Knowledge of partitioning was done via an interface so that
>> > > different
>> > > > > strategies could be used.
>> > > > >
>> > > > > 4. Partitioning was done by the user, with knowledge of the
>> available
>> > > > > partitions provided by #3.
>> > > > >
>> > > > > 5. Serialization was done by the user to simplify the API.
>> > > > >
>> > > > > 6. Futures were used to make asynchronous emulate synchronous
>> calls.
>> > > > >
>> > > > >
>> > > > > The main benefit of this approach is flexibility. For example,
>> since
>> > > the
>> > > > > base client was just a managed connection (and not inherently a
>> > > > producer),
>> > > > > it was easy to composite a produce request and an offsets request
>> > > > together
>> > > > > into a confirmed produce request (officially not available in
>> 0.7).
>> > > > >
>> > > > > Decoupling the basic client from partition management allowed the
>> me
>> > to
>> > > > > implement zk discovery as a separate project so that the main
>> project
>> > > had
>> > > > > no complex dependencies. The same was true of decoupling
>> > serialization.
>> > > > > It's trivial to build an optional layer that adds those features
>> in,
>> > > > while
>> > > > > allowing access to the base APIs for those that need it.
>> > > > >
>> > > > > Using standard Future objects was also beneficial, since I could
>> > > combine
>> > > > > them with existing tools (such as guava).
>> > > > >
>> > > > > It may be too late to be of use, but I have been working with my
>> > > > company's
>> > > > > legal department to release the implementation I described above.
>> If
>> > > > you're
>> > > > > interested in it, let me know.
>> > > > >
>> > > > >
>> > > > > To sum up my thoughts regarding the new API, I think it's a great
>> > > start.
>> > > > I
>> > > > > would like to see a more layered approach so I can use the parts I
>> > > want,
>> > > > > and adapt the other parts as needed. I would also like to see
>> > standard
>> > > > > interfaces (especially Future) used where they makes sense.
>> > > > >
>> > > > > --Tom
>> > > > >
>> > > > >
>> > > > >
>> > > > >
>> > > > >
>> > > > > On Tue, Jan 28, 2014 at 1:33 PM, Roger Hoover <
>> > roger.hoover@gmail.com
>> > > > > >wrote:
>> > > > >
>> > > > > > +1 ListenableFuture: If this works similar to Deferreds in
>> Twisted
>> > > > Python
>> > > > > > or Promised IO in Javascript, I think this is a great pattern
>> for
>> > > > > > decoupling your callback logic from the place where the Future
>> is
>> > > > > > generated.  You can register as many callbacks as you like,
>> each in
>> > > the
>> > > > > > appropriate layer of the code and have each observer get
>> notified
>> > > when
>> > > > > the
>> > > > > > promised i/o is complete without any of them knowing about each
>> > > other.
>> > > > > >
>> > > > > >
>> > > > > > On Tue, Jan 28, 2014 at 11:32 AM, Jay Kreps <
>> jay.kreps@gmail.com>
>> > > > wrote:
>> > > > > >
>> > > > > > > Hey Ross,
>> > > > > > >
>> > > > > > > - ListenableFuture: Interesting. That would be an alternative
>> to
>> > > the
>> > > > > > direct
>> > > > > > > callback support we provide. There could be pros to this, let
>> me
>> > > > think
>> > > > > > > about it.
>> > > > > > > - We could provide layering, but I feel that the
>> serialization is
>> > > > such
>> > > > > a
>> > > > > > > small thing we should just make a decision and chose one, it
>> > > doesn't
>> > > > > seem
>> > > > > > > to me to justify a whole public facing layer.
>> > > > > > > - Yes, this is fairly esoteric, essentially I think it is
>> fairly
>> > > > > similar
>> > > > > > to
>> > > > > > > databases like DynamoDB that allow you to specify two
>> partition
>> > > keys
>> > > > (I
>> > > > > > > think DynamoDB does this...). The reasoning is that in fact
>> there
>> > > are
>> > > > > > > several things you can use the key field for: (1) to compute
>> the
>> > > > > > partition
>> > > > > > > to store the data in, (2) as a unique identifier to
>> deduplicate
>> > > that
>> > > > > > > partition's records within a log. These two things are almost
>> > > always
>> > > > > the
>> > > > > > > same, but occationally may differ when you want to group data
>> in
>> > a
>> > > > more
>> > > > > > > sophisticated way then just a hash of the primary key but
>> still
>> > > > retain
>> > > > > > the
>> > > > > > > proper primary key for delivery to the consumer and log
>> > compaction.
>> > > > > > >
>> > > > > > > -Jay
>> > > > > > >
>> > > > > > >
>> > > > > > > On Tue, Jan 28, 2014 at 3:24 AM, Ross Black <
>> > > ross.w.black@gmail.com>
>> > > > > > > wrote:
>> > > > > > >
>> > > > > > > > Hi Jay,
>> > > > > > > >
>> > > > > > > > - Just to add some more info/confusion about possibly using
>> > > Future
>> > > > > ...
>> > > > > > > >   If Kafka uses a JDK future, it plays nicely with other
>> > > frameworks
>> > > > > as
>> > > > > > > > well.
>> > > > > > > >   Google Guava has a ListenableFuture that allows callback
>> > > handling
>> > > > > to
>> > > > > > be
>> > > > > > > > added via the returned future, and allows the callbacks to
>> be
>> > > > passed
>> > > > > > off
>> > > > > > > to
>> > > > > > > > a specified executor.
>> > > > > > > >
>> > > > > > > >
>> > > > > > > >
>> > > > > > >
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>> http://docs.guava-libraries.googlecode.com/git-history/release/javadoc/com/google/common/util/concurrent/ListenableFuture.html
>> > > > > > > >   The JDK future can easily be converted to a listenable
>> > future.
>> > > > > > > >
>> > > > > > > > - On the question of byte[] vs Object, could this be solved
>> by
>> > > > > layering
>> > > > > > > the
>> > > > > > > > API?  eg. a raw producer (use byte[] and specify the
>> partition
>> > > > > number)
>> > > > > > > and
>> > > > > > > > a normal producer (use generic object and specify a
>> > Partitioner)?
>> > > > > > > >
>> > > > > > > > - I am confused by the keys in ProducerRecord and
>> Partitioner.
>> > > >  What
>> > > > > is
>> > > > > > > the
>> > > > > > > > usage for both a key and a partition key? (I am not yet
>> using
>> > > 0.8)
>> > > > > > > >
>> > > > > > > >
>> > > > > > > > Thanks,
>> > > > > > > > Ross
>> > > > > > > >
>> > > > > > > >
>> > > > > > > >
>> > > > > > > > On 28 January 2014 05:00, Xavier Stevens <xavier@gaikai.com
>> >
>> > > > wrote:
>> > > > > > > >
>> > > > > > > > > AutoCloseable would be nice for us as most of our code is
>> > using
>> > > > > Java
>> > > > > > 7
>> > > > > > > at
>> > > > > > > > > this point.
>> > > > > > > > >
>> > > > > > > > > I like Dropwizard's configuration mapping to POJOs via
>> > Jackson,
>> > > > but
>> > > > > > if
>> > > > > > > > you
>> > > > > > > > > wanted to stick with property maps I don't care enough to
>> > > object.
>> > > > > > > > >
>> > > > > > > > > If the producer only dealt with bytes, is there a way we
>> > could
>> > > > > still
>> > > > > > > due
>> > > > > > > > > partition plugins without specifying the number
>> explicitly? I
>> > > > would
>> > > > > > > > prefer
>> > > > > > > > > to be able to pass in field(s) that would be used by the
>> > > > > partitioner.
>> > > > > > > > > Obviously if this wasn't possible you could always
>> > deserialize
>> > > > the
>> > > > > > > object
>> > > > > > > > > in the partitioner and grab the fields you want, but that
>> > seems
>> > > > > > really
>> > > > > > > > > expensive to do on every message.
>> > > > > > > > >
>> > > > > > > > > It would also be nice to have a Java API Encoder
>> constructor
>> > > > taking
>> > > > > > in
>> > > > > > > > > VerifiableProperties. Scala understands how to handle
>> "props:
>> > > > > > > > > VerifiableProperties = null", but Java doesn't. So you
>> don't
>> > > run
>> > > > > into
>> > > > > > > > this
>> > > > > > > > > problem until runtime.
>> > > > > > > > >
>> > > > > > > > >
>> > > > > > > > > -Xavier
>> > > > > > > > >
>> > > > > > > > >
>> > > > > > > > > On Mon, Jan 27, 2014 at 9:37 AM, Clark Breyman <
>> > > > clark@breyman.com>
>> > > > > > > > wrote:
>> > > > > > > > >
>> > > > > > > > > > Jay -
>> > > > > > > > > >
>> > > > > > > > > > Config - your explanation makes sense. I'm just so
>> > accustomed
>> > > > to
>> > > > > > > having
>> > > > > > > > > > Jackson automatically map my configuration objects to
>> POJOs
>> > > > that
>> > > > > > I've
>> > > > > > > > > > stopped using property files. They are lingua franca.
>> The
>> > > only
>> > > > > > > thought
>> > > > > > > > > > might be to separate the config interface from the
>> > > > implementation
>> > > > > > to
>> > > > > > > > > allow
>> > > > > > > > > > for alternatives, but that might undermine your point of
>> > "do
>> > > it
>> > > > > > this
>> > > > > > > > way
>> > > > > > > > > so
>> > > > > > > > > > that everyone can find it where they expect it".
>> > > > > > > > > >
>> > > > > > > > > > Serialization: Of the options, I like 1A the best,
>> though
>> > > > > possibly
>> > > > > > > with
>> > > > > > > > > > either an option to specify a partition key rather than
>> ID
>> > > or a
>> > > > > > > helper
>> > > > > > > > to
>> > > > > > > > > > translate an arbitrary byte[] or long into a partition
>> > > number.
>> > > > > > > > > >
>> > > > > > > > > > Thanks
>> > > > > > > > > > Clark
>> > > > > > > > > >
>> > > > > > > > > >
>> > > > > > > > > > On Sun, Jan 26, 2014 at 9:13 PM, Jay Kreps <
>> > > > jay.kreps@gmail.com>
>> > > > > > > > wrote:
>> > > > > > > > > >
>> > > > > > > > > > > Thanks for the detailed thoughts. Let me elaborate on
>> the
>> > > > > config
>> > > > > > > > thing.
>> > > > > > > > > > >
>> > > > > > > > > > > I agree that at first glance key-value strings don't
>> seem
>> > > > like
>> > > > > a
>> > > > > > > very
>> > > > > > > > > > good
>> > > > > > > > > > > configuration api for a client. Surely a well-typed
>> > config
>> > > > > class
>> > > > > > > > would
>> > > > > > > > > be
>> > > > > > > > > > > better! I actually disagree and let me see if I can
>> > > convince
>> > > > > you.
>> > > > > > > > > > >
>> > > > > > > > > > > My reasoning has nothing to do with the api and
>> > everything
>> > > to
>> > > > > do
>> > > > > > > with
>> > > > > > > > > > > operations.
>> > > > > > > > > > >
>> > > > > > > > > > > Clients are embedded in applications which are
>> themselves
>> > > > > > > configured.
>> > > > > > > > > In
>> > > > > > > > > > > any place that takes operations seriously the
>> > configuration
>> > > > for
>> > > > > > > these
>> > > > > > > > > > > applications will be version controlled and maintained
>> > > > through
>> > > > > > some
>> > > > > > > > > kind
>> > > > > > > > > > of
>> > > > > > > > > > > config management system. If we give a config class
>> with
>> > > > > getters
>> > > > > > > and
>> > > > > > > > > > > setters the application has to expose those
>> properties to
>> > > its
>> > > > > > > > > > > configuration. What invariably happens is that the
>> > > > application
>> > > > > > > > exposes
>> > > > > > > > > > only
>> > > > > > > > > > > a choice few properties that they thought they would
>> > > change.
>> > > > > > > > > Furthermore
>> > > > > > > > > > > the application will make up a name for these configs
>> > that
>> > > > > seems
>> > > > > > > > > > intuitive
>> > > > > > > > > > > at the time in the 2 seconds the engineer spends
>> thinking
>> > > > about
>> > > > > > it.
>> > > > > > > > > > >
>> > > > > > > > > > > Now consider the result of this in the large. You end
>> up
>> > > with
>> > > > > > > dozens
>> > > > > > > > or
>> > > > > > > > > > > hundreds of applications that have the client
>> embedded.
>> > > Each
>> > > > > > > exposes
>> > > > > > > > a
>> > > > > > > > > > > different, inadequate subset of the possible configs,
>> > each
>> > > > with
>> > > > > > > > > different
>> > > > > > > > > > > names. It is a nightmare.
>> > > > > > > > > > >
>> > > > > > > > > > > If you use a string-string map the config system can
>> > > directly
>> > > > > > get a
>> > > > > > > > > > bundle
>> > > > > > > > > > > of config key-value pairs and put them into the
>> client.
>> > > This
>> > > > > > means
>> > > > > > > > that
>> > > > > > > > > > all
>> > > > > > > > > > > configuration is automatically available with the name
>> > > > > documented
>> > > > > > > on
>> > > > > > > > > the
>> > > > > > > > > > > website in every application that does this. If you
>> > upgrade
>> > > > to
>> > > > > a
>> > > > > > > new
>> > > > > > > > > > kafka
>> > > > > > > > > > > version with more configs those will be exposed too.
>> If
>> > you
>> > > > > > realize
>> > > > > > > > > that
>> > > > > > > > > > > you need to change a default you can just go through
>> your
>> > > > > configs
>> > > > > > > and
>> > > > > > > > > > > change it everywhere as it will have the same name
>> > > > everywhere.
>> > > > > > > > > > >
>> > > > > > > > > > > -Jay
>> > > > > > > > > > >
>> > > > > > > > > > >
>> > > > > > > > > > >
>> > > > > > > > > > >
>> > > > > > > > > > > On Sun, Jan 26, 2014 at 4:47 PM, Clark Breyman <
>> > > > > > clark@breyman.com>
>> > > > > > > > > > wrote:
>> > > > > > > > > > >
>> > > > > > > > > > > > Thanks Jay. I'll see if I can put together a more
>> > > complete
>> > > > > > > > response,
>> > > > > > > > > > > > perhaps as separate threads so that topics don't get
>> > > > > entangled.
>> > > > > > > In
>> > > > > > > > > the
>> > > > > > > > > > > mean
>> > > > > > > > > > > > time, here's a couple responses:
>> > > > > > > > > > > >
>> > > > > > > > > > > > Serialization: you've broken out a sub-thread so
>> i'll
>> > > reply
>> > > > > > > there.
>> > > > > > > > My
>> > > > > > > > > > > bias
>> > > > > > > > > > > > is that I like generics (except for type-erasure)
>> and
>> > in
>> > > > > > > particular
>> > > > > > > > > > they
>> > > > > > > > > > > > make it easy to compose serializers for compound
>> > payloads
>> > > > > (e.g.
>> > > > > > > > when
>> > > > > > > > > a
>> > > > > > > > > > > > common header wraps a payload of parameterized
>> type).
>> > > I'll
>> > > > > > > respond
>> > > > > > > > to
>> > > > > > > > > > > your
>> > > > > > > > > > > > 4-options message with an example.
>> > > > > > > > > > > >
>> > > > > > > > > > > > Build: I've seen a lot of "maven-compatible" build
>> > > systems
>> > > > > > > produce
>> > > > > > > > > > > > "artifacts" that aren't really artifacts - no
>> embedded
>> > > POM
>> > > > > or,
>> > > > > > > > worst,
>> > > > > > > > > > > > malformed POM. I know the sbt-generated artifacts
>> were
>> > > this
>> > > > > > way -
>> > > > > > > > > onus
>> > > > > > > > > > is
>> > > > > > > > > > > > on me to see what gradle is spitting out and what a
>> > maven
>> > > > > build
>> > > > > > > > might
>> > > > > > > > > > > look
>> > > > > > > > > > > > like. Maven may be old and boring, but it gets out
>> of
>> > the
>> > > > way
>> > > > > > and
>> > > > > > > > > > > > integrates really seamlessly with a lot of IDEs.
>> When
>> > > some
>> > > > > > scala
>> > > > > > > > > > > projects I
>> > > > > > > > > > > > was working on in the fall of 2011 switched from
>> sbt to
>> > > > > maven,
>> > > > > > > > build
>> > > > > > > > > > > became
>> > > > > > > > > > > > a non-issue.
>> > > > > > > > > > > >
>> > > > > > > > > > > > Config: Not a big deal  and no, I don't think a
>> > > dropwizard
>> > > > > > > > dependency
>> > > > > > > > > > is
>> > > > > > > > > > > > appropriate. I do like using simple entity beans
>> > (POJO's
>> > > > not
>> > > > > > > j2EE)
>> > > > > > > > > for
>> > > > > > > > > > > > configuration, especially if they can be marshalled
>> > > without
>> > > > > > > > > annotation
>> > > > > > > > > > by
>> > > > > > > > > > > > Jackson. I only mentioned the dropwizard-extras
>> >  because
>> > > it
>> > > > > has
>> > > > > > > > some
>> > > > > > > > > > > entity
>> > > > > > > > > > > > bean versions of the ZK and Kafka configs.
>> > > > > > > > > > > >
>> > > > > > > > > > > > Domain-packaging: Also not a big deal - it's what's
>> > > > expected
>> > > > > > and
>> > > > > > > > it's
>> > > > > > > > > > > > pretty free in most IDE's. The advantages I see is
>> that
>> > > it
>> > > > is
>> > > > > > > clear
>> > > > > > > > > > > whether
>> > > > > > > > > > > > something is from the Apache Kafka project and
>> whether
>> > > > > > something
>> > > > > > > is
>> > > > > > > > > > from
>> > > > > > > > > > > > another org and related to Kafka. That said, nothing
>> > > really
>> > > > > > > > enforces
>> > > > > > > > > > it.
>> > > > > > > > > > > >
>> > > > > > > > > > > > Futures: I'll see if I can create some examples to
>> > > > > demonstrate
>> > > > > > > > Future
>> > > > > > > > > > > > making interop easier.
>> > > > > > > > > > > >
>> > > > > > > > > > > > Regards,
>> > > > > > > > > > > > C
>> > > > > > > > > > > >
>> > > > > > > > > > > >
>> > > > > > > > > > > >
>> > > > > > > > > > > >
>> > > > > > > > > > > > On Fri, Jan 24, 2014 at 4:36 PM, Jay Kreps <
>> > > > > > jay.kreps@gmail.com>
>> > > > > > > > > > wrote:
>> > > > > > > > > > > >
>> > > > > > > > > > > > > Hey Clark,
>> > > > > > > > > > > > >
>> > > > > > > > > > > > > - Serialization: Yes I agree with these though I
>> > don't
>> > > > > > consider
>> > > > > > > > the
>> > > > > > > > > > > loss
>> > > > > > > > > > > > of
>> > > > > > > > > > > > > generics a big issue. I'll try to summarize what I
>> > > would
>> > > > > > > consider
>> > > > > > > > > the
>> > > > > > > > > > > > best
>> > > > > > > > > > > > > alternative api with raw byte[].
>> > > > > > > > > > > > >
>> > > > > > > > > > > > > - Maven: We had this debate a few months back and
>> the
>> > > > > > consensus
>> > > > > > > > was
>> > > > > > > > > > > > gradle.
>> > > > > > > > > > > > > Is there a specific issue with the poms gradle
>> > makes? I
>> > > > am
>> > > > > > > > > extremely
>> > > > > > > > > > > > loath
>> > > > > > > > > > > > > to revisit the issue as build issues are a
>> recurring
>> > > > thing
>> > > > > > and
>> > > > > > > no
>> > > > > > > > > one
>> > > > > > > > > > > > ever
>> > > > > > > > > > > > > agrees and ultimately our build needs are very
>> > simple.
>> > > > > > > > > > > > >
>> > > > > > > > > > > > > - Config: I'm not sure if I follow the point. Are
>> you
>> > > > > saying
>> > > > > > we
>> > > > > > > > > > should
>> > > > > > > > > > > > use
>> > > > > > > > > > > > > something in dropwizard for config? One principle
>> > here
>> > > is
>> > > > > to
>> > > > > > > try
>> > > > > > > > to
>> > > > > > > > > > > > remove
>> > > > > > > > > > > > > as many client dependencies as possible as we
>> > > inevitably
>> > > > > run
>> > > > > > > into
>> > > > > > > > > > > > terrible
>> > > > > > > > > > > > > compatibility issues with users who use the same
>> > > library
>> > > > or
>> > > > > > its
>> > > > > > > > > > > > > dependencies at different versions. Or are you
>> > talking
>> > > > > about
>> > > > > > > > > > > maintaining
>> > > > > > > > > > > > > compatibility with existing config parameters? I
>> > think
>> > > as
>> > > > > > much
>> > > > > > > > as a
>> > > > > > > > > > > > config
>> > > > > > > > > > > > > in the existing client makes sense it should have
>> the
>> > > > same
>> > > > > > name
>> > > > > > > > (I
>> > > > > > > > > > was
>> > > > > > > > > > > a
>> > > > > > > > > > > > > bit sloppy about that so I'll fix any errors
>> there).
>> > > > There
>> > > > > > are
>> > > > > > > a
>> > > > > > > > > few
>> > > > > > > > > > > new
>> > > > > > > > > > > > > things and we should give those reasonable
>> defaults.
>> > I
>> > > > > think
>> > > > > > > > config
>> > > > > > > > > > is
>> > > > > > > > > > > > > important so I'll start a thread on the config
>> > package
>> > > in
>> > > > > > > there.
>> > > > > > > > > > > > >
>> > > > > > > > > > > > > - org.apache.kafka: We could do this. I always
>> > > considered
>> > > > > it
>> > > > > > > kind
>> > > > > > > > > of
>> > > > > > > > > > an
>> > > > > > > > > > > > odd
>> > > > > > > > > > > > > thing Java programmers do that has no real
>> motivation
>> > > > (but
>> > > > > I
>> > > > > > > > could
>> > > > > > > > > be
>> > > > > > > > > > > > > re-educated!). I don't think it ends up reducing
>> > naming
>> > > > > > > conflicts
>> > > > > > > > > in
>> > > > > > > > > > > > > practice and it adds a lot of noise and nested
>> > > > directories.
>> > > > > > Is
>> > > > > > > > > there
>> > > > > > > > > > a
>> > > > > > > > > > > > > reason you prefer this or just to be more
>> standard?
>> > > > > > > > > > > > >
>> > > > > > > > > > > > > - Future: Basically I didn't see any particular
>> > > > advantage.
>> > > > > > The
>> > > > > > > > > > cancel()
>> > > > > > > > > > > > > method doesn't really make sense so probably
>> wouldn't
>> > > > work.
>> > > > > > > > > Likewise
>> > > > > > > > > > I
>> > > > > > > > > > > > > dislike the checked exceptions it requires.
>> > Basically I
>> > > > > just
>> > > > > > > > wrote
>> > > > > > > > > > out
>> > > > > > > > > > > > some
>> > > > > > > > > > > > > code examples and it seemed cleaner with a special
>> > > > purpose
>> > > > > > > > object.
>> > > > > > > > > I
>> > > > > > > > > > > > wasn't
>> > > > > > > > > > > > > actually aware of plans for improved futures in
>> java
>> > 8
>> > > or
>> > > > > the
>> > > > > > > > other
>> > > > > > > > > > > > > integrations. Maybe you could elaborate on this a
>> bit
>> > > and
>> > > > > > show
>> > > > > > > > how
>> > > > > > > > > it
>> > > > > > > > > > > > would
>> > > > > > > > > > > > > be used? Sounds promising, I just don't know a lot
>> > > about
>> > > > > it.
>> > > > > > > > > > > > >
>> > > > > > > > > > > > > -Jay
>> > > > > > > > > > > > >
>> > > > > > > > > > > > >
>> > > > > > > > > > > > > On Fri, Jan 24, 2014 at 3:30 PM, Clark Breyman <
>> > > > > > > > clark@breyman.com>
>> > > > > > > > > > > > wrote:
>> > > > > > > > > > > > >
>> > > > > > > > > > > > > > Jay - Thanks for the call for comments. Here's
>> some
>> > > > > initial
>> > > > > > > > > input:
>> > > > > > > > > > > > > >
>> > > > > > > > > > > > > > - Make message serialization a client
>> > responsibility
>> > > > > > (making
>> > > > > > > > all
>> > > > > > > > > > > > messages
>> > > > > > > > > > > > > > byte[]). Reflection-based loading makes it
>> harder
>> > to
>> > > > use
>> > > > > > > > generic
>> > > > > > > > > > > codecs
>> > > > > > > > > > > > > > (e.g.  Envelope<PREFIX, DATA, SUFFIX>) or build
>> up
>> > > > codec
>> > > > > > > > > > > > > programmatically.
>> > > > > > > > > > > > > > Non-default partitioning should require an
>> explicit
>> > > > > > partition
>> > > > > > > > > key.
>> > > > > > > > > > > > > >
>> > > > > > > > > > > > > > - I really like the fact that it will be native
>> > Java.
>> > > > > > Please
>> > > > > > > > > > consider
>> > > > > > > > > > > > > using
>> > > > > > > > > > > > > > native maven and not sbt, gradle, ivy, etc as
>> they
>> > > > don't
>> > > > > > > > reliably
>> > > > > > > > > > > play
>> > > > > > > > > > > > > nice
>> > > > > > > > > > > > > > in the maven ecosystem. A jar without a
>> well-formed
>> > > pom
>> > > > > > > doesn't
>> > > > > > > > > > feel
>> > > > > > > > > > > > > like a
>> > > > > > > > > > > > > > real artifact. The pom's generated by sbt et al.
>> > are
>> > > > not
>> > > > > > well
>> > > > > > > > > > formed.
>> > > > > > > > > > > > > Using
>> > > > > > > > > > > > > > maven will make builds and IDE integration much
>> > > > smoother.
>> > > > > > > > > > > > > >
>> > > > > > > > > > > > > > - Look at Nick Telford's dropwizard-extras
>> package
>> > in
>> > > > > which
>> > > > > > > he
>> > > > > > > > > > > defines
>> > > > > > > > > > > > > some
>> > > > > > > > > > > > > > Jackson-compatible POJO's for loading
>> > configuration.
>> > > > > Seems
>> > > > > > > like
>> > > > > > > > > > your
>> > > > > > > > > > > > > client
>> > > > > > > > > > > > > > migration is similar. The config objects should
>> > have
>> > > > > > > > constructors
>> > > > > > > > > > or
>> > > > > > > > > > > > > > factories that accept Map<String, String> and
>> > > > Properties
>> > > > > > for
>> > > > > > > > ease
>> > > > > > > > > > of
>> > > > > > > > > > > > > > migration.
>> > > > > > > > > > > > > >
>> > > > > > > > > > > > > > - Would you consider using the org.apache.kafka
>> > > package
>> > > > > for
>> > > > > > > the
>> > > > > > > > > new
>> > > > > > > > > > > API
>> > > > > > > > > > > > > > (quibble)
>> > > > > > > > > > > > > >
>> > > > > > > > > > > > > > - Why create your own futures rather than use
>> > > > > > > > > > > > > > java.util.concurrent.Future<Long> or similar?
>> > > Standard
>> > > > > > > futures
>> > > > > > > > > will
>> > > > > > > > > > > > play
>> > > > > > > > > > > > > > nice with other reactive libs and things like
>> J8's
>> > > > > > > > > > ComposableFuture.
>> > > > > > > > > > > > > >
>> > > > > > > > > > > > > > Thanks again,
>> > > > > > > > > > > > > > C
>> > > > > > > > > > > > > >
>> > > > > > > > > > > > > >
>> > > > > > > > > > > > > >
>> > > > > > > > > > > > > > On Fri, Jan 24, 2014 at 2:46 PM, Roger Hoover <
>> > > > > > > > > > > roger.hoover@gmail.com
>> > > > > > > > > > > > > > >wrote:
>> > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > A couple comments:
>> > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > 1) Why does the config use a broker list
>> instead
>> > of
>> > > > > > > > discovering
>> > > > > > > > > > the
>> > > > > > > > > > > > > > brokers
>> > > > > > > > > > > > > > > in ZooKeeper?  It doesn't match the
>> > > HighLevelConsumer
>> > > > > > API.
>> > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > 2) It looks like broker connections are
>> created
>> > on
>> > > > > > demand.
>> > > > > > > >  I'm
>> > > > > > > > > > > > > wondering
>> > > > > > > > > > > > > > > if sometimes you might want to flush out
>> config
>> > or
>> > > > > > network
>> > > > > > > > > > > > connectivity
>> > > > > > > > > > > > > > > issues before pushing the first message
>> through.
>> > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > Should there also be a
>> KafkaProducer.connect() or
>> > > > > .open()
>> > > > > > > > > method
>> > > > > > > > > > or
>> > > > > > > > > > > > > > > connectAll()?  I guess it would try to
>> connect to
>> > > all
>> > > > > > > brokers
>> > > > > > > > > in
>> > > > > > > > > > > the
>> > > > > > > > > > > > > > > BROKER_LIST_CONFIG
>> > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > HTH,
>> > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > Roger
>> > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > On Fri, Jan 24, 2014 at 11:54 AM, Jay Kreps <
>> > > > > > > > > jay.kreps@gmail.com
>> > > > > > > > > > >
>> > > > > > > > > > > > > wrote:
>> > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > As mentioned in a previous email we are
>> working
>> > > on
>> > > > a
>> > > > > > > > > > > > > re-implementation
>> > > > > > > > > > > > > > of
>> > > > > > > > > > > > > > > > the producer. I would like to use this email
>> > > thread
>> > > > > to
>> > > > > > > > > discuss
>> > > > > > > > > > > the
>> > > > > > > > > > > > > > > details
>> > > > > > > > > > > > > > > > of the public API and the configuration. I
>> > would
>> > > > love
>> > > > > > for
>> > > > > > > > us
>> > > > > > > > > to
>> > > > > > > > > > > be
>> > > > > > > > > > > > > > > > incredibly picky about this public api now
>> so
>> > it
>> > > is
>> > > > > as
>> > > > > > > good
>> > > > > > > > > as
>> > > > > > > > > > > > > possible
>> > > > > > > > > > > > > > > and
>> > > > > > > > > > > > > > > > we don't need to break it in the future.
>> > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > The best way to get a feel for the API is
>> > > actually
>> > > > to
>> > > > > > > take
>> > > > > > > > a
>> > > > > > > > > > look
>> > > > > > > > > > > > at
>> > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > javadoc, my hope is to get the api docs good
>> > > enough
>> > > > > so
>> > > > > > > that
>> > > > > > > > > it
>> > > > > > > > > > is
>> > > > > > > > > > > > > > > > self-explanatory:
>> > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > >
>> > > > > > > > > > > > > >
>> > > > > > > > > > > > >
>> > > > > > > > > > > >
>> > > > > > > > > > >
>> > > > > > > > > >
>> > > > > > > > >
>> > > > > > > >
>> > > > > > >
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>> http://empathybox.com/kafka-javadoc/index.html?kafka/clients/producer/KafkaProducer.html
>> > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > Please take a look at this API and give me
>> any
>> > > > > thoughts
>> > > > > > > you
>> > > > > > > > > may
>> > > > > > > > > > > > have!
>> > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > It may also be reasonable to take a look at
>> the
>> > > > > > configs:
>> > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > >
>> > > > > > > > > > > > > >
>> > > > > > > > > > > > >
>> > > > > > > > > > > >
>> > > > > > > > > > >
>> > > > > > > > > >
>> > > > > > > > >
>> > > > > > > >
>> > > > > > >
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>> http://empathybox.com/kafka-javadoc/kafka/clients/producer/ProducerConfig.html
>> > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > The actual code is posted here:
>> > > > > > > > > > > > > > > >
>> > https://issues.apache.org/jira/browse/KAFKA-1227
>> > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > A few questions or comments to kick things
>> off:
>> > > > > > > > > > > > > > > > 1. We need to make a decision on whether
>> > > > > serialization
>> > > > > > of
>> > > > > > > > the
>> > > > > > > > > > > > user's
>> > > > > > > > > > > > > > key
>> > > > > > > > > > > > > > > > and value should be done by the user (with
>> our
>> > > api
>> > > > > just
>> > > > > > > > > taking
>> > > > > > > > > > > > > byte[])
>> > > > > > > > > > > > > > or
>> > > > > > > > > > > > > > > > if we should take an object and allow the
>> user
>> > to
>> > > > > > > > configure a
>> > > > > > > > > > > > > > Serializer
>> > > > > > > > > > > > > > > > class which we instantiate via reflection.
>> We
>> > > take
>> > > > > the
>> > > > > > > > later
>> > > > > > > > > > > > approach
>> > > > > > > > > > > > > > in
>> > > > > > > > > > > > > > > > the current producer, and I have carried
>> this
>> > > > through
>> > > > > > to
>> > > > > > > > this
>> > > > > > > > > > > > > > prototype.
>> > > > > > > > > > > > > > > > The tradeoff I see is this: taking byte[] is
>> > > > actually
>> > > > > > > > > simpler,
>> > > > > > > > > > > the
>> > > > > > > > > > > > > user
>> > > > > > > > > > > > > > > can
>> > > > > > > > > > > > > > > > directly do whatever serialization they
>> like.
>> > The
>> > > > > > > > > complication
>> > > > > > > > > > is
>> > > > > > > > > > > > > > > actually
>> > > > > > > > > > > > > > > > partitioning. Currently partitioning is done
>> > by a
>> > > > > > similar
>> > > > > > > > > > plug-in
>> > > > > > > > > > > > api
>> > > > > > > > > > > > > > > > (Partitioner) which the user can implement
>> and
>> > > > > > configure
>> > > > > > > to
>> > > > > > > > > > > > override
>> > > > > > > > > > > > > > how
>> > > > > > > > > > > > > > > > partitions are assigned. If we take byte[]
>> as
>> > > input
>> > > > > > then
>> > > > > > > we
>> > > > > > > > > > have
>> > > > > > > > > > > no
>> > > > > > > > > > > > > > > access
>> > > > > > > > > > > > > > > > to the original object and partitioning
>> MUST be
>> > > > done
>> > > > > on
>> > > > > > > the
>> > > > > > > > > > > byte[].
>> > > > > > > > > > > > > > This
>> > > > > > > > > > > > > > > is
>> > > > > > > > > > > > > > > > fine for hash partitioning. However for
>> various
>> > > > types
>> > > > > > of
>> > > > > > > > > > semantic
>> > > > > > > > > > > > > > > > partitioning (range partitioning, or
>> whatever)
>> > > you
>> > > > > > would
>> > > > > > > > want
>> > > > > > > > > > > > access
>> > > > > > > > > > > > > to
>> > > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > original object. In the current approach a
>> > > producer
>> > > > > who
>> > > > > > > > > wishes
>> > > > > > > > > > to
>> > > > > > > > > > > > > send
>> > > > > > > > > > > > > > > > byte[] they have serialized in their own
>> code
>> > can
>> > > > > > > configure
>> > > > > > > > > the
>> > > > > > > > > > > > > > > > BytesSerialization we supply which is just a
>> > "no
>> > > > op"
>> > > > > > > > > > > serialization.
>> > > > > > > > > > > > > > > > 2. We should obsess over naming and make
>> sure
>> > > each
>> > > > of
>> > > > > > the
>> > > > > > > > > class
>> > > > > > > > > > > > names
>> > > > > > > > > > > > > > are
>> > > > > > > > > > > > > > > > good.
>> > > > > > > > > > > > > > > > 3. Jun has already pointed out that we need
>> to
>> > > > > include
>> > > > > > > the
>> > > > > > > > > > topic
>> > > > > > > > > > > > and
>> > > > > > > > > > > > > > > > partition in the response, which is
>> absolutely
>> > > > > right. I
>> > > > > > > > > haven't
>> > > > > > > > > > > > done
>> > > > > > > > > > > > > > that
>> > > > > > > > > > > > > > > > yet but that definitely needs to be there.
>> > > > > > > > > > > > > > > > 4. Currently RecordSend.await will throw an
>> > > > exception
>> > > > > > if
>> > > > > > > > the
>> > > > > > > > > > > > request
>> > > > > > > > > > > > > > > > failed. The intention here is that
>> > > > > > > > > > producer.send(message).await()
>> > > > > > > > > > > > > > exactly
>> > > > > > > > > > > > > > > > simulates a synchronous call. Guozhang has
>> > noted
>> > > > that
>> > > > > > > this
>> > > > > > > > > is a
>> > > > > > > > > > > > > little
>> > > > > > > > > > > > > > > > annoying since the user must then catch
>> > > exceptions.
>> > > > > > > However
>> > > > > > > > > if
>> > > > > > > > > > we
>> > > > > > > > > > > > > > remove
>> > > > > > > > > > > > > > > > this then if the user doesn't check for
>> errors
>> > > they
>> > > > > > won't
>> > > > > > > > > know
>> > > > > > > > > > > one
>> > > > > > > > > > > > > has
>> > > > > > > > > > > > > > > > occurred, which I predict will be a common
>> > > mistake.
>> > > > > > > > > > > > > > > > 5. Perhaps there is more we could do to make
>> > the
>> > > > > async
>> > > > > > > > > > callbacks
>> > > > > > > > > > > > and
>> > > > > > > > > > > > > > > future
>> > > > > > > > > > > > > > > > we give back intuitive and easy to program
>> > > against?
>> > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > Some background info on implementation:
>> > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > At a high level the primary difference in
>> this
>> > > > > producer
>> > > > > > > is
>> > > > > > > > > that
>> > > > > > > > > > > it
>> > > > > > > > > > > > > > > removes
>> > > > > > > > > > > > > > > > the distinction between the "sync" and
>> "async"
>> > > > > > producer.
>> > > > > > > > > > > > Effectively
>> > > > > > > > > > > > > > all
>> > > > > > > > > > > > > > > > requests are sent asynchronously but always
>> > > return
>> > > > a
>> > > > > > > future
>> > > > > > > > > > > > response
>> > > > > > > > > > > > > > > object
>> > > > > > > > > > > > > > > > that gives the offset as well as any error
>> that
>> > > may
>> > > > > > have
>> > > > > > > > > > occurred
>> > > > > > > > > > > > > when
>> > > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > request is complete. The batching that is
>> done
>> > in
>> > > > the
>> > > > > > > async
>> > > > > > > > > > > > producer
>> > > > > > > > > > > > > > only
>> > > > > > > > > > > > > > > > today is done whenever possible now. This
>> means
>> > > > that
>> > > > > > the
>> > > > > > > > sync
>> > > > > > > > > > > > > producer,
>> > > > > > > > > > > > > > > > under load, can get performance as good as
>> the
>> > > > async
>> > > > > > > > producer
>> > > > > > > > > > > > > > > (preliminary
>> > > > > > > > > > > > > > > > results show the producer getting 1m
>> > > messages/sec).
>> > > > > > This
>> > > > > > > > > works
>> > > > > > > > > > > > > similar
>> > > > > > > > > > > > > > to
>> > > > > > > > > > > > > > > > group commit in databases but with respect
>> to
>> > the
>> > > > > > actual
>> > > > > > > > > > network
>> > > > > > > > > > > > > > > > transmission--any messages that arrive
>> while a
>> > > send
>> > > > > is
>> > > > > > in
>> > > > > > > > > > > progress
>> > > > > > > > > > > > > are
>> > > > > > > > > > > > > > > > batched together. It is also possible to
>> > > encourage
>> > > > > > > batching
>> > > > > > > > > > even
>> > > > > > > > > > > > > under
>> > > > > > > > > > > > > > > low
>> > > > > > > > > > > > > > > > load to save server resources by
>> introducing a
>> > > > delay
>> > > > > on
>> > > > > > > the
>> > > > > > > > > > send
>> > > > > > > > > > > to
>> > > > > > > > > > > > > > allow
>> > > > > > > > > > > > > > > > more messages to accumulate; this is done
>> using
>> > > the
>> > > > > > > > > > > linger.msconfig
>> > > > > > > > > > > > > > > (this
>> > > > > > > > > > > > > > > > is similar to Nagle's algorithm in TCP).
>> > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > This producer does all network communication
>> > > > > > > asynchronously
>> > > > > > > > > and
>> > > > > > > > > > > in
>> > > > > > > > > > > > > > > parallel
>> > > > > > > > > > > > > > > > to all servers so the performance penalty
>> for
>> > > > acks=-1
>> > > > > > and
>> > > > > > > > > > waiting
>> > > > > > > > > > > > on
>> > > > > > > > > > > > > > > > replication should be much reduced. I
>> haven't
>> > > done
>> > > > > much
>> > > > > > > > > > > > benchmarking
>> > > > > > > > > > > > > on
>> > > > > > > > > > > > > > > > this yet, though.
>> > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > The high level design is described a little
>> > here,
>> > > > > > though
>> > > > > > > > this
>> > > > > > > > > > is
>> > > > > > > > > > > > now
>> > > > > > > > > > > > > a
>> > > > > > > > > > > > > > > > little out of date:
>> > > > > > > > > > > > > > > >
>> > > > > > > > > >
>> > > > https://cwiki.apache.org/confluence/display/KAFKA/Client+Rewrite
>> > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > -Jay
>> > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > >
>> > > > > > > > > > > > > >
>> > > > > > > > > > > > >
>> > > > > > > > > > > >
>> > > > > > > > > > >
>> > > > > > > > > >
>> > > > > > > > >
>> > > > > > > >
>> > > > > > >
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>>
>
>

Re: New Producer Public API

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

Error handling in RecordSend works as in Future you will get the exception
if there is one from any of the accessor methods or await().

The purpose of hasError was that you can write things slightly more simply
(which some people expressed preference for):
  if(send.hasError())
    // do something
  long offset = send.offset();

Instead of the more the slightly longer:
try {
   long offset = send.offset();
} catch (KafkaException e) {
   // do something
}


On Wed, Jan 29, 2014 at 10:01 AM, Neha Narkhede <ne...@gmail.com>wrote:

> Regarding the use of Futures -
>
> Agree that there are some downsides to using Futures but both approaches
> have some tradeoffs.
>
> - Standardization and usability
> Future is a widely used and understood Java API and given that the
> functionality that RecordSend hopes to provide is essentially that of
> Future, I think it makes sense to expose a widely understood public API for
> our clients. RecordSend, on the other hand, seems to provide some APIs that
> are very similar to that of Future, in addition to exposing a bunch of APIs
> that belong to ProduceRequestResult. As a user, I would've really preferred
> to deal with ProduceRequestResult directly -
> Future<ProduceRequestResult> send(...)
>
> - Error handling
> RecordSend's error handling is quite unintuitive where the user has to
> remember to invoke hasError and error, instead of just throwing the
> exception. Now there are
> some downsides regarding error handling with the Future as well, where the
> user has to catch InterruptedException when we would never run into it.
> However, it seems like a price worth paying for supporting a standard API
> and error handling
>
> - Unused APIs
> This is a downside of using Future, where the cancel() operation would
> always return false and mean nothing. But we can mention that caveat in our
> Java docs.
>
> To summarize, I would prefer to expose a well understood and widely adopted
> Java API and put up with the overhead of catching one unnecessary checked
> exception, rather than wrap the useful ProduceRequestResult in a custom
> async object (RecordSend) and explain that to our many users.
>
> Thanks,
> Neha
>
>
>
>
> On Tue, Jan 28, 2014 at 8:10 PM, Jay Kreps <ja...@gmail.com> wrote:
>
> > Hey Neha,
> >
> > Can you elaborate on why you prefer using Java's Future? The downside in
> my
> > mind is the use of the checked InterruptedException and
> ExecutionException.
> > ExecutionException is arguable, but forcing you to catch
> > InterruptedException, often in code that can't be interrupted, seems
> > perverse. It also leaves us with the cancel() method which I don't think
> we
> > really can implement.
> >
> > Option 1A, to recap/elaborate, was the following. There is no Serializer
> or
> > Partitioner api. We take a byte[] key and value and an optional integer
> > partition. If you specify the integer partition it will be used. If you
> do
> > not specify a key or a partition the partition will be chosen in a round
> > robin fashion. If you specify a key but no partition we will chose a
> > partition based on a hash of the key. In order to let the user find the
> > partition we will need to given them access to the Cluster instance
> > directly from the producer.
> >
> > -Jay
> >
> >
> > On Tue, Jan 28, 2014 at 6:25 PM, Neha Narkhede <neha.narkhede@gmail.com
> > >wrote:
> >
> > > Here are more thoughts on the public APIs -
> > >
> > > - I suggest we use java's Future instead of custom Future especially
> > since
> > > it is part of the public API
> > >
> > > - Serialization: I like the simplicity of the producer APIs with the
> > > absence of serialization where we just deal with byte arrays for keys
> and
> > > values. What I don't like about this is the performance overhead on the
> > > Partitioner for any kind of custom partitioning based on the
> > partitionKey.
> > > Since the only purpose of partitionKey is to do custom partitioning,
> why
> > > can't we take it in directly as an integer and let the user figure out
> > the
> > > mapping from partition_key -> partition_id using the getCluster() API?
> > If I
> > > understand correctly, this is similar to what you suggested as part of
> > > option 1A. I like this approach since it maintains the simplicity of
> APIs
> > > by allowing us to deal with bytes and does not compromise performance
> in
> > > the custom partitioning case.
> > >
> > > Thanks,
> > > Neha
> > >
> > >
> > >
> > > On Tue, Jan 28, 2014 at 5:42 PM, Jay Kreps <ja...@gmail.com>
> wrote:
> > >
> > > > Hey Tom,
> > > >
> > > > That sounds cool. How did you end up handling parallel I/O if you
> wrap
> > > the
> > > > individual connections? Don't you need some selector that selects
> over
> > > all
> > > > the connections?
> > > >
> > > > -Jay
> > > >
> > > >
> > > > On Tue, Jan 28, 2014 at 2:31 PM, Tom Brown <to...@gmail.com>
> > wrote:
> > > >
> > > > > I implemented a 0.7 client in pure java, and its API very closely
> > > > resembled
> > > > > this. (When multiple people independently engineer the same
> solution,
> > > > it's
> > > > > probably good... right?). However, there were a few architectural
> > > > > differences with my client:
> > > > >
> > > > > 1. The basic client itself was just an asynchronous layer around
> the
> > > > > different server functions. In and of itself it had no knowledge of
> > > > > partitions, only servers (and maintained TCP connections to them).
> > > > >
> > > > > 2. The main producer was an additional layer that provided a
> > high-level
> > > > > interface that could batch individual messages based on partition.
> > > > >
> > > > > 3. Knowledge of partitioning was done via an interface so that
> > > different
> > > > > strategies could be used.
> > > > >
> > > > > 4. Partitioning was done by the user, with knowledge of the
> available
> > > > > partitions provided by #3.
> > > > >
> > > > > 5. Serialization was done by the user to simplify the API.
> > > > >
> > > > > 6. Futures were used to make asynchronous emulate synchronous
> calls.
> > > > >
> > > > >
> > > > > The main benefit of this approach is flexibility. For example,
> since
> > > the
> > > > > base client was just a managed connection (and not inherently a
> > > > producer),
> > > > > it was easy to composite a produce request and an offsets request
> > > > together
> > > > > into a confirmed produce request (officially not available in 0.7).
> > > > >
> > > > > Decoupling the basic client from partition management allowed the
> me
> > to
> > > > > implement zk discovery as a separate project so that the main
> project
> > > had
> > > > > no complex dependencies. The same was true of decoupling
> > serialization.
> > > > > It's trivial to build an optional layer that adds those features
> in,
> > > > while
> > > > > allowing access to the base APIs for those that need it.
> > > > >
> > > > > Using standard Future objects was also beneficial, since I could
> > > combine
> > > > > them with existing tools (such as guava).
> > > > >
> > > > > It may be too late to be of use, but I have been working with my
> > > > company's
> > > > > legal department to release the implementation I described above.
> If
> > > > you're
> > > > > interested in it, let me know.
> > > > >
> > > > >
> > > > > To sum up my thoughts regarding the new API, I think it's a great
> > > start.
> > > > I
> > > > > would like to see a more layered approach so I can use the parts I
> > > want,
> > > > > and adapt the other parts as needed. I would also like to see
> > standard
> > > > > interfaces (especially Future) used where they makes sense.
> > > > >
> > > > > --Tom
> > > > >
> > > > >
> > > > >
> > > > >
> > > > >
> > > > > On Tue, Jan 28, 2014 at 1:33 PM, Roger Hoover <
> > roger.hoover@gmail.com
> > > > > >wrote:
> > > > >
> > > > > > +1 ListenableFuture: If this works similar to Deferreds in
> Twisted
> > > > Python
> > > > > > or Promised IO in Javascript, I think this is a great pattern for
> > > > > > decoupling your callback logic from the place where the Future is
> > > > > > generated.  You can register as many callbacks as you like, each
> in
> > > the
> > > > > > appropriate layer of the code and have each observer get notified
> > > when
> > > > > the
> > > > > > promised i/o is complete without any of them knowing about each
> > > other.
> > > > > >
> > > > > >
> > > > > > On Tue, Jan 28, 2014 at 11:32 AM, Jay Kreps <jay.kreps@gmail.com
> >
> > > > wrote:
> > > > > >
> > > > > > > Hey Ross,
> > > > > > >
> > > > > > > - ListenableFuture: Interesting. That would be an alternative
> to
> > > the
> > > > > > direct
> > > > > > > callback support we provide. There could be pros to this, let
> me
> > > > think
> > > > > > > about it.
> > > > > > > - We could provide layering, but I feel that the serialization
> is
> > > > such
> > > > > a
> > > > > > > small thing we should just make a decision and chose one, it
> > > doesn't
> > > > > seem
> > > > > > > to me to justify a whole public facing layer.
> > > > > > > - Yes, this is fairly esoteric, essentially I think it is
> fairly
> > > > > similar
> > > > > > to
> > > > > > > databases like DynamoDB that allow you to specify two partition
> > > keys
> > > > (I
> > > > > > > think DynamoDB does this...). The reasoning is that in fact
> there
> > > are
> > > > > > > several things you can use the key field for: (1) to compute
> the
> > > > > > partition
> > > > > > > to store the data in, (2) as a unique identifier to deduplicate
> > > that
> > > > > > > partition's records within a log. These two things are almost
> > > always
> > > > > the
> > > > > > > same, but occationally may differ when you want to group data
> in
> > a
> > > > more
> > > > > > > sophisticated way then just a hash of the primary key but still
> > > > retain
> > > > > > the
> > > > > > > proper primary key for delivery to the consumer and log
> > compaction.
> > > > > > >
> > > > > > > -Jay
> > > > > > >
> > > > > > >
> > > > > > > On Tue, Jan 28, 2014 at 3:24 AM, Ross Black <
> > > ross.w.black@gmail.com>
> > > > > > > wrote:
> > > > > > >
> > > > > > > > Hi Jay,
> > > > > > > >
> > > > > > > > - Just to add some more info/confusion about possibly using
> > > Future
> > > > > ...
> > > > > > > >   If Kafka uses a JDK future, it plays nicely with other
> > > frameworks
> > > > > as
> > > > > > > > well.
> > > > > > > >   Google Guava has a ListenableFuture that allows callback
> > > handling
> > > > > to
> > > > > > be
> > > > > > > > added via the returned future, and allows the callbacks to be
> > > > passed
> > > > > > off
> > > > > > > to
> > > > > > > > a specified executor.
> > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> http://docs.guava-libraries.googlecode.com/git-history/release/javadoc/com/google/common/util/concurrent/ListenableFuture.html
> > > > > > > >   The JDK future can easily be converted to a listenable
> > future.
> > > > > > > >
> > > > > > > > - On the question of byte[] vs Object, could this be solved
> by
> > > > > layering
> > > > > > > the
> > > > > > > > API?  eg. a raw producer (use byte[] and specify the
> partition
> > > > > number)
> > > > > > > and
> > > > > > > > a normal producer (use generic object and specify a
> > Partitioner)?
> > > > > > > >
> > > > > > > > - I am confused by the keys in ProducerRecord and
> Partitioner.
> > > >  What
> > > > > is
> > > > > > > the
> > > > > > > > usage for both a key and a partition key? (I am not yet using
> > > 0.8)
> > > > > > > >
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > > Ross
> > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > > On 28 January 2014 05:00, Xavier Stevens <xa...@gaikai.com>
> > > > wrote:
> > > > > > > >
> > > > > > > > > AutoCloseable would be nice for us as most of our code is
> > using
> > > > > Java
> > > > > > 7
> > > > > > > at
> > > > > > > > > this point.
> > > > > > > > >
> > > > > > > > > I like Dropwizard's configuration mapping to POJOs via
> > Jackson,
> > > > but
> > > > > > if
> > > > > > > > you
> > > > > > > > > wanted to stick with property maps I don't care enough to
> > > object.
> > > > > > > > >
> > > > > > > > > If the producer only dealt with bytes, is there a way we
> > could
> > > > > still
> > > > > > > due
> > > > > > > > > partition plugins without specifying the number
> explicitly? I
> > > > would
> > > > > > > > prefer
> > > > > > > > > to be able to pass in field(s) that would be used by the
> > > > > partitioner.
> > > > > > > > > Obviously if this wasn't possible you could always
> > deserialize
> > > > the
> > > > > > > object
> > > > > > > > > in the partitioner and grab the fields you want, but that
> > seems
> > > > > > really
> > > > > > > > > expensive to do on every message.
> > > > > > > > >
> > > > > > > > > It would also be nice to have a Java API Encoder
> constructor
> > > > taking
> > > > > > in
> > > > > > > > > VerifiableProperties. Scala understands how to handle
> "props:
> > > > > > > > > VerifiableProperties = null", but Java doesn't. So you
> don't
> > > run
> > > > > into
> > > > > > > > this
> > > > > > > > > problem until runtime.
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > -Xavier
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > On Mon, Jan 27, 2014 at 9:37 AM, Clark Breyman <
> > > > clark@breyman.com>
> > > > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > Jay -
> > > > > > > > > >
> > > > > > > > > > Config - your explanation makes sense. I'm just so
> > accustomed
> > > > to
> > > > > > > having
> > > > > > > > > > Jackson automatically map my configuration objects to
> POJOs
> > > > that
> > > > > > I've
> > > > > > > > > > stopped using property files. They are lingua franca. The
> > > only
> > > > > > > thought
> > > > > > > > > > might be to separate the config interface from the
> > > > implementation
> > > > > > to
> > > > > > > > > allow
> > > > > > > > > > for alternatives, but that might undermine your point of
> > "do
> > > it
> > > > > > this
> > > > > > > > way
> > > > > > > > > so
> > > > > > > > > > that everyone can find it where they expect it".
> > > > > > > > > >
> > > > > > > > > > Serialization: Of the options, I like 1A the best, though
> > > > > possibly
> > > > > > > with
> > > > > > > > > > either an option to specify a partition key rather than
> ID
> > > or a
> > > > > > > helper
> > > > > > > > to
> > > > > > > > > > translate an arbitrary byte[] or long into a partition
> > > number.
> > > > > > > > > >
> > > > > > > > > > Thanks
> > > > > > > > > > Clark
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > On Sun, Jan 26, 2014 at 9:13 PM, Jay Kreps <
> > > > jay.kreps@gmail.com>
> > > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > Thanks for the detailed thoughts. Let me elaborate on
> the
> > > > > config
> > > > > > > > thing.
> > > > > > > > > > >
> > > > > > > > > > > I agree that at first glance key-value strings don't
> seem
> > > > like
> > > > > a
> > > > > > > very
> > > > > > > > > > good
> > > > > > > > > > > configuration api for a client. Surely a well-typed
> > config
> > > > > class
> > > > > > > > would
> > > > > > > > > be
> > > > > > > > > > > better! I actually disagree and let me see if I can
> > > convince
> > > > > you.
> > > > > > > > > > >
> > > > > > > > > > > My reasoning has nothing to do with the api and
> > everything
> > > to
> > > > > do
> > > > > > > with
> > > > > > > > > > > operations.
> > > > > > > > > > >
> > > > > > > > > > > Clients are embedded in applications which are
> themselves
> > > > > > > configured.
> > > > > > > > > In
> > > > > > > > > > > any place that takes operations seriously the
> > configuration
> > > > for
> > > > > > > these
> > > > > > > > > > > applications will be version controlled and maintained
> > > > through
> > > > > > some
> > > > > > > > > kind
> > > > > > > > > > of
> > > > > > > > > > > config management system. If we give a config class
> with
> > > > > getters
> > > > > > > and
> > > > > > > > > > > setters the application has to expose those properties
> to
> > > its
> > > > > > > > > > > configuration. What invariably happens is that the
> > > > application
> > > > > > > > exposes
> > > > > > > > > > only
> > > > > > > > > > > a choice few properties that they thought they would
> > > change.
> > > > > > > > > Furthermore
> > > > > > > > > > > the application will make up a name for these configs
> > that
> > > > > seems
> > > > > > > > > > intuitive
> > > > > > > > > > > at the time in the 2 seconds the engineer spends
> thinking
> > > > about
> > > > > > it.
> > > > > > > > > > >
> > > > > > > > > > > Now consider the result of this in the large. You end
> up
> > > with
> > > > > > > dozens
> > > > > > > > or
> > > > > > > > > > > hundreds of applications that have the client embedded.
> > > Each
> > > > > > > exposes
> > > > > > > > a
> > > > > > > > > > > different, inadequate subset of the possible configs,
> > each
> > > > with
> > > > > > > > > different
> > > > > > > > > > > names. It is a nightmare.
> > > > > > > > > > >
> > > > > > > > > > > If you use a string-string map the config system can
> > > directly
> > > > > > get a
> > > > > > > > > > bundle
> > > > > > > > > > > of config key-value pairs and put them into the client.
> > > This
> > > > > > means
> > > > > > > > that
> > > > > > > > > > all
> > > > > > > > > > > configuration is automatically available with the name
> > > > > documented
> > > > > > > on
> > > > > > > > > the
> > > > > > > > > > > website in every application that does this. If you
> > upgrade
> > > > to
> > > > > a
> > > > > > > new
> > > > > > > > > > kafka
> > > > > > > > > > > version with more configs those will be exposed too. If
> > you
> > > > > > realize
> > > > > > > > > that
> > > > > > > > > > > you need to change a default you can just go through
> your
> > > > > configs
> > > > > > > and
> > > > > > > > > > > change it everywhere as it will have the same name
> > > > everywhere.
> > > > > > > > > > >
> > > > > > > > > > > -Jay
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > On Sun, Jan 26, 2014 at 4:47 PM, Clark Breyman <
> > > > > > clark@breyman.com>
> > > > > > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > Thanks Jay. I'll see if I can put together a more
> > > complete
> > > > > > > > response,
> > > > > > > > > > > > perhaps as separate threads so that topics don't get
> > > > > entangled.
> > > > > > > In
> > > > > > > > > the
> > > > > > > > > > > mean
> > > > > > > > > > > > time, here's a couple responses:
> > > > > > > > > > > >
> > > > > > > > > > > > Serialization: you've broken out a sub-thread so i'll
> > > reply
> > > > > > > there.
> > > > > > > > My
> > > > > > > > > > > bias
> > > > > > > > > > > > is that I like generics (except for type-erasure) and
> > in
> > > > > > > particular
> > > > > > > > > > they
> > > > > > > > > > > > make it easy to compose serializers for compound
> > payloads
> > > > > (e.g.
> > > > > > > > when
> > > > > > > > > a
> > > > > > > > > > > > common header wraps a payload of parameterized type).
> > > I'll
> > > > > > > respond
> > > > > > > > to
> > > > > > > > > > > your
> > > > > > > > > > > > 4-options message with an example.
> > > > > > > > > > > >
> > > > > > > > > > > > Build: I've seen a lot of "maven-compatible" build
> > > systems
> > > > > > > produce
> > > > > > > > > > > > "artifacts" that aren't really artifacts - no
> embedded
> > > POM
> > > > > or,
> > > > > > > > worst,
> > > > > > > > > > > > malformed POM. I know the sbt-generated artifacts
> were
> > > this
> > > > > > way -
> > > > > > > > > onus
> > > > > > > > > > is
> > > > > > > > > > > > on me to see what gradle is spitting out and what a
> > maven
> > > > > build
> > > > > > > > might
> > > > > > > > > > > look
> > > > > > > > > > > > like. Maven may be old and boring, but it gets out of
> > the
> > > > way
> > > > > > and
> > > > > > > > > > > > integrates really seamlessly with a lot of IDEs. When
> > > some
> > > > > > scala
> > > > > > > > > > > projects I
> > > > > > > > > > > > was working on in the fall of 2011 switched from sbt
> to
> > > > > maven,
> > > > > > > > build
> > > > > > > > > > > became
> > > > > > > > > > > > a non-issue.
> > > > > > > > > > > >
> > > > > > > > > > > > Config: Not a big deal  and no, I don't think a
> > > dropwizard
> > > > > > > > dependency
> > > > > > > > > > is
> > > > > > > > > > > > appropriate. I do like using simple entity beans
> > (POJO's
> > > > not
> > > > > > > j2EE)
> > > > > > > > > for
> > > > > > > > > > > > configuration, especially if they can be marshalled
> > > without
> > > > > > > > > annotation
> > > > > > > > > > by
> > > > > > > > > > > > Jackson. I only mentioned the dropwizard-extras
> >  because
> > > it
> > > > > has
> > > > > > > > some
> > > > > > > > > > > entity
> > > > > > > > > > > > bean versions of the ZK and Kafka configs.
> > > > > > > > > > > >
> > > > > > > > > > > > Domain-packaging: Also not a big deal - it's what's
> > > > expected
> > > > > > and
> > > > > > > > it's
> > > > > > > > > > > > pretty free in most IDE's. The advantages I see is
> that
> > > it
> > > > is
> > > > > > > clear
> > > > > > > > > > > whether
> > > > > > > > > > > > something is from the Apache Kafka project and
> whether
> > > > > > something
> > > > > > > is
> > > > > > > > > > from
> > > > > > > > > > > > another org and related to Kafka. That said, nothing
> > > really
> > > > > > > > enforces
> > > > > > > > > > it.
> > > > > > > > > > > >
> > > > > > > > > > > > Futures: I'll see if I can create some examples to
> > > > > demonstrate
> > > > > > > > Future
> > > > > > > > > > > > making interop easier.
> > > > > > > > > > > >
> > > > > > > > > > > > Regards,
> > > > > > > > > > > > C
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > On Fri, Jan 24, 2014 at 4:36 PM, Jay Kreps <
> > > > > > jay.kreps@gmail.com>
> > > > > > > > > > wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > > Hey Clark,
> > > > > > > > > > > > >
> > > > > > > > > > > > > - Serialization: Yes I agree with these though I
> > don't
> > > > > > consider
> > > > > > > > the
> > > > > > > > > > > loss
> > > > > > > > > > > > of
> > > > > > > > > > > > > generics a big issue. I'll try to summarize what I
> > > would
> > > > > > > consider
> > > > > > > > > the
> > > > > > > > > > > > best
> > > > > > > > > > > > > alternative api with raw byte[].
> > > > > > > > > > > > >
> > > > > > > > > > > > > - Maven: We had this debate a few months back and
> the
> > > > > > consensus
> > > > > > > > was
> > > > > > > > > > > > gradle.
> > > > > > > > > > > > > Is there a specific issue with the poms gradle
> > makes? I
> > > > am
> > > > > > > > > extremely
> > > > > > > > > > > > loath
> > > > > > > > > > > > > to revisit the issue as build issues are a
> recurring
> > > > thing
> > > > > > and
> > > > > > > no
> > > > > > > > > one
> > > > > > > > > > > > ever
> > > > > > > > > > > > > agrees and ultimately our build needs are very
> > simple.
> > > > > > > > > > > > >
> > > > > > > > > > > > > - Config: I'm not sure if I follow the point. Are
> you
> > > > > saying
> > > > > > we
> > > > > > > > > > should
> > > > > > > > > > > > use
> > > > > > > > > > > > > something in dropwizard for config? One principle
> > here
> > > is
> > > > > to
> > > > > > > try
> > > > > > > > to
> > > > > > > > > > > > remove
> > > > > > > > > > > > > as many client dependencies as possible as we
> > > inevitably
> > > > > run
> > > > > > > into
> > > > > > > > > > > > terrible
> > > > > > > > > > > > > compatibility issues with users who use the same
> > > library
> > > > or
> > > > > > its
> > > > > > > > > > > > > dependencies at different versions. Or are you
> > talking
> > > > > about
> > > > > > > > > > > maintaining
> > > > > > > > > > > > > compatibility with existing config parameters? I
> > think
> > > as
> > > > > > much
> > > > > > > > as a
> > > > > > > > > > > > config
> > > > > > > > > > > > > in the existing client makes sense it should have
> the
> > > > same
> > > > > > name
> > > > > > > > (I
> > > > > > > > > > was
> > > > > > > > > > > a
> > > > > > > > > > > > > bit sloppy about that so I'll fix any errors
> there).
> > > > There
> > > > > > are
> > > > > > > a
> > > > > > > > > few
> > > > > > > > > > > new
> > > > > > > > > > > > > things and we should give those reasonable
> defaults.
> > I
> > > > > think
> > > > > > > > config
> > > > > > > > > > is
> > > > > > > > > > > > > important so I'll start a thread on the config
> > package
> > > in
> > > > > > > there.
> > > > > > > > > > > > >
> > > > > > > > > > > > > - org.apache.kafka: We could do this. I always
> > > considered
> > > > > it
> > > > > > > kind
> > > > > > > > > of
> > > > > > > > > > an
> > > > > > > > > > > > odd
> > > > > > > > > > > > > thing Java programmers do that has no real
> motivation
> > > > (but
> > > > > I
> > > > > > > > could
> > > > > > > > > be
> > > > > > > > > > > > > re-educated!). I don't think it ends up reducing
> > naming
> > > > > > > conflicts
> > > > > > > > > in
> > > > > > > > > > > > > practice and it adds a lot of noise and nested
> > > > directories.
> > > > > > Is
> > > > > > > > > there
> > > > > > > > > > a
> > > > > > > > > > > > > reason you prefer this or just to be more standard?
> > > > > > > > > > > > >
> > > > > > > > > > > > > - Future: Basically I didn't see any particular
> > > > advantage.
> > > > > > The
> > > > > > > > > > cancel()
> > > > > > > > > > > > > method doesn't really make sense so probably
> wouldn't
> > > > work.
> > > > > > > > > Likewise
> > > > > > > > > > I
> > > > > > > > > > > > > dislike the checked exceptions it requires.
> > Basically I
> > > > > just
> > > > > > > > wrote
> > > > > > > > > > out
> > > > > > > > > > > > some
> > > > > > > > > > > > > code examples and it seemed cleaner with a special
> > > > purpose
> > > > > > > > object.
> > > > > > > > > I
> > > > > > > > > > > > wasn't
> > > > > > > > > > > > > actually aware of plans for improved futures in
> java
> > 8
> > > or
> > > > > the
> > > > > > > > other
> > > > > > > > > > > > > integrations. Maybe you could elaborate on this a
> bit
> > > and
> > > > > > show
> > > > > > > > how
> > > > > > > > > it
> > > > > > > > > > > > would
> > > > > > > > > > > > > be used? Sounds promising, I just don't know a lot
> > > about
> > > > > it.
> > > > > > > > > > > > >
> > > > > > > > > > > > > -Jay
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Fri, Jan 24, 2014 at 3:30 PM, Clark Breyman <
> > > > > > > > clark@breyman.com>
> > > > > > > > > > > > wrote:
> > > > > > > > > > > > >
> > > > > > > > > > > > > > Jay - Thanks for the call for comments. Here's
> some
> > > > > initial
> > > > > > > > > input:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > - Make message serialization a client
> > responsibility
> > > > > > (making
> > > > > > > > all
> > > > > > > > > > > > messages
> > > > > > > > > > > > > > byte[]). Reflection-based loading makes it harder
> > to
> > > > use
> > > > > > > > generic
> > > > > > > > > > > codecs
> > > > > > > > > > > > > > (e.g.  Envelope<PREFIX, DATA, SUFFIX>) or build
> up
> > > > codec
> > > > > > > > > > > > > programmatically.
> > > > > > > > > > > > > > Non-default partitioning should require an
> explicit
> > > > > > partition
> > > > > > > > > key.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > - I really like the fact that it will be native
> > Java.
> > > > > > Please
> > > > > > > > > > consider
> > > > > > > > > > > > > using
> > > > > > > > > > > > > > native maven and not sbt, gradle, ivy, etc as
> they
> > > > don't
> > > > > > > > reliably
> > > > > > > > > > > play
> > > > > > > > > > > > > nice
> > > > > > > > > > > > > > in the maven ecosystem. A jar without a
> well-formed
> > > pom
> > > > > > > doesn't
> > > > > > > > > > feel
> > > > > > > > > > > > > like a
> > > > > > > > > > > > > > real artifact. The pom's generated by sbt et al.
> > are
> > > > not
> > > > > > well
> > > > > > > > > > formed.
> > > > > > > > > > > > > Using
> > > > > > > > > > > > > > maven will make builds and IDE integration much
> > > > smoother.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > - Look at Nick Telford's dropwizard-extras
> package
> > in
> > > > > which
> > > > > > > he
> > > > > > > > > > > defines
> > > > > > > > > > > > > some
> > > > > > > > > > > > > > Jackson-compatible POJO's for loading
> > configuration.
> > > > > Seems
> > > > > > > like
> > > > > > > > > > your
> > > > > > > > > > > > > client
> > > > > > > > > > > > > > migration is similar. The config objects should
> > have
> > > > > > > > constructors
> > > > > > > > > > or
> > > > > > > > > > > > > > factories that accept Map<String, String> and
> > > > Properties
> > > > > > for
> > > > > > > > ease
> > > > > > > > > > of
> > > > > > > > > > > > > > migration.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > - Would you consider using the org.apache.kafka
> > > package
> > > > > for
> > > > > > > the
> > > > > > > > > new
> > > > > > > > > > > API
> > > > > > > > > > > > > > (quibble)
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > - Why create your own futures rather than use
> > > > > > > > > > > > > > java.util.concurrent.Future<Long> or similar?
> > > Standard
> > > > > > > futures
> > > > > > > > > will
> > > > > > > > > > > > play
> > > > > > > > > > > > > > nice with other reactive libs and things like
> J8's
> > > > > > > > > > ComposableFuture.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Thanks again,
> > > > > > > > > > > > > > C
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On Fri, Jan 24, 2014 at 2:46 PM, Roger Hoover <
> > > > > > > > > > > roger.hoover@gmail.com
> > > > > > > > > > > > > > >wrote:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > > A couple comments:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 1) Why does the config use a broker list
> instead
> > of
> > > > > > > > discovering
> > > > > > > > > > the
> > > > > > > > > > > > > > brokers
> > > > > > > > > > > > > > > in ZooKeeper?  It doesn't match the
> > > HighLevelConsumer
> > > > > > API.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 2) It looks like broker connections are created
> > on
> > > > > > demand.
> > > > > > > >  I'm
> > > > > > > > > > > > > wondering
> > > > > > > > > > > > > > > if sometimes you might want to flush out config
> > or
> > > > > > network
> > > > > > > > > > > > connectivity
> > > > > > > > > > > > > > > issues before pushing the first message
> through.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Should there also be a KafkaProducer.connect()
> or
> > > > > .open()
> > > > > > > > > method
> > > > > > > > > > or
> > > > > > > > > > > > > > > connectAll()?  I guess it would try to connect
> to
> > > all
> > > > > > > brokers
> > > > > > > > > in
> > > > > > > > > > > the
> > > > > > > > > > > > > > > BROKER_LIST_CONFIG
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > HTH,
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Roger
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > On Fri, Jan 24, 2014 at 11:54 AM, Jay Kreps <
> > > > > > > > > jay.kreps@gmail.com
> > > > > > > > > > >
> > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > As mentioned in a previous email we are
> working
> > > on
> > > > a
> > > > > > > > > > > > > re-implementation
> > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > the producer. I would like to use this email
> > > thread
> > > > > to
> > > > > > > > > discuss
> > > > > > > > > > > the
> > > > > > > > > > > > > > > details
> > > > > > > > > > > > > > > > of the public API and the configuration. I
> > would
> > > > love
> > > > > > for
> > > > > > > > us
> > > > > > > > > to
> > > > > > > > > > > be
> > > > > > > > > > > > > > > > incredibly picky about this public api now so
> > it
> > > is
> > > > > as
> > > > > > > good
> > > > > > > > > as
> > > > > > > > > > > > > possible
> > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > we don't need to break it in the future.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > The best way to get a feel for the API is
> > > actually
> > > > to
> > > > > > > take
> > > > > > > > a
> > > > > > > > > > look
> > > > > > > > > > > > at
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > javadoc, my hope is to get the api docs good
> > > enough
> > > > > so
> > > > > > > that
> > > > > > > > > it
> > > > > > > > > > is
> > > > > > > > > > > > > > > > self-explanatory:
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> http://empathybox.com/kafka-javadoc/index.html?kafka/clients/producer/KafkaProducer.html
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Please take a look at this API and give me
> any
> > > > > thoughts
> > > > > > > you
> > > > > > > > > may
> > > > > > > > > > > > have!
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > It may also be reasonable to take a look at
> the
> > > > > > configs:
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> http://empathybox.com/kafka-javadoc/kafka/clients/producer/ProducerConfig.html
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > The actual code is posted here:
> > > > > > > > > > > > > > > >
> > https://issues.apache.org/jira/browse/KAFKA-1227
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > A few questions or comments to kick things
> off:
> > > > > > > > > > > > > > > > 1. We need to make a decision on whether
> > > > > serialization
> > > > > > of
> > > > > > > > the
> > > > > > > > > > > > user's
> > > > > > > > > > > > > > key
> > > > > > > > > > > > > > > > and value should be done by the user (with
> our
> > > api
> > > > > just
> > > > > > > > > taking
> > > > > > > > > > > > > byte[])
> > > > > > > > > > > > > > or
> > > > > > > > > > > > > > > > if we should take an object and allow the
> user
> > to
> > > > > > > > configure a
> > > > > > > > > > > > > > Serializer
> > > > > > > > > > > > > > > > class which we instantiate via reflection. We
> > > take
> > > > > the
> > > > > > > > later
> > > > > > > > > > > > approach
> > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > the current producer, and I have carried this
> > > > through
> > > > > > to
> > > > > > > > this
> > > > > > > > > > > > > > prototype.
> > > > > > > > > > > > > > > > The tradeoff I see is this: taking byte[] is
> > > > actually
> > > > > > > > > simpler,
> > > > > > > > > > > the
> > > > > > > > > > > > > user
> > > > > > > > > > > > > > > can
> > > > > > > > > > > > > > > > directly do whatever serialization they like.
> > The
> > > > > > > > > complication
> > > > > > > > > > is
> > > > > > > > > > > > > > > actually
> > > > > > > > > > > > > > > > partitioning. Currently partitioning is done
> > by a
> > > > > > similar
> > > > > > > > > > plug-in
> > > > > > > > > > > > api
> > > > > > > > > > > > > > > > (Partitioner) which the user can implement
> and
> > > > > > configure
> > > > > > > to
> > > > > > > > > > > > override
> > > > > > > > > > > > > > how
> > > > > > > > > > > > > > > > partitions are assigned. If we take byte[] as
> > > input
> > > > > > then
> > > > > > > we
> > > > > > > > > > have
> > > > > > > > > > > no
> > > > > > > > > > > > > > > access
> > > > > > > > > > > > > > > > to the original object and partitioning MUST
> be
> > > > done
> > > > > on
> > > > > > > the
> > > > > > > > > > > byte[].
> > > > > > > > > > > > > > This
> > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > fine for hash partitioning. However for
> various
> > > > types
> > > > > > of
> > > > > > > > > > semantic
> > > > > > > > > > > > > > > > partitioning (range partitioning, or
> whatever)
> > > you
> > > > > > would
> > > > > > > > want
> > > > > > > > > > > > access
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > original object. In the current approach a
> > > producer
> > > > > who
> > > > > > > > > wishes
> > > > > > > > > > to
> > > > > > > > > > > > > send
> > > > > > > > > > > > > > > > byte[] they have serialized in their own code
> > can
> > > > > > > configure
> > > > > > > > > the
> > > > > > > > > > > > > > > > BytesSerialization we supply which is just a
> > "no
> > > > op"
> > > > > > > > > > > serialization.
> > > > > > > > > > > > > > > > 2. We should obsess over naming and make sure
> > > each
> > > > of
> > > > > > the
> > > > > > > > > class
> > > > > > > > > > > > names
> > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > good.
> > > > > > > > > > > > > > > > 3. Jun has already pointed out that we need
> to
> > > > > include
> > > > > > > the
> > > > > > > > > > topic
> > > > > > > > > > > > and
> > > > > > > > > > > > > > > > partition in the response, which is
> absolutely
> > > > > right. I
> > > > > > > > > haven't
> > > > > > > > > > > > done
> > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > yet but that definitely needs to be there.
> > > > > > > > > > > > > > > > 4. Currently RecordSend.await will throw an
> > > > exception
> > > > > > if
> > > > > > > > the
> > > > > > > > > > > > request
> > > > > > > > > > > > > > > > failed. The intention here is that
> > > > > > > > > > producer.send(message).await()
> > > > > > > > > > > > > > exactly
> > > > > > > > > > > > > > > > simulates a synchronous call. Guozhang has
> > noted
> > > > that
> > > > > > > this
> > > > > > > > > is a
> > > > > > > > > > > > > little
> > > > > > > > > > > > > > > > annoying since the user must then catch
> > > exceptions.
> > > > > > > However
> > > > > > > > > if
> > > > > > > > > > we
> > > > > > > > > > > > > > remove
> > > > > > > > > > > > > > > > this then if the user doesn't check for
> errors
> > > they
> > > > > > won't
> > > > > > > > > know
> > > > > > > > > > > one
> > > > > > > > > > > > > has
> > > > > > > > > > > > > > > > occurred, which I predict will be a common
> > > mistake.
> > > > > > > > > > > > > > > > 5. Perhaps there is more we could do to make
> > the
> > > > > async
> > > > > > > > > > callbacks
> > > > > > > > > > > > and
> > > > > > > > > > > > > > > future
> > > > > > > > > > > > > > > > we give back intuitive and easy to program
> > > against?
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Some background info on implementation:
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > At a high level the primary difference in
> this
> > > > > producer
> > > > > > > is
> > > > > > > > > that
> > > > > > > > > > > it
> > > > > > > > > > > > > > > removes
> > > > > > > > > > > > > > > > the distinction between the "sync" and
> "async"
> > > > > > producer.
> > > > > > > > > > > > Effectively
> > > > > > > > > > > > > > all
> > > > > > > > > > > > > > > > requests are sent asynchronously but always
> > > return
> > > > a
> > > > > > > future
> > > > > > > > > > > > response
> > > > > > > > > > > > > > > object
> > > > > > > > > > > > > > > > that gives the offset as well as any error
> that
> > > may
> > > > > > have
> > > > > > > > > > occurred
> > > > > > > > > > > > > when
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > request is complete. The batching that is
> done
> > in
> > > > the
> > > > > > > async
> > > > > > > > > > > > producer
> > > > > > > > > > > > > > only
> > > > > > > > > > > > > > > > today is done whenever possible now. This
> means
> > > > that
> > > > > > the
> > > > > > > > sync
> > > > > > > > > > > > > producer,
> > > > > > > > > > > > > > > > under load, can get performance as good as
> the
> > > > async
> > > > > > > > producer
> > > > > > > > > > > > > > > (preliminary
> > > > > > > > > > > > > > > > results show the producer getting 1m
> > > messages/sec).
> > > > > > This
> > > > > > > > > works
> > > > > > > > > > > > > similar
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > group commit in databases but with respect to
> > the
> > > > > > actual
> > > > > > > > > > network
> > > > > > > > > > > > > > > > transmission--any messages that arrive while
> a
> > > send
> > > > > is
> > > > > > in
> > > > > > > > > > > progress
> > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > batched together. It is also possible to
> > > encourage
> > > > > > > batching
> > > > > > > > > > even
> > > > > > > > > > > > > under
> > > > > > > > > > > > > > > low
> > > > > > > > > > > > > > > > load to save server resources by introducing
> a
> > > > delay
> > > > > on
> > > > > > > the
> > > > > > > > > > send
> > > > > > > > > > > to
> > > > > > > > > > > > > > allow
> > > > > > > > > > > > > > > > more messages to accumulate; this is done
> using
> > > the
> > > > > > > > > > > linger.msconfig
> > > > > > > > > > > > > > > (this
> > > > > > > > > > > > > > > > is similar to Nagle's algorithm in TCP).
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > This producer does all network communication
> > > > > > > asynchronously
> > > > > > > > > and
> > > > > > > > > > > in
> > > > > > > > > > > > > > > parallel
> > > > > > > > > > > > > > > > to all servers so the performance penalty for
> > > > acks=-1
> > > > > > and
> > > > > > > > > > waiting
> > > > > > > > > > > > on
> > > > > > > > > > > > > > > > replication should be much reduced. I haven't
> > > done
> > > > > much
> > > > > > > > > > > > benchmarking
> > > > > > > > > > > > > on
> > > > > > > > > > > > > > > > this yet, though.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > The high level design is described a little
> > here,
> > > > > > though
> > > > > > > > this
> > > > > > > > > > is
> > > > > > > > > > > > now
> > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > little out of date:
> > > > > > > > > > > > > > > >
> > > > > > > > > >
> > > > https://cwiki.apache.org/confluence/display/KAFKA/Client+Rewrite
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > -Jay
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: New Producer Public API

Posted by Neha Narkhede <ne...@gmail.com>.
Regarding the use of Futures -

Agree that there are some downsides to using Futures but both approaches
have some tradeoffs.

- Standardization and usability
Future is a widely used and understood Java API and given that the
functionality that RecordSend hopes to provide is essentially that of
Future, I think it makes sense to expose a widely understood public API for
our clients. RecordSend, on the other hand, seems to provide some APIs that
are very similar to that of Future, in addition to exposing a bunch of APIs
that belong to ProduceRequestResult. As a user, I would've really preferred
to deal with ProduceRequestResult directly -
Future<ProduceRequestResult> send(...)

- Error handling
RecordSend's error handling is quite unintuitive where the user has to
remember to invoke hasError and error, instead of just throwing the
exception. Now there are
some downsides regarding error handling with the Future as well, where the
user has to catch InterruptedException when we would never run into it.
However, it seems like a price worth paying for supporting a standard API
and error handling

- Unused APIs
This is a downside of using Future, where the cancel() operation would
always return false and mean nothing. But we can mention that caveat in our
Java docs.

To summarize, I would prefer to expose a well understood and widely adopted
Java API and put up with the overhead of catching one unnecessary checked
exception, rather than wrap the useful ProduceRequestResult in a custom
async object (RecordSend) and explain that to our many users.

Thanks,
Neha




On Tue, Jan 28, 2014 at 8:10 PM, Jay Kreps <ja...@gmail.com> wrote:

> Hey Neha,
>
> Can you elaborate on why you prefer using Java's Future? The downside in my
> mind is the use of the checked InterruptedException and ExecutionException.
> ExecutionException is arguable, but forcing you to catch
> InterruptedException, often in code that can't be interrupted, seems
> perverse. It also leaves us with the cancel() method which I don't think we
> really can implement.
>
> Option 1A, to recap/elaborate, was the following. There is no Serializer or
> Partitioner api. We take a byte[] key and value and an optional integer
> partition. If you specify the integer partition it will be used. If you do
> not specify a key or a partition the partition will be chosen in a round
> robin fashion. If you specify a key but no partition we will chose a
> partition based on a hash of the key. In order to let the user find the
> partition we will need to given them access to the Cluster instance
> directly from the producer.
>
> -Jay
>
>
> On Tue, Jan 28, 2014 at 6:25 PM, Neha Narkhede <neha.narkhede@gmail.com
> >wrote:
>
> > Here are more thoughts on the public APIs -
> >
> > - I suggest we use java's Future instead of custom Future especially
> since
> > it is part of the public API
> >
> > - Serialization: I like the simplicity of the producer APIs with the
> > absence of serialization where we just deal with byte arrays for keys and
> > values. What I don't like about this is the performance overhead on the
> > Partitioner for any kind of custom partitioning based on the
> partitionKey.
> > Since the only purpose of partitionKey is to do custom partitioning, why
> > can't we take it in directly as an integer and let the user figure out
> the
> > mapping from partition_key -> partition_id using the getCluster() API?
> If I
> > understand correctly, this is similar to what you suggested as part of
> > option 1A. I like this approach since it maintains the simplicity of APIs
> > by allowing us to deal with bytes and does not compromise performance in
> > the custom partitioning case.
> >
> > Thanks,
> > Neha
> >
> >
> >
> > On Tue, Jan 28, 2014 at 5:42 PM, Jay Kreps <ja...@gmail.com> wrote:
> >
> > > Hey Tom,
> > >
> > > That sounds cool. How did you end up handling parallel I/O if you wrap
> > the
> > > individual connections? Don't you need some selector that selects over
> > all
> > > the connections?
> > >
> > > -Jay
> > >
> > >
> > > On Tue, Jan 28, 2014 at 2:31 PM, Tom Brown <to...@gmail.com>
> wrote:
> > >
> > > > I implemented a 0.7 client in pure java, and its API very closely
> > > resembled
> > > > this. (When multiple people independently engineer the same solution,
> > > it's
> > > > probably good... right?). However, there were a few architectural
> > > > differences with my client:
> > > >
> > > > 1. The basic client itself was just an asynchronous layer around the
> > > > different server functions. In and of itself it had no knowledge of
> > > > partitions, only servers (and maintained TCP connections to them).
> > > >
> > > > 2. The main producer was an additional layer that provided a
> high-level
> > > > interface that could batch individual messages based on partition.
> > > >
> > > > 3. Knowledge of partitioning was done via an interface so that
> > different
> > > > strategies could be used.
> > > >
> > > > 4. Partitioning was done by the user, with knowledge of the available
> > > > partitions provided by #3.
> > > >
> > > > 5. Serialization was done by the user to simplify the API.
> > > >
> > > > 6. Futures were used to make asynchronous emulate synchronous calls.
> > > >
> > > >
> > > > The main benefit of this approach is flexibility. For example, since
> > the
> > > > base client was just a managed connection (and not inherently a
> > > producer),
> > > > it was easy to composite a produce request and an offsets request
> > > together
> > > > into a confirmed produce request (officially not available in 0.7).
> > > >
> > > > Decoupling the basic client from partition management allowed the me
> to
> > > > implement zk discovery as a separate project so that the main project
> > had
> > > > no complex dependencies. The same was true of decoupling
> serialization.
> > > > It's trivial to build an optional layer that adds those features in,
> > > while
> > > > allowing access to the base APIs for those that need it.
> > > >
> > > > Using standard Future objects was also beneficial, since I could
> > combine
> > > > them with existing tools (such as guava).
> > > >
> > > > It may be too late to be of use, but I have been working with my
> > > company's
> > > > legal department to release the implementation I described above. If
> > > you're
> > > > interested in it, let me know.
> > > >
> > > >
> > > > To sum up my thoughts regarding the new API, I think it's a great
> > start.
> > > I
> > > > would like to see a more layered approach so I can use the parts I
> > want,
> > > > and adapt the other parts as needed. I would also like to see
> standard
> > > > interfaces (especially Future) used where they makes sense.
> > > >
> > > > --Tom
> > > >
> > > >
> > > >
> > > >
> > > >
> > > > On Tue, Jan 28, 2014 at 1:33 PM, Roger Hoover <
> roger.hoover@gmail.com
> > > > >wrote:
> > > >
> > > > > +1 ListenableFuture: If this works similar to Deferreds in Twisted
> > > Python
> > > > > or Promised IO in Javascript, I think this is a great pattern for
> > > > > decoupling your callback logic from the place where the Future is
> > > > > generated.  You can register as many callbacks as you like, each in
> > the
> > > > > appropriate layer of the code and have each observer get notified
> > when
> > > > the
> > > > > promised i/o is complete without any of them knowing about each
> > other.
> > > > >
> > > > >
> > > > > On Tue, Jan 28, 2014 at 11:32 AM, Jay Kreps <ja...@gmail.com>
> > > wrote:
> > > > >
> > > > > > Hey Ross,
> > > > > >
> > > > > > - ListenableFuture: Interesting. That would be an alternative to
> > the
> > > > > direct
> > > > > > callback support we provide. There could be pros to this, let me
> > > think
> > > > > > about it.
> > > > > > - We could provide layering, but I feel that the serialization is
> > > such
> > > > a
> > > > > > small thing we should just make a decision and chose one, it
> > doesn't
> > > > seem
> > > > > > to me to justify a whole public facing layer.
> > > > > > - Yes, this is fairly esoteric, essentially I think it is fairly
> > > > similar
> > > > > to
> > > > > > databases like DynamoDB that allow you to specify two partition
> > keys
> > > (I
> > > > > > think DynamoDB does this...). The reasoning is that in fact there
> > are
> > > > > > several things you can use the key field for: (1) to compute the
> > > > > partition
> > > > > > to store the data in, (2) as a unique identifier to deduplicate
> > that
> > > > > > partition's records within a log. These two things are almost
> > always
> > > > the
> > > > > > same, but occationally may differ when you want to group data in
> a
> > > more
> > > > > > sophisticated way then just a hash of the primary key but still
> > > retain
> > > > > the
> > > > > > proper primary key for delivery to the consumer and log
> compaction.
> > > > > >
> > > > > > -Jay
> > > > > >
> > > > > >
> > > > > > On Tue, Jan 28, 2014 at 3:24 AM, Ross Black <
> > ross.w.black@gmail.com>
> > > > > > wrote:
> > > > > >
> > > > > > > Hi Jay,
> > > > > > >
> > > > > > > - Just to add some more info/confusion about possibly using
> > Future
> > > > ...
> > > > > > >   If Kafka uses a JDK future, it plays nicely with other
> > frameworks
> > > > as
> > > > > > > well.
> > > > > > >   Google Guava has a ListenableFuture that allows callback
> > handling
> > > > to
> > > > > be
> > > > > > > added via the returned future, and allows the callbacks to be
> > > passed
> > > > > off
> > > > > > to
> > > > > > > a specified executor.
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> http://docs.guava-libraries.googlecode.com/git-history/release/javadoc/com/google/common/util/concurrent/ListenableFuture.html
> > > > > > >   The JDK future can easily be converted to a listenable
> future.
> > > > > > >
> > > > > > > - On the question of byte[] vs Object, could this be solved by
> > > > layering
> > > > > > the
> > > > > > > API?  eg. a raw producer (use byte[] and specify the partition
> > > > number)
> > > > > > and
> > > > > > > a normal producer (use generic object and specify a
> Partitioner)?
> > > > > > >
> > > > > > > - I am confused by the keys in ProducerRecord and Partitioner.
> > >  What
> > > > is
> > > > > > the
> > > > > > > usage for both a key and a partition key? (I am not yet using
> > 0.8)
> > > > > > >
> > > > > > >
> > > > > > > Thanks,
> > > > > > > Ross
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > On 28 January 2014 05:00, Xavier Stevens <xa...@gaikai.com>
> > > wrote:
> > > > > > >
> > > > > > > > AutoCloseable would be nice for us as most of our code is
> using
> > > > Java
> > > > > 7
> > > > > > at
> > > > > > > > this point.
> > > > > > > >
> > > > > > > > I like Dropwizard's configuration mapping to POJOs via
> Jackson,
> > > but
> > > > > if
> > > > > > > you
> > > > > > > > wanted to stick with property maps I don't care enough to
> > object.
> > > > > > > >
> > > > > > > > If the producer only dealt with bytes, is there a way we
> could
> > > > still
> > > > > > due
> > > > > > > > partition plugins without specifying the number explicitly? I
> > > would
> > > > > > > prefer
> > > > > > > > to be able to pass in field(s) that would be used by the
> > > > partitioner.
> > > > > > > > Obviously if this wasn't possible you could always
> deserialize
> > > the
> > > > > > object
> > > > > > > > in the partitioner and grab the fields you want, but that
> seems
> > > > > really
> > > > > > > > expensive to do on every message.
> > > > > > > >
> > > > > > > > It would also be nice to have a Java API Encoder constructor
> > > taking
> > > > > in
> > > > > > > > VerifiableProperties. Scala understands how to handle "props:
> > > > > > > > VerifiableProperties = null", but Java doesn't. So you don't
> > run
> > > > into
> > > > > > > this
> > > > > > > > problem until runtime.
> > > > > > > >
> > > > > > > >
> > > > > > > > -Xavier
> > > > > > > >
> > > > > > > >
> > > > > > > > On Mon, Jan 27, 2014 at 9:37 AM, Clark Breyman <
> > > clark@breyman.com>
> > > > > > > wrote:
> > > > > > > >
> > > > > > > > > Jay -
> > > > > > > > >
> > > > > > > > > Config - your explanation makes sense. I'm just so
> accustomed
> > > to
> > > > > > having
> > > > > > > > > Jackson automatically map my configuration objects to POJOs
> > > that
> > > > > I've
> > > > > > > > > stopped using property files. They are lingua franca. The
> > only
> > > > > > thought
> > > > > > > > > might be to separate the config interface from the
> > > implementation
> > > > > to
> > > > > > > > allow
> > > > > > > > > for alternatives, but that might undermine your point of
> "do
> > it
> > > > > this
> > > > > > > way
> > > > > > > > so
> > > > > > > > > that everyone can find it where they expect it".
> > > > > > > > >
> > > > > > > > > Serialization: Of the options, I like 1A the best, though
> > > > possibly
> > > > > > with
> > > > > > > > > either an option to specify a partition key rather than ID
> > or a
> > > > > > helper
> > > > > > > to
> > > > > > > > > translate an arbitrary byte[] or long into a partition
> > number.
> > > > > > > > >
> > > > > > > > > Thanks
> > > > > > > > > Clark
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > On Sun, Jan 26, 2014 at 9:13 PM, Jay Kreps <
> > > jay.kreps@gmail.com>
> > > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > Thanks for the detailed thoughts. Let me elaborate on the
> > > > config
> > > > > > > thing.
> > > > > > > > > >
> > > > > > > > > > I agree that at first glance key-value strings don't seem
> > > like
> > > > a
> > > > > > very
> > > > > > > > > good
> > > > > > > > > > configuration api for a client. Surely a well-typed
> config
> > > > class
> > > > > > > would
> > > > > > > > be
> > > > > > > > > > better! I actually disagree and let me see if I can
> > convince
> > > > you.
> > > > > > > > > >
> > > > > > > > > > My reasoning has nothing to do with the api and
> everything
> > to
> > > > do
> > > > > > with
> > > > > > > > > > operations.
> > > > > > > > > >
> > > > > > > > > > Clients are embedded in applications which are themselves
> > > > > > configured.
> > > > > > > > In
> > > > > > > > > > any place that takes operations seriously the
> configuration
> > > for
> > > > > > these
> > > > > > > > > > applications will be version controlled and maintained
> > > through
> > > > > some
> > > > > > > > kind
> > > > > > > > > of
> > > > > > > > > > config management system. If we give a config class with
> > > > getters
> > > > > > and
> > > > > > > > > > setters the application has to expose those properties to
> > its
> > > > > > > > > > configuration. What invariably happens is that the
> > > application
> > > > > > > exposes
> > > > > > > > > only
> > > > > > > > > > a choice few properties that they thought they would
> > change.
> > > > > > > > Furthermore
> > > > > > > > > > the application will make up a name for these configs
> that
> > > > seems
> > > > > > > > > intuitive
> > > > > > > > > > at the time in the 2 seconds the engineer spends thinking
> > > about
> > > > > it.
> > > > > > > > > >
> > > > > > > > > > Now consider the result of this in the large. You end up
> > with
> > > > > > dozens
> > > > > > > or
> > > > > > > > > > hundreds of applications that have the client embedded.
> > Each
> > > > > > exposes
> > > > > > > a
> > > > > > > > > > different, inadequate subset of the possible configs,
> each
> > > with
> > > > > > > > different
> > > > > > > > > > names. It is a nightmare.
> > > > > > > > > >
> > > > > > > > > > If you use a string-string map the config system can
> > directly
> > > > > get a
> > > > > > > > > bundle
> > > > > > > > > > of config key-value pairs and put them into the client.
> > This
> > > > > means
> > > > > > > that
> > > > > > > > > all
> > > > > > > > > > configuration is automatically available with the name
> > > > documented
> > > > > > on
> > > > > > > > the
> > > > > > > > > > website in every application that does this. If you
> upgrade
> > > to
> > > > a
> > > > > > new
> > > > > > > > > kafka
> > > > > > > > > > version with more configs those will be exposed too. If
> you
> > > > > realize
> > > > > > > > that
> > > > > > > > > > you need to change a default you can just go through your
> > > > configs
> > > > > > and
> > > > > > > > > > change it everywhere as it will have the same name
> > > everywhere.
> > > > > > > > > >
> > > > > > > > > > -Jay
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > On Sun, Jan 26, 2014 at 4:47 PM, Clark Breyman <
> > > > > clark@breyman.com>
> > > > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > Thanks Jay. I'll see if I can put together a more
> > complete
> > > > > > > response,
> > > > > > > > > > > perhaps as separate threads so that topics don't get
> > > > entangled.
> > > > > > In
> > > > > > > > the
> > > > > > > > > > mean
> > > > > > > > > > > time, here's a couple responses:
> > > > > > > > > > >
> > > > > > > > > > > Serialization: you've broken out a sub-thread so i'll
> > reply
> > > > > > there.
> > > > > > > My
> > > > > > > > > > bias
> > > > > > > > > > > is that I like generics (except for type-erasure) and
> in
> > > > > > particular
> > > > > > > > > they
> > > > > > > > > > > make it easy to compose serializers for compound
> payloads
> > > > (e.g.
> > > > > > > when
> > > > > > > > a
> > > > > > > > > > > common header wraps a payload of parameterized type).
> > I'll
> > > > > > respond
> > > > > > > to
> > > > > > > > > > your
> > > > > > > > > > > 4-options message with an example.
> > > > > > > > > > >
> > > > > > > > > > > Build: I've seen a lot of "maven-compatible" build
> > systems
> > > > > > produce
> > > > > > > > > > > "artifacts" that aren't really artifacts - no embedded
> > POM
> > > > or,
> > > > > > > worst,
> > > > > > > > > > > malformed POM. I know the sbt-generated artifacts were
> > this
> > > > > way -
> > > > > > > > onus
> > > > > > > > > is
> > > > > > > > > > > on me to see what gradle is spitting out and what a
> maven
> > > > build
> > > > > > > might
> > > > > > > > > > look
> > > > > > > > > > > like. Maven may be old and boring, but it gets out of
> the
> > > way
> > > > > and
> > > > > > > > > > > integrates really seamlessly with a lot of IDEs. When
> > some
> > > > > scala
> > > > > > > > > > projects I
> > > > > > > > > > > was working on in the fall of 2011 switched from sbt to
> > > > maven,
> > > > > > > build
> > > > > > > > > > became
> > > > > > > > > > > a non-issue.
> > > > > > > > > > >
> > > > > > > > > > > Config: Not a big deal  and no, I don't think a
> > dropwizard
> > > > > > > dependency
> > > > > > > > > is
> > > > > > > > > > > appropriate. I do like using simple entity beans
> (POJO's
> > > not
> > > > > > j2EE)
> > > > > > > > for
> > > > > > > > > > > configuration, especially if they can be marshalled
> > without
> > > > > > > > annotation
> > > > > > > > > by
> > > > > > > > > > > Jackson. I only mentioned the dropwizard-extras
>  because
> > it
> > > > has
> > > > > > > some
> > > > > > > > > > entity
> > > > > > > > > > > bean versions of the ZK and Kafka configs.
> > > > > > > > > > >
> > > > > > > > > > > Domain-packaging: Also not a big deal - it's what's
> > > expected
> > > > > and
> > > > > > > it's
> > > > > > > > > > > pretty free in most IDE's. The advantages I see is that
> > it
> > > is
> > > > > > clear
> > > > > > > > > > whether
> > > > > > > > > > > something is from the Apache Kafka project and whether
> > > > > something
> > > > > > is
> > > > > > > > > from
> > > > > > > > > > > another org and related to Kafka. That said, nothing
> > really
> > > > > > > enforces
> > > > > > > > > it.
> > > > > > > > > > >
> > > > > > > > > > > Futures: I'll see if I can create some examples to
> > > > demonstrate
> > > > > > > Future
> > > > > > > > > > > making interop easier.
> > > > > > > > > > >
> > > > > > > > > > > Regards,
> > > > > > > > > > > C
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > On Fri, Jan 24, 2014 at 4:36 PM, Jay Kreps <
> > > > > jay.kreps@gmail.com>
> > > > > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > Hey Clark,
> > > > > > > > > > > >
> > > > > > > > > > > > - Serialization: Yes I agree with these though I
> don't
> > > > > consider
> > > > > > > the
> > > > > > > > > > loss
> > > > > > > > > > > of
> > > > > > > > > > > > generics a big issue. I'll try to summarize what I
> > would
> > > > > > consider
> > > > > > > > the
> > > > > > > > > > > best
> > > > > > > > > > > > alternative api with raw byte[].
> > > > > > > > > > > >
> > > > > > > > > > > > - Maven: We had this debate a few months back and the
> > > > > consensus
> > > > > > > was
> > > > > > > > > > > gradle.
> > > > > > > > > > > > Is there a specific issue with the poms gradle
> makes? I
> > > am
> > > > > > > > extremely
> > > > > > > > > > > loath
> > > > > > > > > > > > to revisit the issue as build issues are a recurring
> > > thing
> > > > > and
> > > > > > no
> > > > > > > > one
> > > > > > > > > > > ever
> > > > > > > > > > > > agrees and ultimately our build needs are very
> simple.
> > > > > > > > > > > >
> > > > > > > > > > > > - Config: I'm not sure if I follow the point. Are you
> > > > saying
> > > > > we
> > > > > > > > > should
> > > > > > > > > > > use
> > > > > > > > > > > > something in dropwizard for config? One principle
> here
> > is
> > > > to
> > > > > > try
> > > > > > > to
> > > > > > > > > > > remove
> > > > > > > > > > > > as many client dependencies as possible as we
> > inevitably
> > > > run
> > > > > > into
> > > > > > > > > > > terrible
> > > > > > > > > > > > compatibility issues with users who use the same
> > library
> > > or
> > > > > its
> > > > > > > > > > > > dependencies at different versions. Or are you
> talking
> > > > about
> > > > > > > > > > maintaining
> > > > > > > > > > > > compatibility with existing config parameters? I
> think
> > as
> > > > > much
> > > > > > > as a
> > > > > > > > > > > config
> > > > > > > > > > > > in the existing client makes sense it should have the
> > > same
> > > > > name
> > > > > > > (I
> > > > > > > > > was
> > > > > > > > > > a
> > > > > > > > > > > > bit sloppy about that so I'll fix any errors there).
> > > There
> > > > > are
> > > > > > a
> > > > > > > > few
> > > > > > > > > > new
> > > > > > > > > > > > things and we should give those reasonable defaults.
> I
> > > > think
> > > > > > > config
> > > > > > > > > is
> > > > > > > > > > > > important so I'll start a thread on the config
> package
> > in
> > > > > > there.
> > > > > > > > > > > >
> > > > > > > > > > > > - org.apache.kafka: We could do this. I always
> > considered
> > > > it
> > > > > > kind
> > > > > > > > of
> > > > > > > > > an
> > > > > > > > > > > odd
> > > > > > > > > > > > thing Java programmers do that has no real motivation
> > > (but
> > > > I
> > > > > > > could
> > > > > > > > be
> > > > > > > > > > > > re-educated!). I don't think it ends up reducing
> naming
> > > > > > conflicts
> > > > > > > > in
> > > > > > > > > > > > practice and it adds a lot of noise and nested
> > > directories.
> > > > > Is
> > > > > > > > there
> > > > > > > > > a
> > > > > > > > > > > > reason you prefer this or just to be more standard?
> > > > > > > > > > > >
> > > > > > > > > > > > - Future: Basically I didn't see any particular
> > > advantage.
> > > > > The
> > > > > > > > > cancel()
> > > > > > > > > > > > method doesn't really make sense so probably wouldn't
> > > work.
> > > > > > > > Likewise
> > > > > > > > > I
> > > > > > > > > > > > dislike the checked exceptions it requires.
> Basically I
> > > > just
> > > > > > > wrote
> > > > > > > > > out
> > > > > > > > > > > some
> > > > > > > > > > > > code examples and it seemed cleaner with a special
> > > purpose
> > > > > > > object.
> > > > > > > > I
> > > > > > > > > > > wasn't
> > > > > > > > > > > > actually aware of plans for improved futures in java
> 8
> > or
> > > > the
> > > > > > > other
> > > > > > > > > > > > integrations. Maybe you could elaborate on this a bit
> > and
> > > > > show
> > > > > > > how
> > > > > > > > it
> > > > > > > > > > > would
> > > > > > > > > > > > be used? Sounds promising, I just don't know a lot
> > about
> > > > it.
> > > > > > > > > > > >
> > > > > > > > > > > > -Jay
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > On Fri, Jan 24, 2014 at 3:30 PM, Clark Breyman <
> > > > > > > clark@breyman.com>
> > > > > > > > > > > wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > > Jay - Thanks for the call for comments. Here's some
> > > > initial
> > > > > > > > input:
> > > > > > > > > > > > >
> > > > > > > > > > > > > - Make message serialization a client
> responsibility
> > > > > (making
> > > > > > > all
> > > > > > > > > > > messages
> > > > > > > > > > > > > byte[]). Reflection-based loading makes it harder
> to
> > > use
> > > > > > > generic
> > > > > > > > > > codecs
> > > > > > > > > > > > > (e.g.  Envelope<PREFIX, DATA, SUFFIX>) or build up
> > > codec
> > > > > > > > > > > > programmatically.
> > > > > > > > > > > > > Non-default partitioning should require an explicit
> > > > > partition
> > > > > > > > key.
> > > > > > > > > > > > >
> > > > > > > > > > > > > - I really like the fact that it will be native
> Java.
> > > > > Please
> > > > > > > > > consider
> > > > > > > > > > > > using
> > > > > > > > > > > > > native maven and not sbt, gradle, ivy, etc as they
> > > don't
> > > > > > > reliably
> > > > > > > > > > play
> > > > > > > > > > > > nice
> > > > > > > > > > > > > in the maven ecosystem. A jar without a well-formed
> > pom
> > > > > > doesn't
> > > > > > > > > feel
> > > > > > > > > > > > like a
> > > > > > > > > > > > > real artifact. The pom's generated by sbt et al.
> are
> > > not
> > > > > well
> > > > > > > > > formed.
> > > > > > > > > > > > Using
> > > > > > > > > > > > > maven will make builds and IDE integration much
> > > smoother.
> > > > > > > > > > > > >
> > > > > > > > > > > > > - Look at Nick Telford's dropwizard-extras package
> in
> > > > which
> > > > > > he
> > > > > > > > > > defines
> > > > > > > > > > > > some
> > > > > > > > > > > > > Jackson-compatible POJO's for loading
> configuration.
> > > > Seems
> > > > > > like
> > > > > > > > > your
> > > > > > > > > > > > client
> > > > > > > > > > > > > migration is similar. The config objects should
> have
> > > > > > > constructors
> > > > > > > > > or
> > > > > > > > > > > > > factories that accept Map<String, String> and
> > > Properties
> > > > > for
> > > > > > > ease
> > > > > > > > > of
> > > > > > > > > > > > > migration.
> > > > > > > > > > > > >
> > > > > > > > > > > > > - Would you consider using the org.apache.kafka
> > package
> > > > for
> > > > > > the
> > > > > > > > new
> > > > > > > > > > API
> > > > > > > > > > > > > (quibble)
> > > > > > > > > > > > >
> > > > > > > > > > > > > - Why create your own futures rather than use
> > > > > > > > > > > > > java.util.concurrent.Future<Long> or similar?
> > Standard
> > > > > > futures
> > > > > > > > will
> > > > > > > > > > > play
> > > > > > > > > > > > > nice with other reactive libs and things like J8's
> > > > > > > > > ComposableFuture.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Thanks again,
> > > > > > > > > > > > > C
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Fri, Jan 24, 2014 at 2:46 PM, Roger Hoover <
> > > > > > > > > > roger.hoover@gmail.com
> > > > > > > > > > > > > >wrote:
> > > > > > > > > > > > >
> > > > > > > > > > > > > > A couple comments:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 1) Why does the config use a broker list instead
> of
> > > > > > > discovering
> > > > > > > > > the
> > > > > > > > > > > > > brokers
> > > > > > > > > > > > > > in ZooKeeper?  It doesn't match the
> > HighLevelConsumer
> > > > > API.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 2) It looks like broker connections are created
> on
> > > > > demand.
> > > > > > >  I'm
> > > > > > > > > > > > wondering
> > > > > > > > > > > > > > if sometimes you might want to flush out config
> or
> > > > > network
> > > > > > > > > > > connectivity
> > > > > > > > > > > > > > issues before pushing the first message through.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Should there also be a KafkaProducer.connect() or
> > > > .open()
> > > > > > > > method
> > > > > > > > > or
> > > > > > > > > > > > > > connectAll()?  I guess it would try to connect to
> > all
> > > > > > brokers
> > > > > > > > in
> > > > > > > > > > the
> > > > > > > > > > > > > > BROKER_LIST_CONFIG
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > HTH,
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Roger
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On Fri, Jan 24, 2014 at 11:54 AM, Jay Kreps <
> > > > > > > > jay.kreps@gmail.com
> > > > > > > > > >
> > > > > > > > > > > > wrote:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > > As mentioned in a previous email we are working
> > on
> > > a
> > > > > > > > > > > > re-implementation
> > > > > > > > > > > > > of
> > > > > > > > > > > > > > > the producer. I would like to use this email
> > thread
> > > > to
> > > > > > > > discuss
> > > > > > > > > > the
> > > > > > > > > > > > > > details
> > > > > > > > > > > > > > > of the public API and the configuration. I
> would
> > > love
> > > > > for
> > > > > > > us
> > > > > > > > to
> > > > > > > > > > be
> > > > > > > > > > > > > > > incredibly picky about this public api now so
> it
> > is
> > > > as
> > > > > > good
> > > > > > > > as
> > > > > > > > > > > > possible
> > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > we don't need to break it in the future.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > The best way to get a feel for the API is
> > actually
> > > to
> > > > > > take
> > > > > > > a
> > > > > > > > > look
> > > > > > > > > > > at
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > javadoc, my hope is to get the api docs good
> > enough
> > > > so
> > > > > > that
> > > > > > > > it
> > > > > > > > > is
> > > > > > > > > > > > > > > self-explanatory:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> http://empathybox.com/kafka-javadoc/index.html?kafka/clients/producer/KafkaProducer.html
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Please take a look at this API and give me any
> > > > thoughts
> > > > > > you
> > > > > > > > may
> > > > > > > > > > > have!
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > It may also be reasonable to take a look at the
> > > > > configs:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> http://empathybox.com/kafka-javadoc/kafka/clients/producer/ProducerConfig.html
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > The actual code is posted here:
> > > > > > > > > > > > > > >
> https://issues.apache.org/jira/browse/KAFKA-1227
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > A few questions or comments to kick things off:
> > > > > > > > > > > > > > > 1. We need to make a decision on whether
> > > > serialization
> > > > > of
> > > > > > > the
> > > > > > > > > > > user's
> > > > > > > > > > > > > key
> > > > > > > > > > > > > > > and value should be done by the user (with our
> > api
> > > > just
> > > > > > > > taking
> > > > > > > > > > > > byte[])
> > > > > > > > > > > > > or
> > > > > > > > > > > > > > > if we should take an object and allow the user
> to
> > > > > > > configure a
> > > > > > > > > > > > > Serializer
> > > > > > > > > > > > > > > class which we instantiate via reflection. We
> > take
> > > > the
> > > > > > > later
> > > > > > > > > > > approach
> > > > > > > > > > > > > in
> > > > > > > > > > > > > > > the current producer, and I have carried this
> > > through
> > > > > to
> > > > > > > this
> > > > > > > > > > > > > prototype.
> > > > > > > > > > > > > > > The tradeoff I see is this: taking byte[] is
> > > actually
> > > > > > > > simpler,
> > > > > > > > > > the
> > > > > > > > > > > > user
> > > > > > > > > > > > > > can
> > > > > > > > > > > > > > > directly do whatever serialization they like.
> The
> > > > > > > > complication
> > > > > > > > > is
> > > > > > > > > > > > > > actually
> > > > > > > > > > > > > > > partitioning. Currently partitioning is done
> by a
> > > > > similar
> > > > > > > > > plug-in
> > > > > > > > > > > api
> > > > > > > > > > > > > > > (Partitioner) which the user can implement and
> > > > > configure
> > > > > > to
> > > > > > > > > > > override
> > > > > > > > > > > > > how
> > > > > > > > > > > > > > > partitions are assigned. If we take byte[] as
> > input
> > > > > then
> > > > > > we
> > > > > > > > > have
> > > > > > > > > > no
> > > > > > > > > > > > > > access
> > > > > > > > > > > > > > > to the original object and partitioning MUST be
> > > done
> > > > on
> > > > > > the
> > > > > > > > > > byte[].
> > > > > > > > > > > > > This
> > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > fine for hash partitioning. However for various
> > > types
> > > > > of
> > > > > > > > > semantic
> > > > > > > > > > > > > > > partitioning (range partitioning, or whatever)
> > you
> > > > > would
> > > > > > > want
> > > > > > > > > > > access
> > > > > > > > > > > > to
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > original object. In the current approach a
> > producer
> > > > who
> > > > > > > > wishes
> > > > > > > > > to
> > > > > > > > > > > > send
> > > > > > > > > > > > > > > byte[] they have serialized in their own code
> can
> > > > > > configure
> > > > > > > > the
> > > > > > > > > > > > > > > BytesSerialization we supply which is just a
> "no
> > > op"
> > > > > > > > > > serialization.
> > > > > > > > > > > > > > > 2. We should obsess over naming and make sure
> > each
> > > of
> > > > > the
> > > > > > > > class
> > > > > > > > > > > names
> > > > > > > > > > > > > are
> > > > > > > > > > > > > > > good.
> > > > > > > > > > > > > > > 3. Jun has already pointed out that we need to
> > > > include
> > > > > > the
> > > > > > > > > topic
> > > > > > > > > > > and
> > > > > > > > > > > > > > > partition in the response, which is absolutely
> > > > right. I
> > > > > > > > haven't
> > > > > > > > > > > done
> > > > > > > > > > > > > that
> > > > > > > > > > > > > > > yet but that definitely needs to be there.
> > > > > > > > > > > > > > > 4. Currently RecordSend.await will throw an
> > > exception
> > > > > if
> > > > > > > the
> > > > > > > > > > > request
> > > > > > > > > > > > > > > failed. The intention here is that
> > > > > > > > > producer.send(message).await()
> > > > > > > > > > > > > exactly
> > > > > > > > > > > > > > > simulates a synchronous call. Guozhang has
> noted
> > > that
> > > > > > this
> > > > > > > > is a
> > > > > > > > > > > > little
> > > > > > > > > > > > > > > annoying since the user must then catch
> > exceptions.
> > > > > > However
> > > > > > > > if
> > > > > > > > > we
> > > > > > > > > > > > > remove
> > > > > > > > > > > > > > > this then if the user doesn't check for errors
> > they
> > > > > won't
> > > > > > > > know
> > > > > > > > > > one
> > > > > > > > > > > > has
> > > > > > > > > > > > > > > occurred, which I predict will be a common
> > mistake.
> > > > > > > > > > > > > > > 5. Perhaps there is more we could do to make
> the
> > > > async
> > > > > > > > > callbacks
> > > > > > > > > > > and
> > > > > > > > > > > > > > future
> > > > > > > > > > > > > > > we give back intuitive and easy to program
> > against?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Some background info on implementation:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > At a high level the primary difference in this
> > > > producer
> > > > > > is
> > > > > > > > that
> > > > > > > > > > it
> > > > > > > > > > > > > > removes
> > > > > > > > > > > > > > > the distinction between the "sync" and "async"
> > > > > producer.
> > > > > > > > > > > Effectively
> > > > > > > > > > > > > all
> > > > > > > > > > > > > > > requests are sent asynchronously but always
> > return
> > > a
> > > > > > future
> > > > > > > > > > > response
> > > > > > > > > > > > > > object
> > > > > > > > > > > > > > > that gives the offset as well as any error that
> > may
> > > > > have
> > > > > > > > > occurred
> > > > > > > > > > > > when
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > request is complete. The batching that is done
> in
> > > the
> > > > > > async
> > > > > > > > > > > producer
> > > > > > > > > > > > > only
> > > > > > > > > > > > > > > today is done whenever possible now. This means
> > > that
> > > > > the
> > > > > > > sync
> > > > > > > > > > > > producer,
> > > > > > > > > > > > > > > under load, can get performance as good as the
> > > async
> > > > > > > producer
> > > > > > > > > > > > > > (preliminary
> > > > > > > > > > > > > > > results show the producer getting 1m
> > messages/sec).
> > > > > This
> > > > > > > > works
> > > > > > > > > > > > similar
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > group commit in databases but with respect to
> the
> > > > > actual
> > > > > > > > > network
> > > > > > > > > > > > > > > transmission--any messages that arrive while a
> > send
> > > > is
> > > > > in
> > > > > > > > > > progress
> > > > > > > > > > > > are
> > > > > > > > > > > > > > > batched together. It is also possible to
> > encourage
> > > > > > batching
> > > > > > > > > even
> > > > > > > > > > > > under
> > > > > > > > > > > > > > low
> > > > > > > > > > > > > > > load to save server resources by introducing a
> > > delay
> > > > on
> > > > > > the
> > > > > > > > > send
> > > > > > > > > > to
> > > > > > > > > > > > > allow
> > > > > > > > > > > > > > > more messages to accumulate; this is done using
> > the
> > > > > > > > > > linger.msconfig
> > > > > > > > > > > > > > (this
> > > > > > > > > > > > > > > is similar to Nagle's algorithm in TCP).
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > This producer does all network communication
> > > > > > asynchronously
> > > > > > > > and
> > > > > > > > > > in
> > > > > > > > > > > > > > parallel
> > > > > > > > > > > > > > > to all servers so the performance penalty for
> > > acks=-1
> > > > > and
> > > > > > > > > waiting
> > > > > > > > > > > on
> > > > > > > > > > > > > > > replication should be much reduced. I haven't
> > done
> > > > much
> > > > > > > > > > > benchmarking
> > > > > > > > > > > > on
> > > > > > > > > > > > > > > this yet, though.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > The high level design is described a little
> here,
> > > > > though
> > > > > > > this
> > > > > > > > > is
> > > > > > > > > > > now
> > > > > > > > > > > > a
> > > > > > > > > > > > > > > little out of date:
> > > > > > > > > > > > > > >
> > > > > > > > >
> > > https://cwiki.apache.org/confluence/display/KAFKA/Client+Rewrite
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > -Jay
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: New Producer Public API

Posted by Tom Brown <to...@gmail.com>.
Jay,

I think you're confused between my use of "basic client" and "connection".
There is one basic client for a cluster. An IO thread manages the tcp
connections for any number of brokers. The basic client has a queue of
requests each broker. When a tcp connection (associated with broker X) is
ready to send the next request, it asks the basic client for the next
request for broker X.

The producer is just a layer that maps partitions to brokers so you only
have to tell it to send to partiton #3, and it knows that partition #3 goes
to broker X, and adds a produce request to the queue for broker X.

Conceivably (though I haven't implemented it yet), a multi-produce request
could be used in the same way. Since request pipelining is in place, I
don't see a good reason to use multi-produce.

Did I clear it up any, or is this just more confusing?

--Tom





On Wed, Jan 29, 2014 at 11:00 AM, Jay Kreps <ja...@gmail.com> wrote:

> Hey Tom,
>
> So is there one connection and I/O thread per broker and a low-level client
> for each of those, and then you hash into that to partition? Is it possible
> to batch across partitions or only within a partition?
>
> -Jay
>
>
> On Wed, Jan 29, 2014 at 8:41 AM, Tom Brown <to...@gmail.com> wrote:
>
> > Jay,
> >
> > There is both a basic client object, and a number of IO processing
> threads.
> > The client object manages connections, creating new ones when new
> machines
> > are connected, or when existing connections die. It also manages a queue
> of
> > requests for each server. The IO processing thread has a selector, and
> > performs the work of sending/receiving (removing items from the queue,
> > interpreting the response at a basic level, etc). Since asynchronous
> > sockets by nature decouple sending and receiving, request pipelining is
> > inherent.
> >
> > Using the basic client, you can send individual produce requests
> (singular
> > or batched). The "producer" layer adds an additional queue for each
> > partition, allowing individual messages to be batched together.
> >
> > --Tom
> >
> >
> >
> > On Tue, Jan 28, 2014 at 9:10 PM, Jay Kreps <ja...@gmail.com> wrote:
> >
> > > Hey Neha,
> > >
> > > Can you elaborate on why you prefer using Java's Future? The downside
> in
> > my
> > > mind is the use of the checked InterruptedException and
> > ExecutionException.
> > > ExecutionException is arguable, but forcing you to catch
> > > InterruptedException, often in code that can't be interrupted, seems
> > > perverse. It also leaves us with the cancel() method which I don't
> think
> > we
> > > really can implement.
> > >
> > > Option 1A, to recap/elaborate, was the following. There is no
> Serializer
> > or
> > > Partitioner api. We take a byte[] key and value and an optional integer
> > > partition. If you specify the integer partition it will be used. If you
> > do
> > > not specify a key or a partition the partition will be chosen in a
> round
> > > robin fashion. If you specify a key but no partition we will chose a
> > > partition based on a hash of the key. In order to let the user find the
> > > partition we will need to given them access to the Cluster instance
> > > directly from the producer.
> > >
> > > -Jay
> > >
> > >
> > > On Tue, Jan 28, 2014 at 6:25 PM, Neha Narkhede <
> neha.narkhede@gmail.com
> > > >wrote:
> > >
> > > > Here are more thoughts on the public APIs -
> > > >
> > > > - I suggest we use java's Future instead of custom Future especially
> > > since
> > > > it is part of the public API
> > > >
> > > > - Serialization: I like the simplicity of the producer APIs with the
> > > > absence of serialization where we just deal with byte arrays for keys
> > and
> > > > values. What I don't like about this is the performance overhead on
> the
> > > > Partitioner for any kind of custom partitioning based on the
> > > partitionKey.
> > > > Since the only purpose of partitionKey is to do custom partitioning,
> > why
> > > > can't we take it in directly as an integer and let the user figure
> out
> > > the
> > > > mapping from partition_key -> partition_id using the getCluster()
> API?
> > > If I
> > > > understand correctly, this is similar to what you suggested as part
> of
> > > > option 1A. I like this approach since it maintains the simplicity of
> > APIs
> > > > by allowing us to deal with bytes and does not compromise performance
> > in
> > > > the custom partitioning case.
> > > >
> > > > Thanks,
> > > > Neha
> > > >
> > > >
> > > >
> > > > On Tue, Jan 28, 2014 at 5:42 PM, Jay Kreps <ja...@gmail.com>
> > wrote:
> > > >
> > > > > Hey Tom,
> > > > >
> > > > > That sounds cool. How did you end up handling parallel I/O if you
> > wrap
> > > > the
> > > > > individual connections? Don't you need some selector that selects
> > over
> > > > all
> > > > > the connections?
> > > > >
> > > > > -Jay
> > > > >
> > > > >
> > > > > On Tue, Jan 28, 2014 at 2:31 PM, Tom Brown <to...@gmail.com>
> > > wrote:
> > > > >
> > > > > > I implemented a 0.7 client in pure java, and its API very closely
> > > > > resembled
> > > > > > this. (When multiple people independently engineer the same
> > solution,
> > > > > it's
> > > > > > probably good... right?). However, there were a few architectural
> > > > > > differences with my client:
> > > > > >
> > > > > > 1. The basic client itself was just an asynchronous layer around
> > the
> > > > > > different server functions. In and of itself it had no knowledge
> of
> > > > > > partitions, only servers (and maintained TCP connections to
> them).
> > > > > >
> > > > > > 2. The main producer was an additional layer that provided a
> > > high-level
> > > > > > interface that could batch individual messages based on
> partition.
> > > > > >
> > > > > > 3. Knowledge of partitioning was done via an interface so that
> > > > different
> > > > > > strategies could be used.
> > > > > >
> > > > > > 4. Partitioning was done by the user, with knowledge of the
> > available
> > > > > > partitions provided by #3.
> > > > > >
> > > > > > 5. Serialization was done by the user to simplify the API.
> > > > > >
> > > > > > 6. Futures were used to make asynchronous emulate synchronous
> > calls.
> > > > > >
> > > > > >
> > > > > > The main benefit of this approach is flexibility. For example,
> > since
> > > > the
> > > > > > base client was just a managed connection (and not inherently a
> > > > > producer),
> > > > > > it was easy to composite a produce request and an offsets request
> > > > > together
> > > > > > into a confirmed produce request (officially not available in
> 0.7).
> > > > > >
> > > > > > Decoupling the basic client from partition management allowed the
> > me
> > > to
> > > > > > implement zk discovery as a separate project so that the main
> > project
> > > > had
> > > > > > no complex dependencies. The same was true of decoupling
> > > serialization.
> > > > > > It's trivial to build an optional layer that adds those features
> > in,
> > > > > while
> > > > > > allowing access to the base APIs for those that need it.
> > > > > >
> > > > > > Using standard Future objects was also beneficial, since I could
> > > > combine
> > > > > > them with existing tools (such as guava).
> > > > > >
> > > > > > It may be too late to be of use, but I have been working with my
> > > > > company's
> > > > > > legal department to release the implementation I described above.
> > If
> > > > > you're
> > > > > > interested in it, let me know.
> > > > > >
> > > > > >
> > > > > > To sum up my thoughts regarding the new API, I think it's a great
> > > > start.
> > > > > I
> > > > > > would like to see a more layered approach so I can use the parts
> I
> > > > want,
> > > > > > and adapt the other parts as needed. I would also like to see
> > > standard
> > > > > > interfaces (especially Future) used where they makes sense.
> > > > > >
> > > > > > --Tom
> > > > > >
> > > > > >
> > > > > >
> > > > > >
> > > > > >
> > > > > > On Tue, Jan 28, 2014 at 1:33 PM, Roger Hoover <
> > > roger.hoover@gmail.com
> > > > > > >wrote:
> > > > > >
> > > > > > > +1 ListenableFuture: If this works similar to Deferreds in
> > Twisted
> > > > > Python
> > > > > > > or Promised IO in Javascript, I think this is a great pattern
> for
> > > > > > > decoupling your callback logic from the place where the Future
> is
> > > > > > > generated.  You can register as many callbacks as you like,
> each
> > in
> > > > the
> > > > > > > appropriate layer of the code and have each observer get
> notified
> > > > when
> > > > > > the
> > > > > > > promised i/o is complete without any of them knowing about each
> > > > other.
> > > > > > >
> > > > > > >
> > > > > > > On Tue, Jan 28, 2014 at 11:32 AM, Jay Kreps <
> jay.kreps@gmail.com
> > >
> > > > > wrote:
> > > > > > >
> > > > > > > > Hey Ross,
> > > > > > > >
> > > > > > > > - ListenableFuture: Interesting. That would be an alternative
> > to
> > > > the
> > > > > > > direct
> > > > > > > > callback support we provide. There could be pros to this, let
> > me
> > > > > think
> > > > > > > > about it.
> > > > > > > > - We could provide layering, but I feel that the
> serialization
> > is
> > > > > such
> > > > > > a
> > > > > > > > small thing we should just make a decision and chose one, it
> > > > doesn't
> > > > > > seem
> > > > > > > > to me to justify a whole public facing layer.
> > > > > > > > - Yes, this is fairly esoteric, essentially I think it is
> > fairly
> > > > > > similar
> > > > > > > to
> > > > > > > > databases like DynamoDB that allow you to specify two
> partition
> > > > keys
> > > > > (I
> > > > > > > > think DynamoDB does this...). The reasoning is that in fact
> > there
> > > > are
> > > > > > > > several things you can use the key field for: (1) to compute
> > the
> > > > > > > partition
> > > > > > > > to store the data in, (2) as a unique identifier to
> deduplicate
> > > > that
> > > > > > > > partition's records within a log. These two things are almost
> > > > always
> > > > > > the
> > > > > > > > same, but occationally may differ when you want to group data
> > in
> > > a
> > > > > more
> > > > > > > > sophisticated way then just a hash of the primary key but
> still
> > > > > retain
> > > > > > > the
> > > > > > > > proper primary key for delivery to the consumer and log
> > > compaction.
> > > > > > > >
> > > > > > > > -Jay
> > > > > > > >
> > > > > > > >
> > > > > > > > On Tue, Jan 28, 2014 at 3:24 AM, Ross Black <
> > > > ross.w.black@gmail.com>
> > > > > > > > wrote:
> > > > > > > >
> > > > > > > > > Hi Jay,
> > > > > > > > >
> > > > > > > > > - Just to add some more info/confusion about possibly using
> > > > Future
> > > > > > ...
> > > > > > > > >   If Kafka uses a JDK future, it plays nicely with other
> > > > frameworks
> > > > > > as
> > > > > > > > > well.
> > > > > > > > >   Google Guava has a ListenableFuture that allows callback
> > > > handling
> > > > > > to
> > > > > > > be
> > > > > > > > > added via the returned future, and allows the callbacks to
> be
> > > > > passed
> > > > > > > off
> > > > > > > > to
> > > > > > > > > a specified executor.
> > > > > > > > >
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> http://docs.guava-libraries.googlecode.com/git-history/release/javadoc/com/google/common/util/concurrent/ListenableFuture.html
> > > > > > > > >   The JDK future can easily be converted to a listenable
> > > future.
> > > > > > > > >
> > > > > > > > > - On the question of byte[] vs Object, could this be solved
> > by
> > > > > > layering
> > > > > > > > the
> > > > > > > > > API?  eg. a raw producer (use byte[] and specify the
> > partition
> > > > > > number)
> > > > > > > > and
> > > > > > > > > a normal producer (use generic object and specify a
> > > Partitioner)?
> > > > > > > > >
> > > > > > > > > - I am confused by the keys in ProducerRecord and
> > Partitioner.
> > > > >  What
> > > > > > is
> > > > > > > > the
> > > > > > > > > usage for both a key and a partition key? (I am not yet
> using
> > > > 0.8)
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > Thanks,
> > > > > > > > > Ross
> > > > > > > > >
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > On 28 January 2014 05:00, Xavier Stevens <
> xavier@gaikai.com>
> > > > > wrote:
> > > > > > > > >
> > > > > > > > > > AutoCloseable would be nice for us as most of our code is
> > > using
> > > > > > Java
> > > > > > > 7
> > > > > > > > at
> > > > > > > > > > this point.
> > > > > > > > > >
> > > > > > > > > > I like Dropwizard's configuration mapping to POJOs via
> > > Jackson,
> > > > > but
> > > > > > > if
> > > > > > > > > you
> > > > > > > > > > wanted to stick with property maps I don't care enough to
> > > > object.
> > > > > > > > > >
> > > > > > > > > > If the producer only dealt with bytes, is there a way we
> > > could
> > > > > > still
> > > > > > > > due
> > > > > > > > > > partition plugins without specifying the number
> > explicitly? I
> > > > > would
> > > > > > > > > prefer
> > > > > > > > > > to be able to pass in field(s) that would be used by the
> > > > > > partitioner.
> > > > > > > > > > Obviously if this wasn't possible you could always
> > > deserialize
> > > > > the
> > > > > > > > object
> > > > > > > > > > in the partitioner and grab the fields you want, but that
> > > seems
> > > > > > > really
> > > > > > > > > > expensive to do on every message.
> > > > > > > > > >
> > > > > > > > > > It would also be nice to have a Java API Encoder
> > constructor
> > > > > taking
> > > > > > > in
> > > > > > > > > > VerifiableProperties. Scala understands how to handle
> > "props:
> > > > > > > > > > VerifiableProperties = null", but Java doesn't. So you
> > don't
> > > > run
> > > > > > into
> > > > > > > > > this
> > > > > > > > > > problem until runtime.
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > -Xavier
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > On Mon, Jan 27, 2014 at 9:37 AM, Clark Breyman <
> > > > > clark@breyman.com>
> > > > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > Jay -
> > > > > > > > > > >
> > > > > > > > > > > Config - your explanation makes sense. I'm just so
> > > accustomed
> > > > > to
> > > > > > > > having
> > > > > > > > > > > Jackson automatically map my configuration objects to
> > POJOs
> > > > > that
> > > > > > > I've
> > > > > > > > > > > stopped using property files. They are lingua franca.
> The
> > > > only
> > > > > > > > thought
> > > > > > > > > > > might be to separate the config interface from the
> > > > > implementation
> > > > > > > to
> > > > > > > > > > allow
> > > > > > > > > > > for alternatives, but that might undermine your point
> of
> > > "do
> > > > it
> > > > > > > this
> > > > > > > > > way
> > > > > > > > > > so
> > > > > > > > > > > that everyone can find it where they expect it".
> > > > > > > > > > >
> > > > > > > > > > > Serialization: Of the options, I like 1A the best,
> though
> > > > > > possibly
> > > > > > > > with
> > > > > > > > > > > either an option to specify a partition key rather than
> > ID
> > > > or a
> > > > > > > > helper
> > > > > > > > > to
> > > > > > > > > > > translate an arbitrary byte[] or long into a partition
> > > > number.
> > > > > > > > > > >
> > > > > > > > > > > Thanks
> > > > > > > > > > > Clark
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > On Sun, Jan 26, 2014 at 9:13 PM, Jay Kreps <
> > > > > jay.kreps@gmail.com>
> > > > > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > Thanks for the detailed thoughts. Let me elaborate on
> > the
> > > > > > config
> > > > > > > > > thing.
> > > > > > > > > > > >
> > > > > > > > > > > > I agree that at first glance key-value strings don't
> > seem
> > > > > like
> > > > > > a
> > > > > > > > very
> > > > > > > > > > > good
> > > > > > > > > > > > configuration api for a client. Surely a well-typed
> > > config
> > > > > > class
> > > > > > > > > would
> > > > > > > > > > be
> > > > > > > > > > > > better! I actually disagree and let me see if I can
> > > > convince
> > > > > > you.
> > > > > > > > > > > >
> > > > > > > > > > > > My reasoning has nothing to do with the api and
> > > everything
> > > > to
> > > > > > do
> > > > > > > > with
> > > > > > > > > > > > operations.
> > > > > > > > > > > >
> > > > > > > > > > > > Clients are embedded in applications which are
> > themselves
> > > > > > > > configured.
> > > > > > > > > > In
> > > > > > > > > > > > any place that takes operations seriously the
> > > configuration
> > > > > for
> > > > > > > > these
> > > > > > > > > > > > applications will be version controlled and
> maintained
> > > > > through
> > > > > > > some
> > > > > > > > > > kind
> > > > > > > > > > > of
> > > > > > > > > > > > config management system. If we give a config class
> > with
> > > > > > getters
> > > > > > > > and
> > > > > > > > > > > > setters the application has to expose those
> properties
> > to
> > > > its
> > > > > > > > > > > > configuration. What invariably happens is that the
> > > > > application
> > > > > > > > > exposes
> > > > > > > > > > > only
> > > > > > > > > > > > a choice few properties that they thought they would
> > > > change.
> > > > > > > > > > Furthermore
> > > > > > > > > > > > the application will make up a name for these configs
> > > that
> > > > > > seems
> > > > > > > > > > > intuitive
> > > > > > > > > > > > at the time in the 2 seconds the engineer spends
> > thinking
> > > > > about
> > > > > > > it.
> > > > > > > > > > > >
> > > > > > > > > > > > Now consider the result of this in the large. You end
> > up
> > > > with
> > > > > > > > dozens
> > > > > > > > > or
> > > > > > > > > > > > hundreds of applications that have the client
> embedded.
> > > > Each
> > > > > > > > exposes
> > > > > > > > > a
> > > > > > > > > > > > different, inadequate subset of the possible configs,
> > > each
> > > > > with
> > > > > > > > > > different
> > > > > > > > > > > > names. It is a nightmare.
> > > > > > > > > > > >
> > > > > > > > > > > > If you use a string-string map the config system can
> > > > directly
> > > > > > > get a
> > > > > > > > > > > bundle
> > > > > > > > > > > > of config key-value pairs and put them into the
> client.
> > > > This
> > > > > > > means
> > > > > > > > > that
> > > > > > > > > > > all
> > > > > > > > > > > > configuration is automatically available with the
> name
> > > > > > documented
> > > > > > > > on
> > > > > > > > > > the
> > > > > > > > > > > > website in every application that does this. If you
> > > upgrade
> > > > > to
> > > > > > a
> > > > > > > > new
> > > > > > > > > > > kafka
> > > > > > > > > > > > version with more configs those will be exposed too.
> If
> > > you
> > > > > > > realize
> > > > > > > > > > that
> > > > > > > > > > > > you need to change a default you can just go through
> > your
> > > > > > configs
> > > > > > > > and
> > > > > > > > > > > > change it everywhere as it will have the same name
> > > > > everywhere.
> > > > > > > > > > > >
> > > > > > > > > > > > -Jay
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > On Sun, Jan 26, 2014 at 4:47 PM, Clark Breyman <
> > > > > > > clark@breyman.com>
> > > > > > > > > > > wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > > Thanks Jay. I'll see if I can put together a more
> > > > complete
> > > > > > > > > response,
> > > > > > > > > > > > > perhaps as separate threads so that topics don't
> get
> > > > > > entangled.
> > > > > > > > In
> > > > > > > > > > the
> > > > > > > > > > > > mean
> > > > > > > > > > > > > time, here's a couple responses:
> > > > > > > > > > > > >
> > > > > > > > > > > > > Serialization: you've broken out a sub-thread so
> i'll
> > > > reply
> > > > > > > > there.
> > > > > > > > > My
> > > > > > > > > > > > bias
> > > > > > > > > > > > > is that I like generics (except for type-erasure)
> and
> > > in
> > > > > > > > particular
> > > > > > > > > > > they
> > > > > > > > > > > > > make it easy to compose serializers for compound
> > > payloads
> > > > > > (e.g.
> > > > > > > > > when
> > > > > > > > > > a
> > > > > > > > > > > > > common header wraps a payload of parameterized
> type).
> > > > I'll
> > > > > > > > respond
> > > > > > > > > to
> > > > > > > > > > > > your
> > > > > > > > > > > > > 4-options message with an example.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Build: I've seen a lot of "maven-compatible" build
> > > > systems
> > > > > > > > produce
> > > > > > > > > > > > > "artifacts" that aren't really artifacts - no
> > embedded
> > > > POM
> > > > > > or,
> > > > > > > > > worst,
> > > > > > > > > > > > > malformed POM. I know the sbt-generated artifacts
> > were
> > > > this
> > > > > > > way -
> > > > > > > > > > onus
> > > > > > > > > > > is
> > > > > > > > > > > > > on me to see what gradle is spitting out and what a
> > > maven
> > > > > > build
> > > > > > > > > might
> > > > > > > > > > > > look
> > > > > > > > > > > > > like. Maven may be old and boring, but it gets out
> of
> > > the
> > > > > way
> > > > > > > and
> > > > > > > > > > > > > integrates really seamlessly with a lot of IDEs.
> When
> > > > some
> > > > > > > scala
> > > > > > > > > > > > projects I
> > > > > > > > > > > > > was working on in the fall of 2011 switched from
> sbt
> > to
> > > > > > maven,
> > > > > > > > > build
> > > > > > > > > > > > became
> > > > > > > > > > > > > a non-issue.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Config: Not a big deal  and no, I don't think a
> > > > dropwizard
> > > > > > > > > dependency
> > > > > > > > > > > is
> > > > > > > > > > > > > appropriate. I do like using simple entity beans
> > > (POJO's
> > > > > not
> > > > > > > > j2EE)
> > > > > > > > > > for
> > > > > > > > > > > > > configuration, especially if they can be marshalled
> > > > without
> > > > > > > > > > annotation
> > > > > > > > > > > by
> > > > > > > > > > > > > Jackson. I only mentioned the dropwizard-extras
> > >  because
> > > > it
> > > > > > has
> > > > > > > > > some
> > > > > > > > > > > > entity
> > > > > > > > > > > > > bean versions of the ZK and Kafka configs.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Domain-packaging: Also not a big deal - it's what's
> > > > > expected
> > > > > > > and
> > > > > > > > > it's
> > > > > > > > > > > > > pretty free in most IDE's. The advantages I see is
> > that
> > > > it
> > > > > is
> > > > > > > > clear
> > > > > > > > > > > > whether
> > > > > > > > > > > > > something is from the Apache Kafka project and
> > whether
> > > > > > > something
> > > > > > > > is
> > > > > > > > > > > from
> > > > > > > > > > > > > another org and related to Kafka. That said,
> nothing
> > > > really
> > > > > > > > > enforces
> > > > > > > > > > > it.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Futures: I'll see if I can create some examples to
> > > > > > demonstrate
> > > > > > > > > Future
> > > > > > > > > > > > > making interop easier.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Regards,
> > > > > > > > > > > > > C
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Fri, Jan 24, 2014 at 4:36 PM, Jay Kreps <
> > > > > > > jay.kreps@gmail.com>
> > > > > > > > > > > wrote:
> > > > > > > > > > > > >
> > > > > > > > > > > > > > Hey Clark,
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > - Serialization: Yes I agree with these though I
> > > don't
> > > > > > > consider
> > > > > > > > > the
> > > > > > > > > > > > loss
> > > > > > > > > > > > > of
> > > > > > > > > > > > > > generics a big issue. I'll try to summarize what
> I
> > > > would
> > > > > > > > consider
> > > > > > > > > > the
> > > > > > > > > > > > > best
> > > > > > > > > > > > > > alternative api with raw byte[].
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > - Maven: We had this debate a few months back and
> > the
> > > > > > > consensus
> > > > > > > > > was
> > > > > > > > > > > > > gradle.
> > > > > > > > > > > > > > Is there a specific issue with the poms gradle
> > > makes? I
> > > > > am
> > > > > > > > > > extremely
> > > > > > > > > > > > > loath
> > > > > > > > > > > > > > to revisit the issue as build issues are a
> > recurring
> > > > > thing
> > > > > > > and
> > > > > > > > no
> > > > > > > > > > one
> > > > > > > > > > > > > ever
> > > > > > > > > > > > > > agrees and ultimately our build needs are very
> > > simple.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > - Config: I'm not sure if I follow the point. Are
> > you
> > > > > > saying
> > > > > > > we
> > > > > > > > > > > should
> > > > > > > > > > > > > use
> > > > > > > > > > > > > > something in dropwizard for config? One principle
> > > here
> > > > is
> > > > > > to
> > > > > > > > try
> > > > > > > > > to
> > > > > > > > > > > > > remove
> > > > > > > > > > > > > > as many client dependencies as possible as we
> > > > inevitably
> > > > > > run
> > > > > > > > into
> > > > > > > > > > > > > terrible
> > > > > > > > > > > > > > compatibility issues with users who use the same
> > > > library
> > > > > or
> > > > > > > its
> > > > > > > > > > > > > > dependencies at different versions. Or are you
> > > talking
> > > > > > about
> > > > > > > > > > > > maintaining
> > > > > > > > > > > > > > compatibility with existing config parameters? I
> > > think
> > > > as
> > > > > > > much
> > > > > > > > > as a
> > > > > > > > > > > > > config
> > > > > > > > > > > > > > in the existing client makes sense it should have
> > the
> > > > > same
> > > > > > > name
> > > > > > > > > (I
> > > > > > > > > > > was
> > > > > > > > > > > > a
> > > > > > > > > > > > > > bit sloppy about that so I'll fix any errors
> > there).
> > > > > There
> > > > > > > are
> > > > > > > > a
> > > > > > > > > > few
> > > > > > > > > > > > new
> > > > > > > > > > > > > > things and we should give those reasonable
> > defaults.
> > > I
> > > > > > think
> > > > > > > > > config
> > > > > > > > > > > is
> > > > > > > > > > > > > > important so I'll start a thread on the config
> > > package
> > > > in
> > > > > > > > there.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > - org.apache.kafka: We could do this. I always
> > > > considered
> > > > > > it
> > > > > > > > kind
> > > > > > > > > > of
> > > > > > > > > > > an
> > > > > > > > > > > > > odd
> > > > > > > > > > > > > > thing Java programmers do that has no real
> > motivation
> > > > > (but
> > > > > > I
> > > > > > > > > could
> > > > > > > > > > be
> > > > > > > > > > > > > > re-educated!). I don't think it ends up reducing
> > > naming
> > > > > > > > conflicts
> > > > > > > > > > in
> > > > > > > > > > > > > > practice and it adds a lot of noise and nested
> > > > > directories.
> > > > > > > Is
> > > > > > > > > > there
> > > > > > > > > > > a
> > > > > > > > > > > > > > reason you prefer this or just to be more
> standard?
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > - Future: Basically I didn't see any particular
> > > > > advantage.
> > > > > > > The
> > > > > > > > > > > cancel()
> > > > > > > > > > > > > > method doesn't really make sense so probably
> > wouldn't
> > > > > work.
> > > > > > > > > > Likewise
> > > > > > > > > > > I
> > > > > > > > > > > > > > dislike the checked exceptions it requires.
> > > Basically I
> > > > > > just
> > > > > > > > > wrote
> > > > > > > > > > > out
> > > > > > > > > > > > > some
> > > > > > > > > > > > > > code examples and it seemed cleaner with a
> special
> > > > > purpose
> > > > > > > > > object.
> > > > > > > > > > I
> > > > > > > > > > > > > wasn't
> > > > > > > > > > > > > > actually aware of plans for improved futures in
> > java
> > > 8
> > > > or
> > > > > > the
> > > > > > > > > other
> > > > > > > > > > > > > > integrations. Maybe you could elaborate on this a
> > bit
> > > > and
> > > > > > > show
> > > > > > > > > how
> > > > > > > > > > it
> > > > > > > > > > > > > would
> > > > > > > > > > > > > > be used? Sounds promising, I just don't know a
> lot
> > > > about
> > > > > > it.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > -Jay
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On Fri, Jan 24, 2014 at 3:30 PM, Clark Breyman <
> > > > > > > > > clark@breyman.com>
> > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Jay - Thanks for the call for comments. Here's
> > some
> > > > > > initial
> > > > > > > > > > input:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > - Make message serialization a client
> > > responsibility
> > > > > > > (making
> > > > > > > > > all
> > > > > > > > > > > > > messages
> > > > > > > > > > > > > > > byte[]). Reflection-based loading makes it
> harder
> > > to
> > > > > use
> > > > > > > > > generic
> > > > > > > > > > > > codecs
> > > > > > > > > > > > > > > (e.g.  Envelope<PREFIX, DATA, SUFFIX>) or build
> > up
> > > > > codec
> > > > > > > > > > > > > > programmatically.
> > > > > > > > > > > > > > > Non-default partitioning should require an
> > explicit
> > > > > > > partition
> > > > > > > > > > key.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > - I really like the fact that it will be native
> > > Java.
> > > > > > > Please
> > > > > > > > > > > consider
> > > > > > > > > > > > > > using
> > > > > > > > > > > > > > > native maven and not sbt, gradle, ivy, etc as
> > they
> > > > > don't
> > > > > > > > > reliably
> > > > > > > > > > > > play
> > > > > > > > > > > > > > nice
> > > > > > > > > > > > > > > in the maven ecosystem. A jar without a
> > well-formed
> > > > pom
> > > > > > > > doesn't
> > > > > > > > > > > feel
> > > > > > > > > > > > > > like a
> > > > > > > > > > > > > > > real artifact. The pom's generated by sbt et
> al.
> > > are
> > > > > not
> > > > > > > well
> > > > > > > > > > > formed.
> > > > > > > > > > > > > > Using
> > > > > > > > > > > > > > > maven will make builds and IDE integration much
> > > > > smoother.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > - Look at Nick Telford's dropwizard-extras
> > package
> > > in
> > > > > > which
> > > > > > > > he
> > > > > > > > > > > > defines
> > > > > > > > > > > > > > some
> > > > > > > > > > > > > > > Jackson-compatible POJO's for loading
> > > configuration.
> > > > > > Seems
> > > > > > > > like
> > > > > > > > > > > your
> > > > > > > > > > > > > > client
> > > > > > > > > > > > > > > migration is similar. The config objects should
> > > have
> > > > > > > > > constructors
> > > > > > > > > > > or
> > > > > > > > > > > > > > > factories that accept Map<String, String> and
> > > > > Properties
> > > > > > > for
> > > > > > > > > ease
> > > > > > > > > > > of
> > > > > > > > > > > > > > > migration.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > - Would you consider using the org.apache.kafka
> > > > package
> > > > > > for
> > > > > > > > the
> > > > > > > > > > new
> > > > > > > > > > > > API
> > > > > > > > > > > > > > > (quibble)
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > - Why create your own futures rather than use
> > > > > > > > > > > > > > > java.util.concurrent.Future<Long> or similar?
> > > > Standard
> > > > > > > > futures
> > > > > > > > > > will
> > > > > > > > > > > > > play
> > > > > > > > > > > > > > > nice with other reactive libs and things like
> > J8's
> > > > > > > > > > > ComposableFuture.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Thanks again,
> > > > > > > > > > > > > > > C
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > On Fri, Jan 24, 2014 at 2:46 PM, Roger Hoover <
> > > > > > > > > > > > roger.hoover@gmail.com
> > > > > > > > > > > > > > > >wrote:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > A couple comments:
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > 1) Why does the config use a broker list
> > instead
> > > of
> > > > > > > > > discovering
> > > > > > > > > > > the
> > > > > > > > > > > > > > > brokers
> > > > > > > > > > > > > > > > in ZooKeeper?  It doesn't match the
> > > > HighLevelConsumer
> > > > > > > API.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > 2) It looks like broker connections are
> created
> > > on
> > > > > > > demand.
> > > > > > > > >  I'm
> > > > > > > > > > > > > > wondering
> > > > > > > > > > > > > > > > if sometimes you might want to flush out
> config
> > > or
> > > > > > > network
> > > > > > > > > > > > > connectivity
> > > > > > > > > > > > > > > > issues before pushing the first message
> > through.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Should there also be a
> KafkaProducer.connect()
> > or
> > > > > > .open()
> > > > > > > > > > method
> > > > > > > > > > > or
> > > > > > > > > > > > > > > > connectAll()?  I guess it would try to
> connect
> > to
> > > > all
> > > > > > > > brokers
> > > > > > > > > > in
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > BROKER_LIST_CONFIG
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > HTH,
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Roger
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > On Fri, Jan 24, 2014 at 11:54 AM, Jay Kreps <
> > > > > > > > > > jay.kreps@gmail.com
> > > > > > > > > > > >
> > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > As mentioned in a previous email we are
> > working
> > > > on
> > > > > a
> > > > > > > > > > > > > > re-implementation
> > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > the producer. I would like to use this
> email
> > > > thread
> > > > > > to
> > > > > > > > > > discuss
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > details
> > > > > > > > > > > > > > > > > of the public API and the configuration. I
> > > would
> > > > > love
> > > > > > > for
> > > > > > > > > us
> > > > > > > > > > to
> > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > incredibly picky about this public api now
> so
> > > it
> > > > is
> > > > > > as
> > > > > > > > good
> > > > > > > > > > as
> > > > > > > > > > > > > > possible
> > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > we don't need to break it in the future.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > The best way to get a feel for the API is
> > > > actually
> > > > > to
> > > > > > > > take
> > > > > > > > > a
> > > > > > > > > > > look
> > > > > > > > > > > > > at
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > javadoc, my hope is to get the api docs
> good
> > > > enough
> > > > > > so
> > > > > > > > that
> > > > > > > > > > it
> > > > > > > > > > > is
> > > > > > > > > > > > > > > > > self-explanatory:
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> http://empathybox.com/kafka-javadoc/index.html?kafka/clients/producer/KafkaProducer.html
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Please take a look at this API and give me
> > any
> > > > > > thoughts
> > > > > > > > you
> > > > > > > > > > may
> > > > > > > > > > > > > have!
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > It may also be reasonable to take a look at
> > the
> > > > > > > configs:
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> http://empathybox.com/kafka-javadoc/kafka/clients/producer/ProducerConfig.html
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > The actual code is posted here:
> > > > > > > > > > > > > > > > >
> > > https://issues.apache.org/jira/browse/KAFKA-1227
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > A few questions or comments to kick things
> > off:
> > > > > > > > > > > > > > > > > 1. We need to make a decision on whether
> > > > > > serialization
> > > > > > > of
> > > > > > > > > the
> > > > > > > > > > > > > user's
> > > > > > > > > > > > > > > key
> > > > > > > > > > > > > > > > > and value should be done by the user (with
> > our
> > > > api
> > > > > > just
> > > > > > > > > > taking
> > > > > > > > > > > > > > byte[])
> > > > > > > > > > > > > > > or
> > > > > > > > > > > > > > > > > if we should take an object and allow the
> > user
> > > to
> > > > > > > > > configure a
> > > > > > > > > > > > > > > Serializer
> > > > > > > > > > > > > > > > > class which we instantiate via reflection.
> We
> > > > take
> > > > > > the
> > > > > > > > > later
> > > > > > > > > > > > > approach
> > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > the current producer, and I have carried
> this
> > > > > through
> > > > > > > to
> > > > > > > > > this
> > > > > > > > > > > > > > > prototype.
> > > > > > > > > > > > > > > > > The tradeoff I see is this: taking byte[]
> is
> > > > > actually
> > > > > > > > > > simpler,
> > > > > > > > > > > > the
> > > > > > > > > > > > > > user
> > > > > > > > > > > > > > > > can
> > > > > > > > > > > > > > > > > directly do whatever serialization they
> like.
> > > The
> > > > > > > > > > complication
> > > > > > > > > > > is
> > > > > > > > > > > > > > > > actually
> > > > > > > > > > > > > > > > > partitioning. Currently partitioning is
> done
> > > by a
> > > > > > > similar
> > > > > > > > > > > plug-in
> > > > > > > > > > > > > api
> > > > > > > > > > > > > > > > > (Partitioner) which the user can implement
> > and
> > > > > > > configure
> > > > > > > > to
> > > > > > > > > > > > > override
> > > > > > > > > > > > > > > how
> > > > > > > > > > > > > > > > > partitions are assigned. If we take byte[]
> as
> > > > input
> > > > > > > then
> > > > > > > > we
> > > > > > > > > > > have
> > > > > > > > > > > > no
> > > > > > > > > > > > > > > > access
> > > > > > > > > > > > > > > > > to the original object and partitioning
> MUST
> > be
> > > > > done
> > > > > > on
> > > > > > > > the
> > > > > > > > > > > > byte[].
> > > > > > > > > > > > > > > This
> > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > fine for hash partitioning. However for
> > various
> > > > > types
> > > > > > > of
> > > > > > > > > > > semantic
> > > > > > > > > > > > > > > > > partitioning (range partitioning, or
> > whatever)
> > > > you
> > > > > > > would
> > > > > > > > > want
> > > > > > > > > > > > > access
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > original object. In the current approach a
> > > > producer
> > > > > > who
> > > > > > > > > > wishes
> > > > > > > > > > > to
> > > > > > > > > > > > > > send
> > > > > > > > > > > > > > > > > byte[] they have serialized in their own
> code
> > > can
> > > > > > > > configure
> > > > > > > > > > the
> > > > > > > > > > > > > > > > > BytesSerialization we supply which is just
> a
> > > "no
> > > > > op"
> > > > > > > > > > > > serialization.
> > > > > > > > > > > > > > > > > 2. We should obsess over naming and make
> sure
> > > > each
> > > > > of
> > > > > > > the
> > > > > > > > > > class
> > > > > > > > > > > > > names
> > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > good.
> > > > > > > > > > > > > > > > > 3. Jun has already pointed out that we need
> > to
> > > > > > include
> > > > > > > > the
> > > > > > > > > > > topic
> > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > partition in the response, which is
> > absolutely
> > > > > > right. I
> > > > > > > > > > haven't
> > > > > > > > > > > > > done
> > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > yet but that definitely needs to be there.
> > > > > > > > > > > > > > > > > 4. Currently RecordSend.await will throw an
> > > > > exception
> > > > > > > if
> > > > > > > > > the
> > > > > > > > > > > > > request
> > > > > > > > > > > > > > > > > failed. The intention here is that
> > > > > > > > > > > producer.send(message).await()
> > > > > > > > > > > > > > > exactly
> > > > > > > > > > > > > > > > > simulates a synchronous call. Guozhang has
> > > noted
> > > > > that
> > > > > > > > this
> > > > > > > > > > is a
> > > > > > > > > > > > > > little
> > > > > > > > > > > > > > > > > annoying since the user must then catch
> > > > exceptions.
> > > > > > > > However
> > > > > > > > > > if
> > > > > > > > > > > we
> > > > > > > > > > > > > > > remove
> > > > > > > > > > > > > > > > > this then if the user doesn't check for
> > errors
> > > > they
> > > > > > > won't
> > > > > > > > > > know
> > > > > > > > > > > > one
> > > > > > > > > > > > > > has
> > > > > > > > > > > > > > > > > occurred, which I predict will be a common
> > > > mistake.
> > > > > > > > > > > > > > > > > 5. Perhaps there is more we could do to
> make
> > > the
> > > > > > async
> > > > > > > > > > > callbacks
> > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > future
> > > > > > > > > > > > > > > > > we give back intuitive and easy to program
> > > > against?
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Some background info on implementation:
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > At a high level the primary difference in
> > this
> > > > > > producer
> > > > > > > > is
> > > > > > > > > > that
> > > > > > > > > > > > it
> > > > > > > > > > > > > > > > removes
> > > > > > > > > > > > > > > > > the distinction between the "sync" and
> > "async"
> > > > > > > producer.
> > > > > > > > > > > > > Effectively
> > > > > > > > > > > > > > > all
> > > > > > > > > > > > > > > > > requests are sent asynchronously but always
> > > > return
> > > > > a
> > > > > > > > future
> > > > > > > > > > > > > response
> > > > > > > > > > > > > > > > object
> > > > > > > > > > > > > > > > > that gives the offset as well as any error
> > that
> > > > may
> > > > > > > have
> > > > > > > > > > > occurred
> > > > > > > > > > > > > > when
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > request is complete. The batching that is
> > done
> > > in
> > > > > the
> > > > > > > > async
> > > > > > > > > > > > > producer
> > > > > > > > > > > > > > > only
> > > > > > > > > > > > > > > > > today is done whenever possible now. This
> > means
> > > > > that
> > > > > > > the
> > > > > > > > > sync
> > > > > > > > > > > > > > producer,
> > > > > > > > > > > > > > > > > under load, can get performance as good as
> > the
> > > > > async
> > > > > > > > > producer
> > > > > > > > > > > > > > > > (preliminary
> > > > > > > > > > > > > > > > > results show the producer getting 1m
> > > > messages/sec).
> > > > > > > This
> > > > > > > > > > works
> > > > > > > > > > > > > > similar
> > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > group commit in databases but with respect
> to
> > > the
> > > > > > > actual
> > > > > > > > > > > network
> > > > > > > > > > > > > > > > > transmission--any messages that arrive
> while
> > a
> > > > send
> > > > > > is
> > > > > > > in
> > > > > > > > > > > > progress
> > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > batched together. It is also possible to
> > > > encourage
> > > > > > > > batching
> > > > > > > > > > > even
> > > > > > > > > > > > > > under
> > > > > > > > > > > > > > > > low
> > > > > > > > > > > > > > > > > load to save server resources by
> introducing
> > a
> > > > > delay
> > > > > > on
> > > > > > > > the
> > > > > > > > > > > send
> > > > > > > > > > > > to
> > > > > > > > > > > > > > > allow
> > > > > > > > > > > > > > > > > more messages to accumulate; this is done
> > using
> > > > the
> > > > > > > > > > > > linger.msconfig
> > > > > > > > > > > > > > > > (this
> > > > > > > > > > > > > > > > > is similar to Nagle's algorithm in TCP).
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > This producer does all network
> communication
> > > > > > > > asynchronously
> > > > > > > > > > and
> > > > > > > > > > > > in
> > > > > > > > > > > > > > > > parallel
> > > > > > > > > > > > > > > > > to all servers so the performance penalty
> for
> > > > > acks=-1
> > > > > > > and
> > > > > > > > > > > waiting
> > > > > > > > > > > > > on
> > > > > > > > > > > > > > > > > replication should be much reduced. I
> haven't
> > > > done
> > > > > > much
> > > > > > > > > > > > > benchmarking
> > > > > > > > > > > > > > on
> > > > > > > > > > > > > > > > > this yet, though.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > The high level design is described a little
> > > here,
> > > > > > > though
> > > > > > > > > this
> > > > > > > > > > > is
> > > > > > > > > > > > > now
> > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > little out of date:
> > > > > > > > > > > > > > > > >
> > > > > > > > > > >
> > > > > https://cwiki.apache.org/confluence/display/KAFKA/Client+Rewrite
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > -Jay
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: New Producer Public API

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

So is there one connection and I/O thread per broker and a low-level client
for each of those, and then you hash into that to partition? Is it possible
to batch across partitions or only within a partition?

-Jay


On Wed, Jan 29, 2014 at 8:41 AM, Tom Brown <to...@gmail.com> wrote:

> Jay,
>
> There is both a basic client object, and a number of IO processing threads.
> The client object manages connections, creating new ones when new machines
> are connected, or when existing connections die. It also manages a queue of
> requests for each server. The IO processing thread has a selector, and
> performs the work of sending/receiving (removing items from the queue,
> interpreting the response at a basic level, etc). Since asynchronous
> sockets by nature decouple sending and receiving, request pipelining is
> inherent.
>
> Using the basic client, you can send individual produce requests (singular
> or batched). The "producer" layer adds an additional queue for each
> partition, allowing individual messages to be batched together.
>
> --Tom
>
>
>
> On Tue, Jan 28, 2014 at 9:10 PM, Jay Kreps <ja...@gmail.com> wrote:
>
> > Hey Neha,
> >
> > Can you elaborate on why you prefer using Java's Future? The downside in
> my
> > mind is the use of the checked InterruptedException and
> ExecutionException.
> > ExecutionException is arguable, but forcing you to catch
> > InterruptedException, often in code that can't be interrupted, seems
> > perverse. It also leaves us with the cancel() method which I don't think
> we
> > really can implement.
> >
> > Option 1A, to recap/elaborate, was the following. There is no Serializer
> or
> > Partitioner api. We take a byte[] key and value and an optional integer
> > partition. If you specify the integer partition it will be used. If you
> do
> > not specify a key or a partition the partition will be chosen in a round
> > robin fashion. If you specify a key but no partition we will chose a
> > partition based on a hash of the key. In order to let the user find the
> > partition we will need to given them access to the Cluster instance
> > directly from the producer.
> >
> > -Jay
> >
> >
> > On Tue, Jan 28, 2014 at 6:25 PM, Neha Narkhede <neha.narkhede@gmail.com
> > >wrote:
> >
> > > Here are more thoughts on the public APIs -
> > >
> > > - I suggest we use java's Future instead of custom Future especially
> > since
> > > it is part of the public API
> > >
> > > - Serialization: I like the simplicity of the producer APIs with the
> > > absence of serialization where we just deal with byte arrays for keys
> and
> > > values. What I don't like about this is the performance overhead on the
> > > Partitioner for any kind of custom partitioning based on the
> > partitionKey.
> > > Since the only purpose of partitionKey is to do custom partitioning,
> why
> > > can't we take it in directly as an integer and let the user figure out
> > the
> > > mapping from partition_key -> partition_id using the getCluster() API?
> > If I
> > > understand correctly, this is similar to what you suggested as part of
> > > option 1A. I like this approach since it maintains the simplicity of
> APIs
> > > by allowing us to deal with bytes and does not compromise performance
> in
> > > the custom partitioning case.
> > >
> > > Thanks,
> > > Neha
> > >
> > >
> > >
> > > On Tue, Jan 28, 2014 at 5:42 PM, Jay Kreps <ja...@gmail.com>
> wrote:
> > >
> > > > Hey Tom,
> > > >
> > > > That sounds cool. How did you end up handling parallel I/O if you
> wrap
> > > the
> > > > individual connections? Don't you need some selector that selects
> over
> > > all
> > > > the connections?
> > > >
> > > > -Jay
> > > >
> > > >
> > > > On Tue, Jan 28, 2014 at 2:31 PM, Tom Brown <to...@gmail.com>
> > wrote:
> > > >
> > > > > I implemented a 0.7 client in pure java, and its API very closely
> > > > resembled
> > > > > this. (When multiple people independently engineer the same
> solution,
> > > > it's
> > > > > probably good... right?). However, there were a few architectural
> > > > > differences with my client:
> > > > >
> > > > > 1. The basic client itself was just an asynchronous layer around
> the
> > > > > different server functions. In and of itself it had no knowledge of
> > > > > partitions, only servers (and maintained TCP connections to them).
> > > > >
> > > > > 2. The main producer was an additional layer that provided a
> > high-level
> > > > > interface that could batch individual messages based on partition.
> > > > >
> > > > > 3. Knowledge of partitioning was done via an interface so that
> > > different
> > > > > strategies could be used.
> > > > >
> > > > > 4. Partitioning was done by the user, with knowledge of the
> available
> > > > > partitions provided by #3.
> > > > >
> > > > > 5. Serialization was done by the user to simplify the API.
> > > > >
> > > > > 6. Futures were used to make asynchronous emulate synchronous
> calls.
> > > > >
> > > > >
> > > > > The main benefit of this approach is flexibility. For example,
> since
> > > the
> > > > > base client was just a managed connection (and not inherently a
> > > > producer),
> > > > > it was easy to composite a produce request and an offsets request
> > > > together
> > > > > into a confirmed produce request (officially not available in 0.7).
> > > > >
> > > > > Decoupling the basic client from partition management allowed the
> me
> > to
> > > > > implement zk discovery as a separate project so that the main
> project
> > > had
> > > > > no complex dependencies. The same was true of decoupling
> > serialization.
> > > > > It's trivial to build an optional layer that adds those features
> in,
> > > > while
> > > > > allowing access to the base APIs for those that need it.
> > > > >
> > > > > Using standard Future objects was also beneficial, since I could
> > > combine
> > > > > them with existing tools (such as guava).
> > > > >
> > > > > It may be too late to be of use, but I have been working with my
> > > > company's
> > > > > legal department to release the implementation I described above.
> If
> > > > you're
> > > > > interested in it, let me know.
> > > > >
> > > > >
> > > > > To sum up my thoughts regarding the new API, I think it's a great
> > > start.
> > > > I
> > > > > would like to see a more layered approach so I can use the parts I
> > > want,
> > > > > and adapt the other parts as needed. I would also like to see
> > standard
> > > > > interfaces (especially Future) used where they makes sense.
> > > > >
> > > > > --Tom
> > > > >
> > > > >
> > > > >
> > > > >
> > > > >
> > > > > On Tue, Jan 28, 2014 at 1:33 PM, Roger Hoover <
> > roger.hoover@gmail.com
> > > > > >wrote:
> > > > >
> > > > > > +1 ListenableFuture: If this works similar to Deferreds in
> Twisted
> > > > Python
> > > > > > or Promised IO in Javascript, I think this is a great pattern for
> > > > > > decoupling your callback logic from the place where the Future is
> > > > > > generated.  You can register as many callbacks as you like, each
> in
> > > the
> > > > > > appropriate layer of the code and have each observer get notified
> > > when
> > > > > the
> > > > > > promised i/o is complete without any of them knowing about each
> > > other.
> > > > > >
> > > > > >
> > > > > > On Tue, Jan 28, 2014 at 11:32 AM, Jay Kreps <jay.kreps@gmail.com
> >
> > > > wrote:
> > > > > >
> > > > > > > Hey Ross,
> > > > > > >
> > > > > > > - ListenableFuture: Interesting. That would be an alternative
> to
> > > the
> > > > > > direct
> > > > > > > callback support we provide. There could be pros to this, let
> me
> > > > think
> > > > > > > about it.
> > > > > > > - We could provide layering, but I feel that the serialization
> is
> > > > such
> > > > > a
> > > > > > > small thing we should just make a decision and chose one, it
> > > doesn't
> > > > > seem
> > > > > > > to me to justify a whole public facing layer.
> > > > > > > - Yes, this is fairly esoteric, essentially I think it is
> fairly
> > > > > similar
> > > > > > to
> > > > > > > databases like DynamoDB that allow you to specify two partition
> > > keys
> > > > (I
> > > > > > > think DynamoDB does this...). The reasoning is that in fact
> there
> > > are
> > > > > > > several things you can use the key field for: (1) to compute
> the
> > > > > > partition
> > > > > > > to store the data in, (2) as a unique identifier to deduplicate
> > > that
> > > > > > > partition's records within a log. These two things are almost
> > > always
> > > > > the
> > > > > > > same, but occationally may differ when you want to group data
> in
> > a
> > > > more
> > > > > > > sophisticated way then just a hash of the primary key but still
> > > > retain
> > > > > > the
> > > > > > > proper primary key for delivery to the consumer and log
> > compaction.
> > > > > > >
> > > > > > > -Jay
> > > > > > >
> > > > > > >
> > > > > > > On Tue, Jan 28, 2014 at 3:24 AM, Ross Black <
> > > ross.w.black@gmail.com>
> > > > > > > wrote:
> > > > > > >
> > > > > > > > Hi Jay,
> > > > > > > >
> > > > > > > > - Just to add some more info/confusion about possibly using
> > > Future
> > > > > ...
> > > > > > > >   If Kafka uses a JDK future, it plays nicely with other
> > > frameworks
> > > > > as
> > > > > > > > well.
> > > > > > > >   Google Guava has a ListenableFuture that allows callback
> > > handling
> > > > > to
> > > > > > be
> > > > > > > > added via the returned future, and allows the callbacks to be
> > > > passed
> > > > > > off
> > > > > > > to
> > > > > > > > a specified executor.
> > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> http://docs.guava-libraries.googlecode.com/git-history/release/javadoc/com/google/common/util/concurrent/ListenableFuture.html
> > > > > > > >   The JDK future can easily be converted to a listenable
> > future.
> > > > > > > >
> > > > > > > > - On the question of byte[] vs Object, could this be solved
> by
> > > > > layering
> > > > > > > the
> > > > > > > > API?  eg. a raw producer (use byte[] and specify the
> partition
> > > > > number)
> > > > > > > and
> > > > > > > > a normal producer (use generic object and specify a
> > Partitioner)?
> > > > > > > >
> > > > > > > > - I am confused by the keys in ProducerRecord and
> Partitioner.
> > > >  What
> > > > > is
> > > > > > > the
> > > > > > > > usage for both a key and a partition key? (I am not yet using
> > > 0.8)
> > > > > > > >
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > > Ross
> > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > > On 28 January 2014 05:00, Xavier Stevens <xa...@gaikai.com>
> > > > wrote:
> > > > > > > >
> > > > > > > > > AutoCloseable would be nice for us as most of our code is
> > using
> > > > > Java
> > > > > > 7
> > > > > > > at
> > > > > > > > > this point.
> > > > > > > > >
> > > > > > > > > I like Dropwizard's configuration mapping to POJOs via
> > Jackson,
> > > > but
> > > > > > if
> > > > > > > > you
> > > > > > > > > wanted to stick with property maps I don't care enough to
> > > object.
> > > > > > > > >
> > > > > > > > > If the producer only dealt with bytes, is there a way we
> > could
> > > > > still
> > > > > > > due
> > > > > > > > > partition plugins without specifying the number
> explicitly? I
> > > > would
> > > > > > > > prefer
> > > > > > > > > to be able to pass in field(s) that would be used by the
> > > > > partitioner.
> > > > > > > > > Obviously if this wasn't possible you could always
> > deserialize
> > > > the
> > > > > > > object
> > > > > > > > > in the partitioner and grab the fields you want, but that
> > seems
> > > > > > really
> > > > > > > > > expensive to do on every message.
> > > > > > > > >
> > > > > > > > > It would also be nice to have a Java API Encoder
> constructor
> > > > taking
> > > > > > in
> > > > > > > > > VerifiableProperties. Scala understands how to handle
> "props:
> > > > > > > > > VerifiableProperties = null", but Java doesn't. So you
> don't
> > > run
> > > > > into
> > > > > > > > this
> > > > > > > > > problem until runtime.
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > -Xavier
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > On Mon, Jan 27, 2014 at 9:37 AM, Clark Breyman <
> > > > clark@breyman.com>
> > > > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > Jay -
> > > > > > > > > >
> > > > > > > > > > Config - your explanation makes sense. I'm just so
> > accustomed
> > > > to
> > > > > > > having
> > > > > > > > > > Jackson automatically map my configuration objects to
> POJOs
> > > > that
> > > > > > I've
> > > > > > > > > > stopped using property files. They are lingua franca. The
> > > only
> > > > > > > thought
> > > > > > > > > > might be to separate the config interface from the
> > > > implementation
> > > > > > to
> > > > > > > > > allow
> > > > > > > > > > for alternatives, but that might undermine your point of
> > "do
> > > it
> > > > > > this
> > > > > > > > way
> > > > > > > > > so
> > > > > > > > > > that everyone can find it where they expect it".
> > > > > > > > > >
> > > > > > > > > > Serialization: Of the options, I like 1A the best, though
> > > > > possibly
> > > > > > > with
> > > > > > > > > > either an option to specify a partition key rather than
> ID
> > > or a
> > > > > > > helper
> > > > > > > > to
> > > > > > > > > > translate an arbitrary byte[] or long into a partition
> > > number.
> > > > > > > > > >
> > > > > > > > > > Thanks
> > > > > > > > > > Clark
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > On Sun, Jan 26, 2014 at 9:13 PM, Jay Kreps <
> > > > jay.kreps@gmail.com>
> > > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > Thanks for the detailed thoughts. Let me elaborate on
> the
> > > > > config
> > > > > > > > thing.
> > > > > > > > > > >
> > > > > > > > > > > I agree that at first glance key-value strings don't
> seem
> > > > like
> > > > > a
> > > > > > > very
> > > > > > > > > > good
> > > > > > > > > > > configuration api for a client. Surely a well-typed
> > config
> > > > > class
> > > > > > > > would
> > > > > > > > > be
> > > > > > > > > > > better! I actually disagree and let me see if I can
> > > convince
> > > > > you.
> > > > > > > > > > >
> > > > > > > > > > > My reasoning has nothing to do with the api and
> > everything
> > > to
> > > > > do
> > > > > > > with
> > > > > > > > > > > operations.
> > > > > > > > > > >
> > > > > > > > > > > Clients are embedded in applications which are
> themselves
> > > > > > > configured.
> > > > > > > > > In
> > > > > > > > > > > any place that takes operations seriously the
> > configuration
> > > > for
> > > > > > > these
> > > > > > > > > > > applications will be version controlled and maintained
> > > > through
> > > > > > some
> > > > > > > > > kind
> > > > > > > > > > of
> > > > > > > > > > > config management system. If we give a config class
> with
> > > > > getters
> > > > > > > and
> > > > > > > > > > > setters the application has to expose those properties
> to
> > > its
> > > > > > > > > > > configuration. What invariably happens is that the
> > > > application
> > > > > > > > exposes
> > > > > > > > > > only
> > > > > > > > > > > a choice few properties that they thought they would
> > > change.
> > > > > > > > > Furthermore
> > > > > > > > > > > the application will make up a name for these configs
> > that
> > > > > seems
> > > > > > > > > > intuitive
> > > > > > > > > > > at the time in the 2 seconds the engineer spends
> thinking
> > > > about
> > > > > > it.
> > > > > > > > > > >
> > > > > > > > > > > Now consider the result of this in the large. You end
> up
> > > with
> > > > > > > dozens
> > > > > > > > or
> > > > > > > > > > > hundreds of applications that have the client embedded.
> > > Each
> > > > > > > exposes
> > > > > > > > a
> > > > > > > > > > > different, inadequate subset of the possible configs,
> > each
> > > > with
> > > > > > > > > different
> > > > > > > > > > > names. It is a nightmare.
> > > > > > > > > > >
> > > > > > > > > > > If you use a string-string map the config system can
> > > directly
> > > > > > get a
> > > > > > > > > > bundle
> > > > > > > > > > > of config key-value pairs and put them into the client.
> > > This
> > > > > > means
> > > > > > > > that
> > > > > > > > > > all
> > > > > > > > > > > configuration is automatically available with the name
> > > > > documented
> > > > > > > on
> > > > > > > > > the
> > > > > > > > > > > website in every application that does this. If you
> > upgrade
> > > > to
> > > > > a
> > > > > > > new
> > > > > > > > > > kafka
> > > > > > > > > > > version with more configs those will be exposed too. If
> > you
> > > > > > realize
> > > > > > > > > that
> > > > > > > > > > > you need to change a default you can just go through
> your
> > > > > configs
> > > > > > > and
> > > > > > > > > > > change it everywhere as it will have the same name
> > > > everywhere.
> > > > > > > > > > >
> > > > > > > > > > > -Jay
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > On Sun, Jan 26, 2014 at 4:47 PM, Clark Breyman <
> > > > > > clark@breyman.com>
> > > > > > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > Thanks Jay. I'll see if I can put together a more
> > > complete
> > > > > > > > response,
> > > > > > > > > > > > perhaps as separate threads so that topics don't get
> > > > > entangled.
> > > > > > > In
> > > > > > > > > the
> > > > > > > > > > > mean
> > > > > > > > > > > > time, here's a couple responses:
> > > > > > > > > > > >
> > > > > > > > > > > > Serialization: you've broken out a sub-thread so i'll
> > > reply
> > > > > > > there.
> > > > > > > > My
> > > > > > > > > > > bias
> > > > > > > > > > > > is that I like generics (except for type-erasure) and
> > in
> > > > > > > particular
> > > > > > > > > > they
> > > > > > > > > > > > make it easy to compose serializers for compound
> > payloads
> > > > > (e.g.
> > > > > > > > when
> > > > > > > > > a
> > > > > > > > > > > > common header wraps a payload of parameterized type).
> > > I'll
> > > > > > > respond
> > > > > > > > to
> > > > > > > > > > > your
> > > > > > > > > > > > 4-options message with an example.
> > > > > > > > > > > >
> > > > > > > > > > > > Build: I've seen a lot of "maven-compatible" build
> > > systems
> > > > > > > produce
> > > > > > > > > > > > "artifacts" that aren't really artifacts - no
> embedded
> > > POM
> > > > > or,
> > > > > > > > worst,
> > > > > > > > > > > > malformed POM. I know the sbt-generated artifacts
> were
> > > this
> > > > > > way -
> > > > > > > > > onus
> > > > > > > > > > is
> > > > > > > > > > > > on me to see what gradle is spitting out and what a
> > maven
> > > > > build
> > > > > > > > might
> > > > > > > > > > > look
> > > > > > > > > > > > like. Maven may be old and boring, but it gets out of
> > the
> > > > way
> > > > > > and
> > > > > > > > > > > > integrates really seamlessly with a lot of IDEs. When
> > > some
> > > > > > scala
> > > > > > > > > > > projects I
> > > > > > > > > > > > was working on in the fall of 2011 switched from sbt
> to
> > > > > maven,
> > > > > > > > build
> > > > > > > > > > > became
> > > > > > > > > > > > a non-issue.
> > > > > > > > > > > >
> > > > > > > > > > > > Config: Not a big deal  and no, I don't think a
> > > dropwizard
> > > > > > > > dependency
> > > > > > > > > > is
> > > > > > > > > > > > appropriate. I do like using simple entity beans
> > (POJO's
> > > > not
> > > > > > > j2EE)
> > > > > > > > > for
> > > > > > > > > > > > configuration, especially if they can be marshalled
> > > without
> > > > > > > > > annotation
> > > > > > > > > > by
> > > > > > > > > > > > Jackson. I only mentioned the dropwizard-extras
> >  because
> > > it
> > > > > has
> > > > > > > > some
> > > > > > > > > > > entity
> > > > > > > > > > > > bean versions of the ZK and Kafka configs.
> > > > > > > > > > > >
> > > > > > > > > > > > Domain-packaging: Also not a big deal - it's what's
> > > > expected
> > > > > > and
> > > > > > > > it's
> > > > > > > > > > > > pretty free in most IDE's. The advantages I see is
> that
> > > it
> > > > is
> > > > > > > clear
> > > > > > > > > > > whether
> > > > > > > > > > > > something is from the Apache Kafka project and
> whether
> > > > > > something
> > > > > > > is
> > > > > > > > > > from
> > > > > > > > > > > > another org and related to Kafka. That said, nothing
> > > really
> > > > > > > > enforces
> > > > > > > > > > it.
> > > > > > > > > > > >
> > > > > > > > > > > > Futures: I'll see if I can create some examples to
> > > > > demonstrate
> > > > > > > > Future
> > > > > > > > > > > > making interop easier.
> > > > > > > > > > > >
> > > > > > > > > > > > Regards,
> > > > > > > > > > > > C
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > On Fri, Jan 24, 2014 at 4:36 PM, Jay Kreps <
> > > > > > jay.kreps@gmail.com>
> > > > > > > > > > wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > > Hey Clark,
> > > > > > > > > > > > >
> > > > > > > > > > > > > - Serialization: Yes I agree with these though I
> > don't
> > > > > > consider
> > > > > > > > the
> > > > > > > > > > > loss
> > > > > > > > > > > > of
> > > > > > > > > > > > > generics a big issue. I'll try to summarize what I
> > > would
> > > > > > > consider
> > > > > > > > > the
> > > > > > > > > > > > best
> > > > > > > > > > > > > alternative api with raw byte[].
> > > > > > > > > > > > >
> > > > > > > > > > > > > - Maven: We had this debate a few months back and
> the
> > > > > > consensus
> > > > > > > > was
> > > > > > > > > > > > gradle.
> > > > > > > > > > > > > Is there a specific issue with the poms gradle
> > makes? I
> > > > am
> > > > > > > > > extremely
> > > > > > > > > > > > loath
> > > > > > > > > > > > > to revisit the issue as build issues are a
> recurring
> > > > thing
> > > > > > and
> > > > > > > no
> > > > > > > > > one
> > > > > > > > > > > > ever
> > > > > > > > > > > > > agrees and ultimately our build needs are very
> > simple.
> > > > > > > > > > > > >
> > > > > > > > > > > > > - Config: I'm not sure if I follow the point. Are
> you
> > > > > saying
> > > > > > we
> > > > > > > > > > should
> > > > > > > > > > > > use
> > > > > > > > > > > > > something in dropwizard for config? One principle
> > here
> > > is
> > > > > to
> > > > > > > try
> > > > > > > > to
> > > > > > > > > > > > remove
> > > > > > > > > > > > > as many client dependencies as possible as we
> > > inevitably
> > > > > run
> > > > > > > into
> > > > > > > > > > > > terrible
> > > > > > > > > > > > > compatibility issues with users who use the same
> > > library
> > > > or
> > > > > > its
> > > > > > > > > > > > > dependencies at different versions. Or are you
> > talking
> > > > > about
> > > > > > > > > > > maintaining
> > > > > > > > > > > > > compatibility with existing config parameters? I
> > think
> > > as
> > > > > > much
> > > > > > > > as a
> > > > > > > > > > > > config
> > > > > > > > > > > > > in the existing client makes sense it should have
> the
> > > > same
> > > > > > name
> > > > > > > > (I
> > > > > > > > > > was
> > > > > > > > > > > a
> > > > > > > > > > > > > bit sloppy about that so I'll fix any errors
> there).
> > > > There
> > > > > > are
> > > > > > > a
> > > > > > > > > few
> > > > > > > > > > > new
> > > > > > > > > > > > > things and we should give those reasonable
> defaults.
> > I
> > > > > think
> > > > > > > > config
> > > > > > > > > > is
> > > > > > > > > > > > > important so I'll start a thread on the config
> > package
> > > in
> > > > > > > there.
> > > > > > > > > > > > >
> > > > > > > > > > > > > - org.apache.kafka: We could do this. I always
> > > considered
> > > > > it
> > > > > > > kind
> > > > > > > > > of
> > > > > > > > > > an
> > > > > > > > > > > > odd
> > > > > > > > > > > > > thing Java programmers do that has no real
> motivation
> > > > (but
> > > > > I
> > > > > > > > could
> > > > > > > > > be
> > > > > > > > > > > > > re-educated!). I don't think it ends up reducing
> > naming
> > > > > > > conflicts
> > > > > > > > > in
> > > > > > > > > > > > > practice and it adds a lot of noise and nested
> > > > directories.
> > > > > > Is
> > > > > > > > > there
> > > > > > > > > > a
> > > > > > > > > > > > > reason you prefer this or just to be more standard?
> > > > > > > > > > > > >
> > > > > > > > > > > > > - Future: Basically I didn't see any particular
> > > > advantage.
> > > > > > The
> > > > > > > > > > cancel()
> > > > > > > > > > > > > method doesn't really make sense so probably
> wouldn't
> > > > work.
> > > > > > > > > Likewise
> > > > > > > > > > I
> > > > > > > > > > > > > dislike the checked exceptions it requires.
> > Basically I
> > > > > just
> > > > > > > > wrote
> > > > > > > > > > out
> > > > > > > > > > > > some
> > > > > > > > > > > > > code examples and it seemed cleaner with a special
> > > > purpose
> > > > > > > > object.
> > > > > > > > > I
> > > > > > > > > > > > wasn't
> > > > > > > > > > > > > actually aware of plans for improved futures in
> java
> > 8
> > > or
> > > > > the
> > > > > > > > other
> > > > > > > > > > > > > integrations. Maybe you could elaborate on this a
> bit
> > > and
> > > > > > show
> > > > > > > > how
> > > > > > > > > it
> > > > > > > > > > > > would
> > > > > > > > > > > > > be used? Sounds promising, I just don't know a lot
> > > about
> > > > > it.
> > > > > > > > > > > > >
> > > > > > > > > > > > > -Jay
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Fri, Jan 24, 2014 at 3:30 PM, Clark Breyman <
> > > > > > > > clark@breyman.com>
> > > > > > > > > > > > wrote:
> > > > > > > > > > > > >
> > > > > > > > > > > > > > Jay - Thanks for the call for comments. Here's
> some
> > > > > initial
> > > > > > > > > input:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > - Make message serialization a client
> > responsibility
> > > > > > (making
> > > > > > > > all
> > > > > > > > > > > > messages
> > > > > > > > > > > > > > byte[]). Reflection-based loading makes it harder
> > to
> > > > use
> > > > > > > > generic
> > > > > > > > > > > codecs
> > > > > > > > > > > > > > (e.g.  Envelope<PREFIX, DATA, SUFFIX>) or build
> up
> > > > codec
> > > > > > > > > > > > > programmatically.
> > > > > > > > > > > > > > Non-default partitioning should require an
> explicit
> > > > > > partition
> > > > > > > > > key.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > - I really like the fact that it will be native
> > Java.
> > > > > > Please
> > > > > > > > > > consider
> > > > > > > > > > > > > using
> > > > > > > > > > > > > > native maven and not sbt, gradle, ivy, etc as
> they
> > > > don't
> > > > > > > > reliably
> > > > > > > > > > > play
> > > > > > > > > > > > > nice
> > > > > > > > > > > > > > in the maven ecosystem. A jar without a
> well-formed
> > > pom
> > > > > > > doesn't
> > > > > > > > > > feel
> > > > > > > > > > > > > like a
> > > > > > > > > > > > > > real artifact. The pom's generated by sbt et al.
> > are
> > > > not
> > > > > > well
> > > > > > > > > > formed.
> > > > > > > > > > > > > Using
> > > > > > > > > > > > > > maven will make builds and IDE integration much
> > > > smoother.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > - Look at Nick Telford's dropwizard-extras
> package
> > in
> > > > > which
> > > > > > > he
> > > > > > > > > > > defines
> > > > > > > > > > > > > some
> > > > > > > > > > > > > > Jackson-compatible POJO's for loading
> > configuration.
> > > > > Seems
> > > > > > > like
> > > > > > > > > > your
> > > > > > > > > > > > > client
> > > > > > > > > > > > > > migration is similar. The config objects should
> > have
> > > > > > > > constructors
> > > > > > > > > > or
> > > > > > > > > > > > > > factories that accept Map<String, String> and
> > > > Properties
> > > > > > for
> > > > > > > > ease
> > > > > > > > > > of
> > > > > > > > > > > > > > migration.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > - Would you consider using the org.apache.kafka
> > > package
> > > > > for
> > > > > > > the
> > > > > > > > > new
> > > > > > > > > > > API
> > > > > > > > > > > > > > (quibble)
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > - Why create your own futures rather than use
> > > > > > > > > > > > > > java.util.concurrent.Future<Long> or similar?
> > > Standard
> > > > > > > futures
> > > > > > > > > will
> > > > > > > > > > > > play
> > > > > > > > > > > > > > nice with other reactive libs and things like
> J8's
> > > > > > > > > > ComposableFuture.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Thanks again,
> > > > > > > > > > > > > > C
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On Fri, Jan 24, 2014 at 2:46 PM, Roger Hoover <
> > > > > > > > > > > roger.hoover@gmail.com
> > > > > > > > > > > > > > >wrote:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > > A couple comments:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 1) Why does the config use a broker list
> instead
> > of
> > > > > > > > discovering
> > > > > > > > > > the
> > > > > > > > > > > > > > brokers
> > > > > > > > > > > > > > > in ZooKeeper?  It doesn't match the
> > > HighLevelConsumer
> > > > > > API.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 2) It looks like broker connections are created
> > on
> > > > > > demand.
> > > > > > > >  I'm
> > > > > > > > > > > > > wondering
> > > > > > > > > > > > > > > if sometimes you might want to flush out config
> > or
> > > > > > network
> > > > > > > > > > > > connectivity
> > > > > > > > > > > > > > > issues before pushing the first message
> through.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Should there also be a KafkaProducer.connect()
> or
> > > > > .open()
> > > > > > > > > method
> > > > > > > > > > or
> > > > > > > > > > > > > > > connectAll()?  I guess it would try to connect
> to
> > > all
> > > > > > > brokers
> > > > > > > > > in
> > > > > > > > > > > the
> > > > > > > > > > > > > > > BROKER_LIST_CONFIG
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > HTH,
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Roger
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > On Fri, Jan 24, 2014 at 11:54 AM, Jay Kreps <
> > > > > > > > > jay.kreps@gmail.com
> > > > > > > > > > >
> > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > As mentioned in a previous email we are
> working
> > > on
> > > > a
> > > > > > > > > > > > > re-implementation
> > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > the producer. I would like to use this email
> > > thread
> > > > > to
> > > > > > > > > discuss
> > > > > > > > > > > the
> > > > > > > > > > > > > > > details
> > > > > > > > > > > > > > > > of the public API and the configuration. I
> > would
> > > > love
> > > > > > for
> > > > > > > > us
> > > > > > > > > to
> > > > > > > > > > > be
> > > > > > > > > > > > > > > > incredibly picky about this public api now so
> > it
> > > is
> > > > > as
> > > > > > > good
> > > > > > > > > as
> > > > > > > > > > > > > possible
> > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > we don't need to break it in the future.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > The best way to get a feel for the API is
> > > actually
> > > > to
> > > > > > > take
> > > > > > > > a
> > > > > > > > > > look
> > > > > > > > > > > > at
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > javadoc, my hope is to get the api docs good
> > > enough
> > > > > so
> > > > > > > that
> > > > > > > > > it
> > > > > > > > > > is
> > > > > > > > > > > > > > > > self-explanatory:
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> http://empathybox.com/kafka-javadoc/index.html?kafka/clients/producer/KafkaProducer.html
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Please take a look at this API and give me
> any
> > > > > thoughts
> > > > > > > you
> > > > > > > > > may
> > > > > > > > > > > > have!
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > It may also be reasonable to take a look at
> the
> > > > > > configs:
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> http://empathybox.com/kafka-javadoc/kafka/clients/producer/ProducerConfig.html
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > The actual code is posted here:
> > > > > > > > > > > > > > > >
> > https://issues.apache.org/jira/browse/KAFKA-1227
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > A few questions or comments to kick things
> off:
> > > > > > > > > > > > > > > > 1. We need to make a decision on whether
> > > > > serialization
> > > > > > of
> > > > > > > > the
> > > > > > > > > > > > user's
> > > > > > > > > > > > > > key
> > > > > > > > > > > > > > > > and value should be done by the user (with
> our
> > > api
> > > > > just
> > > > > > > > > taking
> > > > > > > > > > > > > byte[])
> > > > > > > > > > > > > > or
> > > > > > > > > > > > > > > > if we should take an object and allow the
> user
> > to
> > > > > > > > configure a
> > > > > > > > > > > > > > Serializer
> > > > > > > > > > > > > > > > class which we instantiate via reflection. We
> > > take
> > > > > the
> > > > > > > > later
> > > > > > > > > > > > approach
> > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > the current producer, and I have carried this
> > > > through
> > > > > > to
> > > > > > > > this
> > > > > > > > > > > > > > prototype.
> > > > > > > > > > > > > > > > The tradeoff I see is this: taking byte[] is
> > > > actually
> > > > > > > > > simpler,
> > > > > > > > > > > the
> > > > > > > > > > > > > user
> > > > > > > > > > > > > > > can
> > > > > > > > > > > > > > > > directly do whatever serialization they like.
> > The
> > > > > > > > > complication
> > > > > > > > > > is
> > > > > > > > > > > > > > > actually
> > > > > > > > > > > > > > > > partitioning. Currently partitioning is done
> > by a
> > > > > > similar
> > > > > > > > > > plug-in
> > > > > > > > > > > > api
> > > > > > > > > > > > > > > > (Partitioner) which the user can implement
> and
> > > > > > configure
> > > > > > > to
> > > > > > > > > > > > override
> > > > > > > > > > > > > > how
> > > > > > > > > > > > > > > > partitions are assigned. If we take byte[] as
> > > input
> > > > > > then
> > > > > > > we
> > > > > > > > > > have
> > > > > > > > > > > no
> > > > > > > > > > > > > > > access
> > > > > > > > > > > > > > > > to the original object and partitioning MUST
> be
> > > > done
> > > > > on
> > > > > > > the
> > > > > > > > > > > byte[].
> > > > > > > > > > > > > > This
> > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > fine for hash partitioning. However for
> various
> > > > types
> > > > > > of
> > > > > > > > > > semantic
> > > > > > > > > > > > > > > > partitioning (range partitioning, or
> whatever)
> > > you
> > > > > > would
> > > > > > > > want
> > > > > > > > > > > > access
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > original object. In the current approach a
> > > producer
> > > > > who
> > > > > > > > > wishes
> > > > > > > > > > to
> > > > > > > > > > > > > send
> > > > > > > > > > > > > > > > byte[] they have serialized in their own code
> > can
> > > > > > > configure
> > > > > > > > > the
> > > > > > > > > > > > > > > > BytesSerialization we supply which is just a
> > "no
> > > > op"
> > > > > > > > > > > serialization.
> > > > > > > > > > > > > > > > 2. We should obsess over naming and make sure
> > > each
> > > > of
> > > > > > the
> > > > > > > > > class
> > > > > > > > > > > > names
> > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > good.
> > > > > > > > > > > > > > > > 3. Jun has already pointed out that we need
> to
> > > > > include
> > > > > > > the
> > > > > > > > > > topic
> > > > > > > > > > > > and
> > > > > > > > > > > > > > > > partition in the response, which is
> absolutely
> > > > > right. I
> > > > > > > > > haven't
> > > > > > > > > > > > done
> > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > yet but that definitely needs to be there.
> > > > > > > > > > > > > > > > 4. Currently RecordSend.await will throw an
> > > > exception
> > > > > > if
> > > > > > > > the
> > > > > > > > > > > > request
> > > > > > > > > > > > > > > > failed. The intention here is that
> > > > > > > > > > producer.send(message).await()
> > > > > > > > > > > > > > exactly
> > > > > > > > > > > > > > > > simulates a synchronous call. Guozhang has
> > noted
> > > > that
> > > > > > > this
> > > > > > > > > is a
> > > > > > > > > > > > > little
> > > > > > > > > > > > > > > > annoying since the user must then catch
> > > exceptions.
> > > > > > > However
> > > > > > > > > if
> > > > > > > > > > we
> > > > > > > > > > > > > > remove
> > > > > > > > > > > > > > > > this then if the user doesn't check for
> errors
> > > they
> > > > > > won't
> > > > > > > > > know
> > > > > > > > > > > one
> > > > > > > > > > > > > has
> > > > > > > > > > > > > > > > occurred, which I predict will be a common
> > > mistake.
> > > > > > > > > > > > > > > > 5. Perhaps there is more we could do to make
> > the
> > > > > async
> > > > > > > > > > callbacks
> > > > > > > > > > > > and
> > > > > > > > > > > > > > > future
> > > > > > > > > > > > > > > > we give back intuitive and easy to program
> > > against?
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Some background info on implementation:
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > At a high level the primary difference in
> this
> > > > > producer
> > > > > > > is
> > > > > > > > > that
> > > > > > > > > > > it
> > > > > > > > > > > > > > > removes
> > > > > > > > > > > > > > > > the distinction between the "sync" and
> "async"
> > > > > > producer.
> > > > > > > > > > > > Effectively
> > > > > > > > > > > > > > all
> > > > > > > > > > > > > > > > requests are sent asynchronously but always
> > > return
> > > > a
> > > > > > > future
> > > > > > > > > > > > response
> > > > > > > > > > > > > > > object
> > > > > > > > > > > > > > > > that gives the offset as well as any error
> that
> > > may
> > > > > > have
> > > > > > > > > > occurred
> > > > > > > > > > > > > when
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > request is complete. The batching that is
> done
> > in
> > > > the
> > > > > > > async
> > > > > > > > > > > > producer
> > > > > > > > > > > > > > only
> > > > > > > > > > > > > > > > today is done whenever possible now. This
> means
> > > > that
> > > > > > the
> > > > > > > > sync
> > > > > > > > > > > > > producer,
> > > > > > > > > > > > > > > > under load, can get performance as good as
> the
> > > > async
> > > > > > > > producer
> > > > > > > > > > > > > > > (preliminary
> > > > > > > > > > > > > > > > results show the producer getting 1m
> > > messages/sec).
> > > > > > This
> > > > > > > > > works
> > > > > > > > > > > > > similar
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > group commit in databases but with respect to
> > the
> > > > > > actual
> > > > > > > > > > network
> > > > > > > > > > > > > > > > transmission--any messages that arrive while
> a
> > > send
> > > > > is
> > > > > > in
> > > > > > > > > > > progress
> > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > batched together. It is also possible to
> > > encourage
> > > > > > > batching
> > > > > > > > > > even
> > > > > > > > > > > > > under
> > > > > > > > > > > > > > > low
> > > > > > > > > > > > > > > > load to save server resources by introducing
> a
> > > > delay
> > > > > on
> > > > > > > the
> > > > > > > > > > send
> > > > > > > > > > > to
> > > > > > > > > > > > > > allow
> > > > > > > > > > > > > > > > more messages to accumulate; this is done
> using
> > > the
> > > > > > > > > > > linger.msconfig
> > > > > > > > > > > > > > > (this
> > > > > > > > > > > > > > > > is similar to Nagle's algorithm in TCP).
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > This producer does all network communication
> > > > > > > asynchronously
> > > > > > > > > and
> > > > > > > > > > > in
> > > > > > > > > > > > > > > parallel
> > > > > > > > > > > > > > > > to all servers so the performance penalty for
> > > > acks=-1
> > > > > > and
> > > > > > > > > > waiting
> > > > > > > > > > > > on
> > > > > > > > > > > > > > > > replication should be much reduced. I haven't
> > > done
> > > > > much
> > > > > > > > > > > > benchmarking
> > > > > > > > > > > > > on
> > > > > > > > > > > > > > > > this yet, though.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > The high level design is described a little
> > here,
> > > > > > though
> > > > > > > > this
> > > > > > > > > > is
> > > > > > > > > > > > now
> > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > little out of date:
> > > > > > > > > > > > > > > >
> > > > > > > > > >
> > > > https://cwiki.apache.org/confluence/display/KAFKA/Client+Rewrite
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > -Jay
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: New Producer Public API

Posted by Tom Brown <to...@gmail.com>.
Jay,

There is both a basic client object, and a number of IO processing threads.
The client object manages connections, creating new ones when new machines
are connected, or when existing connections die. It also manages a queue of
requests for each server. The IO processing thread has a selector, and
performs the work of sending/receiving (removing items from the queue,
interpreting the response at a basic level, etc). Since asynchronous
sockets by nature decouple sending and receiving, request pipelining is
inherent.

Using the basic client, you can send individual produce requests (singular
or batched). The "producer" layer adds an additional queue for each
partition, allowing individual messages to be batched together.

--Tom



On Tue, Jan 28, 2014 at 9:10 PM, Jay Kreps <ja...@gmail.com> wrote:

> Hey Neha,
>
> Can you elaborate on why you prefer using Java's Future? The downside in my
> mind is the use of the checked InterruptedException and ExecutionException.
> ExecutionException is arguable, but forcing you to catch
> InterruptedException, often in code that can't be interrupted, seems
> perverse. It also leaves us with the cancel() method which I don't think we
> really can implement.
>
> Option 1A, to recap/elaborate, was the following. There is no Serializer or
> Partitioner api. We take a byte[] key and value and an optional integer
> partition. If you specify the integer partition it will be used. If you do
> not specify a key or a partition the partition will be chosen in a round
> robin fashion. If you specify a key but no partition we will chose a
> partition based on a hash of the key. In order to let the user find the
> partition we will need to given them access to the Cluster instance
> directly from the producer.
>
> -Jay
>
>
> On Tue, Jan 28, 2014 at 6:25 PM, Neha Narkhede <neha.narkhede@gmail.com
> >wrote:
>
> > Here are more thoughts on the public APIs -
> >
> > - I suggest we use java's Future instead of custom Future especially
> since
> > it is part of the public API
> >
> > - Serialization: I like the simplicity of the producer APIs with the
> > absence of serialization where we just deal with byte arrays for keys and
> > values. What I don't like about this is the performance overhead on the
> > Partitioner for any kind of custom partitioning based on the
> partitionKey.
> > Since the only purpose of partitionKey is to do custom partitioning, why
> > can't we take it in directly as an integer and let the user figure out
> the
> > mapping from partition_key -> partition_id using the getCluster() API?
> If I
> > understand correctly, this is similar to what you suggested as part of
> > option 1A. I like this approach since it maintains the simplicity of APIs
> > by allowing us to deal with bytes and does not compromise performance in
> > the custom partitioning case.
> >
> > Thanks,
> > Neha
> >
> >
> >
> > On Tue, Jan 28, 2014 at 5:42 PM, Jay Kreps <ja...@gmail.com> wrote:
> >
> > > Hey Tom,
> > >
> > > That sounds cool. How did you end up handling parallel I/O if you wrap
> > the
> > > individual connections? Don't you need some selector that selects over
> > all
> > > the connections?
> > >
> > > -Jay
> > >
> > >
> > > On Tue, Jan 28, 2014 at 2:31 PM, Tom Brown <to...@gmail.com>
> wrote:
> > >
> > > > I implemented a 0.7 client in pure java, and its API very closely
> > > resembled
> > > > this. (When multiple people independently engineer the same solution,
> > > it's
> > > > probably good... right?). However, there were a few architectural
> > > > differences with my client:
> > > >
> > > > 1. The basic client itself was just an asynchronous layer around the
> > > > different server functions. In and of itself it had no knowledge of
> > > > partitions, only servers (and maintained TCP connections to them).
> > > >
> > > > 2. The main producer was an additional layer that provided a
> high-level
> > > > interface that could batch individual messages based on partition.
> > > >
> > > > 3. Knowledge of partitioning was done via an interface so that
> > different
> > > > strategies could be used.
> > > >
> > > > 4. Partitioning was done by the user, with knowledge of the available
> > > > partitions provided by #3.
> > > >
> > > > 5. Serialization was done by the user to simplify the API.
> > > >
> > > > 6. Futures were used to make asynchronous emulate synchronous calls.
> > > >
> > > >
> > > > The main benefit of this approach is flexibility. For example, since
> > the
> > > > base client was just a managed connection (and not inherently a
> > > producer),
> > > > it was easy to composite a produce request and an offsets request
> > > together
> > > > into a confirmed produce request (officially not available in 0.7).
> > > >
> > > > Decoupling the basic client from partition management allowed the me
> to
> > > > implement zk discovery as a separate project so that the main project
> > had
> > > > no complex dependencies. The same was true of decoupling
> serialization.
> > > > It's trivial to build an optional layer that adds those features in,
> > > while
> > > > allowing access to the base APIs for those that need it.
> > > >
> > > > Using standard Future objects was also beneficial, since I could
> > combine
> > > > them with existing tools (such as guava).
> > > >
> > > > It may be too late to be of use, but I have been working with my
> > > company's
> > > > legal department to release the implementation I described above. If
> > > you're
> > > > interested in it, let me know.
> > > >
> > > >
> > > > To sum up my thoughts regarding the new API, I think it's a great
> > start.
> > > I
> > > > would like to see a more layered approach so I can use the parts I
> > want,
> > > > and adapt the other parts as needed. I would also like to see
> standard
> > > > interfaces (especially Future) used where they makes sense.
> > > >
> > > > --Tom
> > > >
> > > >
> > > >
> > > >
> > > >
> > > > On Tue, Jan 28, 2014 at 1:33 PM, Roger Hoover <
> roger.hoover@gmail.com
> > > > >wrote:
> > > >
> > > > > +1 ListenableFuture: If this works similar to Deferreds in Twisted
> > > Python
> > > > > or Promised IO in Javascript, I think this is a great pattern for
> > > > > decoupling your callback logic from the place where the Future is
> > > > > generated.  You can register as many callbacks as you like, each in
> > the
> > > > > appropriate layer of the code and have each observer get notified
> > when
> > > > the
> > > > > promised i/o is complete without any of them knowing about each
> > other.
> > > > >
> > > > >
> > > > > On Tue, Jan 28, 2014 at 11:32 AM, Jay Kreps <ja...@gmail.com>
> > > wrote:
> > > > >
> > > > > > Hey Ross,
> > > > > >
> > > > > > - ListenableFuture: Interesting. That would be an alternative to
> > the
> > > > > direct
> > > > > > callback support we provide. There could be pros to this, let me
> > > think
> > > > > > about it.
> > > > > > - We could provide layering, but I feel that the serialization is
> > > such
> > > > a
> > > > > > small thing we should just make a decision and chose one, it
> > doesn't
> > > > seem
> > > > > > to me to justify a whole public facing layer.
> > > > > > - Yes, this is fairly esoteric, essentially I think it is fairly
> > > > similar
> > > > > to
> > > > > > databases like DynamoDB that allow you to specify two partition
> > keys
> > > (I
> > > > > > think DynamoDB does this...). The reasoning is that in fact there
> > are
> > > > > > several things you can use the key field for: (1) to compute the
> > > > > partition
> > > > > > to store the data in, (2) as a unique identifier to deduplicate
> > that
> > > > > > partition's records within a log. These two things are almost
> > always
> > > > the
> > > > > > same, but occationally may differ when you want to group data in
> a
> > > more
> > > > > > sophisticated way then just a hash of the primary key but still
> > > retain
> > > > > the
> > > > > > proper primary key for delivery to the consumer and log
> compaction.
> > > > > >
> > > > > > -Jay
> > > > > >
> > > > > >
> > > > > > On Tue, Jan 28, 2014 at 3:24 AM, Ross Black <
> > ross.w.black@gmail.com>
> > > > > > wrote:
> > > > > >
> > > > > > > Hi Jay,
> > > > > > >
> > > > > > > - Just to add some more info/confusion about possibly using
> > Future
> > > > ...
> > > > > > >   If Kafka uses a JDK future, it plays nicely with other
> > frameworks
> > > > as
> > > > > > > well.
> > > > > > >   Google Guava has a ListenableFuture that allows callback
> > handling
> > > > to
> > > > > be
> > > > > > > added via the returned future, and allows the callbacks to be
> > > passed
> > > > > off
> > > > > > to
> > > > > > > a specified executor.
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> http://docs.guava-libraries.googlecode.com/git-history/release/javadoc/com/google/common/util/concurrent/ListenableFuture.html
> > > > > > >   The JDK future can easily be converted to a listenable
> future.
> > > > > > >
> > > > > > > - On the question of byte[] vs Object, could this be solved by
> > > > layering
> > > > > > the
> > > > > > > API?  eg. a raw producer (use byte[] and specify the partition
> > > > number)
> > > > > > and
> > > > > > > a normal producer (use generic object and specify a
> Partitioner)?
> > > > > > >
> > > > > > > - I am confused by the keys in ProducerRecord and Partitioner.
> > >  What
> > > > is
> > > > > > the
> > > > > > > usage for both a key and a partition key? (I am not yet using
> > 0.8)
> > > > > > >
> > > > > > >
> > > > > > > Thanks,
> > > > > > > Ross
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > On 28 January 2014 05:00, Xavier Stevens <xa...@gaikai.com>
> > > wrote:
> > > > > > >
> > > > > > > > AutoCloseable would be nice for us as most of our code is
> using
> > > > Java
> > > > > 7
> > > > > > at
> > > > > > > > this point.
> > > > > > > >
> > > > > > > > I like Dropwizard's configuration mapping to POJOs via
> Jackson,
> > > but
> > > > > if
> > > > > > > you
> > > > > > > > wanted to stick with property maps I don't care enough to
> > object.
> > > > > > > >
> > > > > > > > If the producer only dealt with bytes, is there a way we
> could
> > > > still
> > > > > > due
> > > > > > > > partition plugins without specifying the number explicitly? I
> > > would
> > > > > > > prefer
> > > > > > > > to be able to pass in field(s) that would be used by the
> > > > partitioner.
> > > > > > > > Obviously if this wasn't possible you could always
> deserialize
> > > the
> > > > > > object
> > > > > > > > in the partitioner and grab the fields you want, but that
> seems
> > > > > really
> > > > > > > > expensive to do on every message.
> > > > > > > >
> > > > > > > > It would also be nice to have a Java API Encoder constructor
> > > taking
> > > > > in
> > > > > > > > VerifiableProperties. Scala understands how to handle "props:
> > > > > > > > VerifiableProperties = null", but Java doesn't. So you don't
> > run
> > > > into
> > > > > > > this
> > > > > > > > problem until runtime.
> > > > > > > >
> > > > > > > >
> > > > > > > > -Xavier
> > > > > > > >
> > > > > > > >
> > > > > > > > On Mon, Jan 27, 2014 at 9:37 AM, Clark Breyman <
> > > clark@breyman.com>
> > > > > > > wrote:
> > > > > > > >
> > > > > > > > > Jay -
> > > > > > > > >
> > > > > > > > > Config - your explanation makes sense. I'm just so
> accustomed
> > > to
> > > > > > having
> > > > > > > > > Jackson automatically map my configuration objects to POJOs
> > > that
> > > > > I've
> > > > > > > > > stopped using property files. They are lingua franca. The
> > only
> > > > > > thought
> > > > > > > > > might be to separate the config interface from the
> > > implementation
> > > > > to
> > > > > > > > allow
> > > > > > > > > for alternatives, but that might undermine your point of
> "do
> > it
> > > > > this
> > > > > > > way
> > > > > > > > so
> > > > > > > > > that everyone can find it where they expect it".
> > > > > > > > >
> > > > > > > > > Serialization: Of the options, I like 1A the best, though
> > > > possibly
> > > > > > with
> > > > > > > > > either an option to specify a partition key rather than ID
> > or a
> > > > > > helper
> > > > > > > to
> > > > > > > > > translate an arbitrary byte[] or long into a partition
> > number.
> > > > > > > > >
> > > > > > > > > Thanks
> > > > > > > > > Clark
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > On Sun, Jan 26, 2014 at 9:13 PM, Jay Kreps <
> > > jay.kreps@gmail.com>
> > > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > Thanks for the detailed thoughts. Let me elaborate on the
> > > > config
> > > > > > > thing.
> > > > > > > > > >
> > > > > > > > > > I agree that at first glance key-value strings don't seem
> > > like
> > > > a
> > > > > > very
> > > > > > > > > good
> > > > > > > > > > configuration api for a client. Surely a well-typed
> config
> > > > class
> > > > > > > would
> > > > > > > > be
> > > > > > > > > > better! I actually disagree and let me see if I can
> > convince
> > > > you.
> > > > > > > > > >
> > > > > > > > > > My reasoning has nothing to do with the api and
> everything
> > to
> > > > do
> > > > > > with
> > > > > > > > > > operations.
> > > > > > > > > >
> > > > > > > > > > Clients are embedded in applications which are themselves
> > > > > > configured.
> > > > > > > > In
> > > > > > > > > > any place that takes operations seriously the
> configuration
> > > for
> > > > > > these
> > > > > > > > > > applications will be version controlled and maintained
> > > through
> > > > > some
> > > > > > > > kind
> > > > > > > > > of
> > > > > > > > > > config management system. If we give a config class with
> > > > getters
> > > > > > and
> > > > > > > > > > setters the application has to expose those properties to
> > its
> > > > > > > > > > configuration. What invariably happens is that the
> > > application
> > > > > > > exposes
> > > > > > > > > only
> > > > > > > > > > a choice few properties that they thought they would
> > change.
> > > > > > > > Furthermore
> > > > > > > > > > the application will make up a name for these configs
> that
> > > > seems
> > > > > > > > > intuitive
> > > > > > > > > > at the time in the 2 seconds the engineer spends thinking
> > > about
> > > > > it.
> > > > > > > > > >
> > > > > > > > > > Now consider the result of this in the large. You end up
> > with
> > > > > > dozens
> > > > > > > or
> > > > > > > > > > hundreds of applications that have the client embedded.
> > Each
> > > > > > exposes
> > > > > > > a
> > > > > > > > > > different, inadequate subset of the possible configs,
> each
> > > with
> > > > > > > > different
> > > > > > > > > > names. It is a nightmare.
> > > > > > > > > >
> > > > > > > > > > If you use a string-string map the config system can
> > directly
> > > > > get a
> > > > > > > > > bundle
> > > > > > > > > > of config key-value pairs and put them into the client.
> > This
> > > > > means
> > > > > > > that
> > > > > > > > > all
> > > > > > > > > > configuration is automatically available with the name
> > > > documented
> > > > > > on
> > > > > > > > the
> > > > > > > > > > website in every application that does this. If you
> upgrade
> > > to
> > > > a
> > > > > > new
> > > > > > > > > kafka
> > > > > > > > > > version with more configs those will be exposed too. If
> you
> > > > > realize
> > > > > > > > that
> > > > > > > > > > you need to change a default you can just go through your
> > > > configs
> > > > > > and
> > > > > > > > > > change it everywhere as it will have the same name
> > > everywhere.
> > > > > > > > > >
> > > > > > > > > > -Jay
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > On Sun, Jan 26, 2014 at 4:47 PM, Clark Breyman <
> > > > > clark@breyman.com>
> > > > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > Thanks Jay. I'll see if I can put together a more
> > complete
> > > > > > > response,
> > > > > > > > > > > perhaps as separate threads so that topics don't get
> > > > entangled.
> > > > > > In
> > > > > > > > the
> > > > > > > > > > mean
> > > > > > > > > > > time, here's a couple responses:
> > > > > > > > > > >
> > > > > > > > > > > Serialization: you've broken out a sub-thread so i'll
> > reply
> > > > > > there.
> > > > > > > My
> > > > > > > > > > bias
> > > > > > > > > > > is that I like generics (except for type-erasure) and
> in
> > > > > > particular
> > > > > > > > > they
> > > > > > > > > > > make it easy to compose serializers for compound
> payloads
> > > > (e.g.
> > > > > > > when
> > > > > > > > a
> > > > > > > > > > > common header wraps a payload of parameterized type).
> > I'll
> > > > > > respond
> > > > > > > to
> > > > > > > > > > your
> > > > > > > > > > > 4-options message with an example.
> > > > > > > > > > >
> > > > > > > > > > > Build: I've seen a lot of "maven-compatible" build
> > systems
> > > > > > produce
> > > > > > > > > > > "artifacts" that aren't really artifacts - no embedded
> > POM
> > > > or,
> > > > > > > worst,
> > > > > > > > > > > malformed POM. I know the sbt-generated artifacts were
> > this
> > > > > way -
> > > > > > > > onus
> > > > > > > > > is
> > > > > > > > > > > on me to see what gradle is spitting out and what a
> maven
> > > > build
> > > > > > > might
> > > > > > > > > > look
> > > > > > > > > > > like. Maven may be old and boring, but it gets out of
> the
> > > way
> > > > > and
> > > > > > > > > > > integrates really seamlessly with a lot of IDEs. When
> > some
> > > > > scala
> > > > > > > > > > projects I
> > > > > > > > > > > was working on in the fall of 2011 switched from sbt to
> > > > maven,
> > > > > > > build
> > > > > > > > > > became
> > > > > > > > > > > a non-issue.
> > > > > > > > > > >
> > > > > > > > > > > Config: Not a big deal  and no, I don't think a
> > dropwizard
> > > > > > > dependency
> > > > > > > > > is
> > > > > > > > > > > appropriate. I do like using simple entity beans
> (POJO's
> > > not
> > > > > > j2EE)
> > > > > > > > for
> > > > > > > > > > > configuration, especially if they can be marshalled
> > without
> > > > > > > > annotation
> > > > > > > > > by
> > > > > > > > > > > Jackson. I only mentioned the dropwizard-extras
>  because
> > it
> > > > has
> > > > > > > some
> > > > > > > > > > entity
> > > > > > > > > > > bean versions of the ZK and Kafka configs.
> > > > > > > > > > >
> > > > > > > > > > > Domain-packaging: Also not a big deal - it's what's
> > > expected
> > > > > and
> > > > > > > it's
> > > > > > > > > > > pretty free in most IDE's. The advantages I see is that
> > it
> > > is
> > > > > > clear
> > > > > > > > > > whether
> > > > > > > > > > > something is from the Apache Kafka project and whether
> > > > > something
> > > > > > is
> > > > > > > > > from
> > > > > > > > > > > another org and related to Kafka. That said, nothing
> > really
> > > > > > > enforces
> > > > > > > > > it.
> > > > > > > > > > >
> > > > > > > > > > > Futures: I'll see if I can create some examples to
> > > > demonstrate
> > > > > > > Future
> > > > > > > > > > > making interop easier.
> > > > > > > > > > >
> > > > > > > > > > > Regards,
> > > > > > > > > > > C
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > On Fri, Jan 24, 2014 at 4:36 PM, Jay Kreps <
> > > > > jay.kreps@gmail.com>
> > > > > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > Hey Clark,
> > > > > > > > > > > >
> > > > > > > > > > > > - Serialization: Yes I agree with these though I
> don't
> > > > > consider
> > > > > > > the
> > > > > > > > > > loss
> > > > > > > > > > > of
> > > > > > > > > > > > generics a big issue. I'll try to summarize what I
> > would
> > > > > > consider
> > > > > > > > the
> > > > > > > > > > > best
> > > > > > > > > > > > alternative api with raw byte[].
> > > > > > > > > > > >
> > > > > > > > > > > > - Maven: We had this debate a few months back and the
> > > > > consensus
> > > > > > > was
> > > > > > > > > > > gradle.
> > > > > > > > > > > > Is there a specific issue with the poms gradle
> makes? I
> > > am
> > > > > > > > extremely
> > > > > > > > > > > loath
> > > > > > > > > > > > to revisit the issue as build issues are a recurring
> > > thing
> > > > > and
> > > > > > no
> > > > > > > > one
> > > > > > > > > > > ever
> > > > > > > > > > > > agrees and ultimately our build needs are very
> simple.
> > > > > > > > > > > >
> > > > > > > > > > > > - Config: I'm not sure if I follow the point. Are you
> > > > saying
> > > > > we
> > > > > > > > > should
> > > > > > > > > > > use
> > > > > > > > > > > > something in dropwizard for config? One principle
> here
> > is
> > > > to
> > > > > > try
> > > > > > > to
> > > > > > > > > > > remove
> > > > > > > > > > > > as many client dependencies as possible as we
> > inevitably
> > > > run
> > > > > > into
> > > > > > > > > > > terrible
> > > > > > > > > > > > compatibility issues with users who use the same
> > library
> > > or
> > > > > its
> > > > > > > > > > > > dependencies at different versions. Or are you
> talking
> > > > about
> > > > > > > > > > maintaining
> > > > > > > > > > > > compatibility with existing config parameters? I
> think
> > as
> > > > > much
> > > > > > > as a
> > > > > > > > > > > config
> > > > > > > > > > > > in the existing client makes sense it should have the
> > > same
> > > > > name
> > > > > > > (I
> > > > > > > > > was
> > > > > > > > > > a
> > > > > > > > > > > > bit sloppy about that so I'll fix any errors there).
> > > There
> > > > > are
> > > > > > a
> > > > > > > > few
> > > > > > > > > > new
> > > > > > > > > > > > things and we should give those reasonable defaults.
> I
> > > > think
> > > > > > > config
> > > > > > > > > is
> > > > > > > > > > > > important so I'll start a thread on the config
> package
> > in
> > > > > > there.
> > > > > > > > > > > >
> > > > > > > > > > > > - org.apache.kafka: We could do this. I always
> > considered
> > > > it
> > > > > > kind
> > > > > > > > of
> > > > > > > > > an
> > > > > > > > > > > odd
> > > > > > > > > > > > thing Java programmers do that has no real motivation
> > > (but
> > > > I
> > > > > > > could
> > > > > > > > be
> > > > > > > > > > > > re-educated!). I don't think it ends up reducing
> naming
> > > > > > conflicts
> > > > > > > > in
> > > > > > > > > > > > practice and it adds a lot of noise and nested
> > > directories.
> > > > > Is
> > > > > > > > there
> > > > > > > > > a
> > > > > > > > > > > > reason you prefer this or just to be more standard?
> > > > > > > > > > > >
> > > > > > > > > > > > - Future: Basically I didn't see any particular
> > > advantage.
> > > > > The
> > > > > > > > > cancel()
> > > > > > > > > > > > method doesn't really make sense so probably wouldn't
> > > work.
> > > > > > > > Likewise
> > > > > > > > > I
> > > > > > > > > > > > dislike the checked exceptions it requires.
> Basically I
> > > > just
> > > > > > > wrote
> > > > > > > > > out
> > > > > > > > > > > some
> > > > > > > > > > > > code examples and it seemed cleaner with a special
> > > purpose
> > > > > > > object.
> > > > > > > > I
> > > > > > > > > > > wasn't
> > > > > > > > > > > > actually aware of plans for improved futures in java
> 8
> > or
> > > > the
> > > > > > > other
> > > > > > > > > > > > integrations. Maybe you could elaborate on this a bit
> > and
> > > > > show
> > > > > > > how
> > > > > > > > it
> > > > > > > > > > > would
> > > > > > > > > > > > be used? Sounds promising, I just don't know a lot
> > about
> > > > it.
> > > > > > > > > > > >
> > > > > > > > > > > > -Jay
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > On Fri, Jan 24, 2014 at 3:30 PM, Clark Breyman <
> > > > > > > clark@breyman.com>
> > > > > > > > > > > wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > > Jay - Thanks for the call for comments. Here's some
> > > > initial
> > > > > > > > input:
> > > > > > > > > > > > >
> > > > > > > > > > > > > - Make message serialization a client
> responsibility
> > > > > (making
> > > > > > > all
> > > > > > > > > > > messages
> > > > > > > > > > > > > byte[]). Reflection-based loading makes it harder
> to
> > > use
> > > > > > > generic
> > > > > > > > > > codecs
> > > > > > > > > > > > > (e.g.  Envelope<PREFIX, DATA, SUFFIX>) or build up
> > > codec
> > > > > > > > > > > > programmatically.
> > > > > > > > > > > > > Non-default partitioning should require an explicit
> > > > > partition
> > > > > > > > key.
> > > > > > > > > > > > >
> > > > > > > > > > > > > - I really like the fact that it will be native
> Java.
> > > > > Please
> > > > > > > > > consider
> > > > > > > > > > > > using
> > > > > > > > > > > > > native maven and not sbt, gradle, ivy, etc as they
> > > don't
> > > > > > > reliably
> > > > > > > > > > play
> > > > > > > > > > > > nice
> > > > > > > > > > > > > in the maven ecosystem. A jar without a well-formed
> > pom
> > > > > > doesn't
> > > > > > > > > feel
> > > > > > > > > > > > like a
> > > > > > > > > > > > > real artifact. The pom's generated by sbt et al.
> are
> > > not
> > > > > well
> > > > > > > > > formed.
> > > > > > > > > > > > Using
> > > > > > > > > > > > > maven will make builds and IDE integration much
> > > smoother.
> > > > > > > > > > > > >
> > > > > > > > > > > > > - Look at Nick Telford's dropwizard-extras package
> in
> > > > which
> > > > > > he
> > > > > > > > > > defines
> > > > > > > > > > > > some
> > > > > > > > > > > > > Jackson-compatible POJO's for loading
> configuration.
> > > > Seems
> > > > > > like
> > > > > > > > > your
> > > > > > > > > > > > client
> > > > > > > > > > > > > migration is similar. The config objects should
> have
> > > > > > > constructors
> > > > > > > > > or
> > > > > > > > > > > > > factories that accept Map<String, String> and
> > > Properties
> > > > > for
> > > > > > > ease
> > > > > > > > > of
> > > > > > > > > > > > > migration.
> > > > > > > > > > > > >
> > > > > > > > > > > > > - Would you consider using the org.apache.kafka
> > package
> > > > for
> > > > > > the
> > > > > > > > new
> > > > > > > > > > API
> > > > > > > > > > > > > (quibble)
> > > > > > > > > > > > >
> > > > > > > > > > > > > - Why create your own futures rather than use
> > > > > > > > > > > > > java.util.concurrent.Future<Long> or similar?
> > Standard
> > > > > > futures
> > > > > > > > will
> > > > > > > > > > > play
> > > > > > > > > > > > > nice with other reactive libs and things like J8's
> > > > > > > > > ComposableFuture.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Thanks again,
> > > > > > > > > > > > > C
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Fri, Jan 24, 2014 at 2:46 PM, Roger Hoover <
> > > > > > > > > > roger.hoover@gmail.com
> > > > > > > > > > > > > >wrote:
> > > > > > > > > > > > >
> > > > > > > > > > > > > > A couple comments:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 1) Why does the config use a broker list instead
> of
> > > > > > > discovering
> > > > > > > > > the
> > > > > > > > > > > > > brokers
> > > > > > > > > > > > > > in ZooKeeper?  It doesn't match the
> > HighLevelConsumer
> > > > > API.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 2) It looks like broker connections are created
> on
> > > > > demand.
> > > > > > >  I'm
> > > > > > > > > > > > wondering
> > > > > > > > > > > > > > if sometimes you might want to flush out config
> or
> > > > > network
> > > > > > > > > > > connectivity
> > > > > > > > > > > > > > issues before pushing the first message through.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Should there also be a KafkaProducer.connect() or
> > > > .open()
> > > > > > > > method
> > > > > > > > > or
> > > > > > > > > > > > > > connectAll()?  I guess it would try to connect to
> > all
> > > > > > brokers
> > > > > > > > in
> > > > > > > > > > the
> > > > > > > > > > > > > > BROKER_LIST_CONFIG
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > HTH,
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Roger
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On Fri, Jan 24, 2014 at 11:54 AM, Jay Kreps <
> > > > > > > > jay.kreps@gmail.com
> > > > > > > > > >
> > > > > > > > > > > > wrote:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > > As mentioned in a previous email we are working
> > on
> > > a
> > > > > > > > > > > > re-implementation
> > > > > > > > > > > > > of
> > > > > > > > > > > > > > > the producer. I would like to use this email
> > thread
> > > > to
> > > > > > > > discuss
> > > > > > > > > > the
> > > > > > > > > > > > > > details
> > > > > > > > > > > > > > > of the public API and the configuration. I
> would
> > > love
> > > > > for
> > > > > > > us
> > > > > > > > to
> > > > > > > > > > be
> > > > > > > > > > > > > > > incredibly picky about this public api now so
> it
> > is
> > > > as
> > > > > > good
> > > > > > > > as
> > > > > > > > > > > > possible
> > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > we don't need to break it in the future.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > The best way to get a feel for the API is
> > actually
> > > to
> > > > > > take
> > > > > > > a
> > > > > > > > > look
> > > > > > > > > > > at
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > javadoc, my hope is to get the api docs good
> > enough
> > > > so
> > > > > > that
> > > > > > > > it
> > > > > > > > > is
> > > > > > > > > > > > > > > self-explanatory:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> http://empathybox.com/kafka-javadoc/index.html?kafka/clients/producer/KafkaProducer.html
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Please take a look at this API and give me any
> > > > thoughts
> > > > > > you
> > > > > > > > may
> > > > > > > > > > > have!
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > It may also be reasonable to take a look at the
> > > > > configs:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> http://empathybox.com/kafka-javadoc/kafka/clients/producer/ProducerConfig.html
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > The actual code is posted here:
> > > > > > > > > > > > > > >
> https://issues.apache.org/jira/browse/KAFKA-1227
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > A few questions or comments to kick things off:
> > > > > > > > > > > > > > > 1. We need to make a decision on whether
> > > > serialization
> > > > > of
> > > > > > > the
> > > > > > > > > > > user's
> > > > > > > > > > > > > key
> > > > > > > > > > > > > > > and value should be done by the user (with our
> > api
> > > > just
> > > > > > > > taking
> > > > > > > > > > > > byte[])
> > > > > > > > > > > > > or
> > > > > > > > > > > > > > > if we should take an object and allow the user
> to
> > > > > > > configure a
> > > > > > > > > > > > > Serializer
> > > > > > > > > > > > > > > class which we instantiate via reflection. We
> > take
> > > > the
> > > > > > > later
> > > > > > > > > > > approach
> > > > > > > > > > > > > in
> > > > > > > > > > > > > > > the current producer, and I have carried this
> > > through
> > > > > to
> > > > > > > this
> > > > > > > > > > > > > prototype.
> > > > > > > > > > > > > > > The tradeoff I see is this: taking byte[] is
> > > actually
> > > > > > > > simpler,
> > > > > > > > > > the
> > > > > > > > > > > > user
> > > > > > > > > > > > > > can
> > > > > > > > > > > > > > > directly do whatever serialization they like.
> The
> > > > > > > > complication
> > > > > > > > > is
> > > > > > > > > > > > > > actually
> > > > > > > > > > > > > > > partitioning. Currently partitioning is done
> by a
> > > > > similar
> > > > > > > > > plug-in
> > > > > > > > > > > api
> > > > > > > > > > > > > > > (Partitioner) which the user can implement and
> > > > > configure
> > > > > > to
> > > > > > > > > > > override
> > > > > > > > > > > > > how
> > > > > > > > > > > > > > > partitions are assigned. If we take byte[] as
> > input
> > > > > then
> > > > > > we
> > > > > > > > > have
> > > > > > > > > > no
> > > > > > > > > > > > > > access
> > > > > > > > > > > > > > > to the original object and partitioning MUST be
> > > done
> > > > on
> > > > > > the
> > > > > > > > > > byte[].
> > > > > > > > > > > > > This
> > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > fine for hash partitioning. However for various
> > > types
> > > > > of
> > > > > > > > > semantic
> > > > > > > > > > > > > > > partitioning (range partitioning, or whatever)
> > you
> > > > > would
> > > > > > > want
> > > > > > > > > > > access
> > > > > > > > > > > > to
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > original object. In the current approach a
> > producer
> > > > who
> > > > > > > > wishes
> > > > > > > > > to
> > > > > > > > > > > > send
> > > > > > > > > > > > > > > byte[] they have serialized in their own code
> can
> > > > > > configure
> > > > > > > > the
> > > > > > > > > > > > > > > BytesSerialization we supply which is just a
> "no
> > > op"
> > > > > > > > > > serialization.
> > > > > > > > > > > > > > > 2. We should obsess over naming and make sure
> > each
> > > of
> > > > > the
> > > > > > > > class
> > > > > > > > > > > names
> > > > > > > > > > > > > are
> > > > > > > > > > > > > > > good.
> > > > > > > > > > > > > > > 3. Jun has already pointed out that we need to
> > > > include
> > > > > > the
> > > > > > > > > topic
> > > > > > > > > > > and
> > > > > > > > > > > > > > > partition in the response, which is absolutely
> > > > right. I
> > > > > > > > haven't
> > > > > > > > > > > done
> > > > > > > > > > > > > that
> > > > > > > > > > > > > > > yet but that definitely needs to be there.
> > > > > > > > > > > > > > > 4. Currently RecordSend.await will throw an
> > > exception
> > > > > if
> > > > > > > the
> > > > > > > > > > > request
> > > > > > > > > > > > > > > failed. The intention here is that
> > > > > > > > > producer.send(message).await()
> > > > > > > > > > > > > exactly
> > > > > > > > > > > > > > > simulates a synchronous call. Guozhang has
> noted
> > > that
> > > > > > this
> > > > > > > > is a
> > > > > > > > > > > > little
> > > > > > > > > > > > > > > annoying since the user must then catch
> > exceptions.
> > > > > > However
> > > > > > > > if
> > > > > > > > > we
> > > > > > > > > > > > > remove
> > > > > > > > > > > > > > > this then if the user doesn't check for errors
> > they
> > > > > won't
> > > > > > > > know
> > > > > > > > > > one
> > > > > > > > > > > > has
> > > > > > > > > > > > > > > occurred, which I predict will be a common
> > mistake.
> > > > > > > > > > > > > > > 5. Perhaps there is more we could do to make
> the
> > > > async
> > > > > > > > > callbacks
> > > > > > > > > > > and
> > > > > > > > > > > > > > future
> > > > > > > > > > > > > > > we give back intuitive and easy to program
> > against?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Some background info on implementation:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > At a high level the primary difference in this
> > > > producer
> > > > > > is
> > > > > > > > that
> > > > > > > > > > it
> > > > > > > > > > > > > > removes
> > > > > > > > > > > > > > > the distinction between the "sync" and "async"
> > > > > producer.
> > > > > > > > > > > Effectively
> > > > > > > > > > > > > all
> > > > > > > > > > > > > > > requests are sent asynchronously but always
> > return
> > > a
> > > > > > future
> > > > > > > > > > > response
> > > > > > > > > > > > > > object
> > > > > > > > > > > > > > > that gives the offset as well as any error that
> > may
> > > > > have
> > > > > > > > > occurred
> > > > > > > > > > > > when
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > request is complete. The batching that is done
> in
> > > the
> > > > > > async
> > > > > > > > > > > producer
> > > > > > > > > > > > > only
> > > > > > > > > > > > > > > today is done whenever possible now. This means
> > > that
> > > > > the
> > > > > > > sync
> > > > > > > > > > > > producer,
> > > > > > > > > > > > > > > under load, can get performance as good as the
> > > async
> > > > > > > producer
> > > > > > > > > > > > > > (preliminary
> > > > > > > > > > > > > > > results show the producer getting 1m
> > messages/sec).
> > > > > This
> > > > > > > > works
> > > > > > > > > > > > similar
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > group commit in databases but with respect to
> the
> > > > > actual
> > > > > > > > > network
> > > > > > > > > > > > > > > transmission--any messages that arrive while a
> > send
> > > > is
> > > > > in
> > > > > > > > > > progress
> > > > > > > > > > > > are
> > > > > > > > > > > > > > > batched together. It is also possible to
> > encourage
> > > > > > batching
> > > > > > > > > even
> > > > > > > > > > > > under
> > > > > > > > > > > > > > low
> > > > > > > > > > > > > > > load to save server resources by introducing a
> > > delay
> > > > on
> > > > > > the
> > > > > > > > > send
> > > > > > > > > > to
> > > > > > > > > > > > > allow
> > > > > > > > > > > > > > > more messages to accumulate; this is done using
> > the
> > > > > > > > > > linger.msconfig
> > > > > > > > > > > > > > (this
> > > > > > > > > > > > > > > is similar to Nagle's algorithm in TCP).
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > This producer does all network communication
> > > > > > asynchronously
> > > > > > > > and
> > > > > > > > > > in
> > > > > > > > > > > > > > parallel
> > > > > > > > > > > > > > > to all servers so the performance penalty for
> > > acks=-1
> > > > > and
> > > > > > > > > waiting
> > > > > > > > > > > on
> > > > > > > > > > > > > > > replication should be much reduced. I haven't
> > done
> > > > much
> > > > > > > > > > > benchmarking
> > > > > > > > > > > > on
> > > > > > > > > > > > > > > this yet, though.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > The high level design is described a little
> here,
> > > > > though
> > > > > > > this
> > > > > > > > > is
> > > > > > > > > > > now
> > > > > > > > > > > > a
> > > > > > > > > > > > > > > little out of date:
> > > > > > > > > > > > > > >
> > > > > > > > >
> > > https://cwiki.apache.org/confluence/display/KAFKA/Client+Rewrite
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > -Jay
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: New Producer Public API

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

Can you elaborate on why you prefer using Java's Future? The downside in my
mind is the use of the checked InterruptedException and ExecutionException.
ExecutionException is arguable, but forcing you to catch
InterruptedException, often in code that can't be interrupted, seems
perverse. It also leaves us with the cancel() method which I don't think we
really can implement.

Option 1A, to recap/elaborate, was the following. There is no Serializer or
Partitioner api. We take a byte[] key and value and an optional integer
partition. If you specify the integer partition it will be used. If you do
not specify a key or a partition the partition will be chosen in a round
robin fashion. If you specify a key but no partition we will chose a
partition based on a hash of the key. In order to let the user find the
partition we will need to given them access to the Cluster instance
directly from the producer.

-Jay


On Tue, Jan 28, 2014 at 6:25 PM, Neha Narkhede <ne...@gmail.com>wrote:

> Here are more thoughts on the public APIs -
>
> - I suggest we use java's Future instead of custom Future especially since
> it is part of the public API
>
> - Serialization: I like the simplicity of the producer APIs with the
> absence of serialization where we just deal with byte arrays for keys and
> values. What I don't like about this is the performance overhead on the
> Partitioner for any kind of custom partitioning based on the partitionKey.
> Since the only purpose of partitionKey is to do custom partitioning, why
> can't we take it in directly as an integer and let the user figure out the
> mapping from partition_key -> partition_id using the getCluster() API? If I
> understand correctly, this is similar to what you suggested as part of
> option 1A. I like this approach since it maintains the simplicity of APIs
> by allowing us to deal with bytes and does not compromise performance in
> the custom partitioning case.
>
> Thanks,
> Neha
>
>
>
> On Tue, Jan 28, 2014 at 5:42 PM, Jay Kreps <ja...@gmail.com> wrote:
>
> > Hey Tom,
> >
> > That sounds cool. How did you end up handling parallel I/O if you wrap
> the
> > individual connections? Don't you need some selector that selects over
> all
> > the connections?
> >
> > -Jay
> >
> >
> > On Tue, Jan 28, 2014 at 2:31 PM, Tom Brown <to...@gmail.com> wrote:
> >
> > > I implemented a 0.7 client in pure java, and its API very closely
> > resembled
> > > this. (When multiple people independently engineer the same solution,
> > it's
> > > probably good... right?). However, there were a few architectural
> > > differences with my client:
> > >
> > > 1. The basic client itself was just an asynchronous layer around the
> > > different server functions. In and of itself it had no knowledge of
> > > partitions, only servers (and maintained TCP connections to them).
> > >
> > > 2. The main producer was an additional layer that provided a high-level
> > > interface that could batch individual messages based on partition.
> > >
> > > 3. Knowledge of partitioning was done via an interface so that
> different
> > > strategies could be used.
> > >
> > > 4. Partitioning was done by the user, with knowledge of the available
> > > partitions provided by #3.
> > >
> > > 5. Serialization was done by the user to simplify the API.
> > >
> > > 6. Futures were used to make asynchronous emulate synchronous calls.
> > >
> > >
> > > The main benefit of this approach is flexibility. For example, since
> the
> > > base client was just a managed connection (and not inherently a
> > producer),
> > > it was easy to composite a produce request and an offsets request
> > together
> > > into a confirmed produce request (officially not available in 0.7).
> > >
> > > Decoupling the basic client from partition management allowed the me to
> > > implement zk discovery as a separate project so that the main project
> had
> > > no complex dependencies. The same was true of decoupling serialization.
> > > It's trivial to build an optional layer that adds those features in,
> > while
> > > allowing access to the base APIs for those that need it.
> > >
> > > Using standard Future objects was also beneficial, since I could
> combine
> > > them with existing tools (such as guava).
> > >
> > > It may be too late to be of use, but I have been working with my
> > company's
> > > legal department to release the implementation I described above. If
> > you're
> > > interested in it, let me know.
> > >
> > >
> > > To sum up my thoughts regarding the new API, I think it's a great
> start.
> > I
> > > would like to see a more layered approach so I can use the parts I
> want,
> > > and adapt the other parts as needed. I would also like to see standard
> > > interfaces (especially Future) used where they makes sense.
> > >
> > > --Tom
> > >
> > >
> > >
> > >
> > >
> > > On Tue, Jan 28, 2014 at 1:33 PM, Roger Hoover <roger.hoover@gmail.com
> > > >wrote:
> > >
> > > > +1 ListenableFuture: If this works similar to Deferreds in Twisted
> > Python
> > > > or Promised IO in Javascript, I think this is a great pattern for
> > > > decoupling your callback logic from the place where the Future is
> > > > generated.  You can register as many callbacks as you like, each in
> the
> > > > appropriate layer of the code and have each observer get notified
> when
> > > the
> > > > promised i/o is complete without any of them knowing about each
> other.
> > > >
> > > >
> > > > On Tue, Jan 28, 2014 at 11:32 AM, Jay Kreps <ja...@gmail.com>
> > wrote:
> > > >
> > > > > Hey Ross,
> > > > >
> > > > > - ListenableFuture: Interesting. That would be an alternative to
> the
> > > > direct
> > > > > callback support we provide. There could be pros to this, let me
> > think
> > > > > about it.
> > > > > - We could provide layering, but I feel that the serialization is
> > such
> > > a
> > > > > small thing we should just make a decision and chose one, it
> doesn't
> > > seem
> > > > > to me to justify a whole public facing layer.
> > > > > - Yes, this is fairly esoteric, essentially I think it is fairly
> > > similar
> > > > to
> > > > > databases like DynamoDB that allow you to specify two partition
> keys
> > (I
> > > > > think DynamoDB does this...). The reasoning is that in fact there
> are
> > > > > several things you can use the key field for: (1) to compute the
> > > > partition
> > > > > to store the data in, (2) as a unique identifier to deduplicate
> that
> > > > > partition's records within a log. These two things are almost
> always
> > > the
> > > > > same, but occationally may differ when you want to group data in a
> > more
> > > > > sophisticated way then just a hash of the primary key but still
> > retain
> > > > the
> > > > > proper primary key for delivery to the consumer and log compaction.
> > > > >
> > > > > -Jay
> > > > >
> > > > >
> > > > > On Tue, Jan 28, 2014 at 3:24 AM, Ross Black <
> ross.w.black@gmail.com>
> > > > > wrote:
> > > > >
> > > > > > Hi Jay,
> > > > > >
> > > > > > - Just to add some more info/confusion about possibly using
> Future
> > > ...
> > > > > >   If Kafka uses a JDK future, it plays nicely with other
> frameworks
> > > as
> > > > > > well.
> > > > > >   Google Guava has a ListenableFuture that allows callback
> handling
> > > to
> > > > be
> > > > > > added via the returned future, and allows the callbacks to be
> > passed
> > > > off
> > > > > to
> > > > > > a specified executor.
> > > > > >
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> http://docs.guava-libraries.googlecode.com/git-history/release/javadoc/com/google/common/util/concurrent/ListenableFuture.html
> > > > > >   The JDK future can easily be converted to a listenable future.
> > > > > >
> > > > > > - On the question of byte[] vs Object, could this be solved by
> > > layering
> > > > > the
> > > > > > API?  eg. a raw producer (use byte[] and specify the partition
> > > number)
> > > > > and
> > > > > > a normal producer (use generic object and specify a Partitioner)?
> > > > > >
> > > > > > - I am confused by the keys in ProducerRecord and Partitioner.
> >  What
> > > is
> > > > > the
> > > > > > usage for both a key and a partition key? (I am not yet using
> 0.8)
> > > > > >
> > > > > >
> > > > > > Thanks,
> > > > > > Ross
> > > > > >
> > > > > >
> > > > > >
> > > > > > On 28 January 2014 05:00, Xavier Stevens <xa...@gaikai.com>
> > wrote:
> > > > > >
> > > > > > > AutoCloseable would be nice for us as most of our code is using
> > > Java
> > > > 7
> > > > > at
> > > > > > > this point.
> > > > > > >
> > > > > > > I like Dropwizard's configuration mapping to POJOs via Jackson,
> > but
> > > > if
> > > > > > you
> > > > > > > wanted to stick with property maps I don't care enough to
> object.
> > > > > > >
> > > > > > > If the producer only dealt with bytes, is there a way we could
> > > still
> > > > > due
> > > > > > > partition plugins without specifying the number explicitly? I
> > would
> > > > > > prefer
> > > > > > > to be able to pass in field(s) that would be used by the
> > > partitioner.
> > > > > > > Obviously if this wasn't possible you could always deserialize
> > the
> > > > > object
> > > > > > > in the partitioner and grab the fields you want, but that seems
> > > > really
> > > > > > > expensive to do on every message.
> > > > > > >
> > > > > > > It would also be nice to have a Java API Encoder constructor
> > taking
> > > > in
> > > > > > > VerifiableProperties. Scala understands how to handle "props:
> > > > > > > VerifiableProperties = null", but Java doesn't. So you don't
> run
> > > into
> > > > > > this
> > > > > > > problem until runtime.
> > > > > > >
> > > > > > >
> > > > > > > -Xavier
> > > > > > >
> > > > > > >
> > > > > > > On Mon, Jan 27, 2014 at 9:37 AM, Clark Breyman <
> > clark@breyman.com>
> > > > > > wrote:
> > > > > > >
> > > > > > > > Jay -
> > > > > > > >
> > > > > > > > Config - your explanation makes sense. I'm just so accustomed
> > to
> > > > > having
> > > > > > > > Jackson automatically map my configuration objects to POJOs
> > that
> > > > I've
> > > > > > > > stopped using property files. They are lingua franca. The
> only
> > > > > thought
> > > > > > > > might be to separate the config interface from the
> > implementation
> > > > to
> > > > > > > allow
> > > > > > > > for alternatives, but that might undermine your point of "do
> it
> > > > this
> > > > > > way
> > > > > > > so
> > > > > > > > that everyone can find it where they expect it".
> > > > > > > >
> > > > > > > > Serialization: Of the options, I like 1A the best, though
> > > possibly
> > > > > with
> > > > > > > > either an option to specify a partition key rather than ID
> or a
> > > > > helper
> > > > > > to
> > > > > > > > translate an arbitrary byte[] or long into a partition
> number.
> > > > > > > >
> > > > > > > > Thanks
> > > > > > > > Clark
> > > > > > > >
> > > > > > > >
> > > > > > > > On Sun, Jan 26, 2014 at 9:13 PM, Jay Kreps <
> > jay.kreps@gmail.com>
> > > > > > wrote:
> > > > > > > >
> > > > > > > > > Thanks for the detailed thoughts. Let me elaborate on the
> > > config
> > > > > > thing.
> > > > > > > > >
> > > > > > > > > I agree that at first glance key-value strings don't seem
> > like
> > > a
> > > > > very
> > > > > > > > good
> > > > > > > > > configuration api for a client. Surely a well-typed config
> > > class
> > > > > > would
> > > > > > > be
> > > > > > > > > better! I actually disagree and let me see if I can
> convince
> > > you.
> > > > > > > > >
> > > > > > > > > My reasoning has nothing to do with the api and everything
> to
> > > do
> > > > > with
> > > > > > > > > operations.
> > > > > > > > >
> > > > > > > > > Clients are embedded in applications which are themselves
> > > > > configured.
> > > > > > > In
> > > > > > > > > any place that takes operations seriously the configuration
> > for
> > > > > these
> > > > > > > > > applications will be version controlled and maintained
> > through
> > > > some
> > > > > > > kind
> > > > > > > > of
> > > > > > > > > config management system. If we give a config class with
> > > getters
> > > > > and
> > > > > > > > > setters the application has to expose those properties to
> its
> > > > > > > > > configuration. What invariably happens is that the
> > application
> > > > > > exposes
> > > > > > > > only
> > > > > > > > > a choice few properties that they thought they would
> change.
> > > > > > > Furthermore
> > > > > > > > > the application will make up a name for these configs that
> > > seems
> > > > > > > > intuitive
> > > > > > > > > at the time in the 2 seconds the engineer spends thinking
> > about
> > > > it.
> > > > > > > > >
> > > > > > > > > Now consider the result of this in the large. You end up
> with
> > > > > dozens
> > > > > > or
> > > > > > > > > hundreds of applications that have the client embedded.
> Each
> > > > > exposes
> > > > > > a
> > > > > > > > > different, inadequate subset of the possible configs, each
> > with
> > > > > > > different
> > > > > > > > > names. It is a nightmare.
> > > > > > > > >
> > > > > > > > > If you use a string-string map the config system can
> directly
> > > > get a
> > > > > > > > bundle
> > > > > > > > > of config key-value pairs and put them into the client.
> This
> > > > means
> > > > > > that
> > > > > > > > all
> > > > > > > > > configuration is automatically available with the name
> > > documented
> > > > > on
> > > > > > > the
> > > > > > > > > website in every application that does this. If you upgrade
> > to
> > > a
> > > > > new
> > > > > > > > kafka
> > > > > > > > > version with more configs those will be exposed too. If you
> > > > realize
> > > > > > > that
> > > > > > > > > you need to change a default you can just go through your
> > > configs
> > > > > and
> > > > > > > > > change it everywhere as it will have the same name
> > everywhere.
> > > > > > > > >
> > > > > > > > > -Jay
> > > > > > > > >
> > > > > > > > >
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > On Sun, Jan 26, 2014 at 4:47 PM, Clark Breyman <
> > > > clark@breyman.com>
> > > > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > Thanks Jay. I'll see if I can put together a more
> complete
> > > > > > response,
> > > > > > > > > > perhaps as separate threads so that topics don't get
> > > entangled.
> > > > > In
> > > > > > > the
> > > > > > > > > mean
> > > > > > > > > > time, here's a couple responses:
> > > > > > > > > >
> > > > > > > > > > Serialization: you've broken out a sub-thread so i'll
> reply
> > > > > there.
> > > > > > My
> > > > > > > > > bias
> > > > > > > > > > is that I like generics (except for type-erasure) and in
> > > > > particular
> > > > > > > > they
> > > > > > > > > > make it easy to compose serializers for compound payloads
> > > (e.g.
> > > > > > when
> > > > > > > a
> > > > > > > > > > common header wraps a payload of parameterized type).
> I'll
> > > > > respond
> > > > > > to
> > > > > > > > > your
> > > > > > > > > > 4-options message with an example.
> > > > > > > > > >
> > > > > > > > > > Build: I've seen a lot of "maven-compatible" build
> systems
> > > > > produce
> > > > > > > > > > "artifacts" that aren't really artifacts - no embedded
> POM
> > > or,
> > > > > > worst,
> > > > > > > > > > malformed POM. I know the sbt-generated artifacts were
> this
> > > > way -
> > > > > > > onus
> > > > > > > > is
> > > > > > > > > > on me to see what gradle is spitting out and what a maven
> > > build
> > > > > > might
> > > > > > > > > look
> > > > > > > > > > like. Maven may be old and boring, but it gets out of the
> > way
> > > > and
> > > > > > > > > > integrates really seamlessly with a lot of IDEs. When
> some
> > > > scala
> > > > > > > > > projects I
> > > > > > > > > > was working on in the fall of 2011 switched from sbt to
> > > maven,
> > > > > > build
> > > > > > > > > became
> > > > > > > > > > a non-issue.
> > > > > > > > > >
> > > > > > > > > > Config: Not a big deal  and no, I don't think a
> dropwizard
> > > > > > dependency
> > > > > > > > is
> > > > > > > > > > appropriate. I do like using simple entity beans (POJO's
> > not
> > > > > j2EE)
> > > > > > > for
> > > > > > > > > > configuration, especially if they can be marshalled
> without
> > > > > > > annotation
> > > > > > > > by
> > > > > > > > > > Jackson. I only mentioned the dropwizard-extras  because
> it
> > > has
> > > > > > some
> > > > > > > > > entity
> > > > > > > > > > bean versions of the ZK and Kafka configs.
> > > > > > > > > >
> > > > > > > > > > Domain-packaging: Also not a big deal - it's what's
> > expected
> > > > and
> > > > > > it's
> > > > > > > > > > pretty free in most IDE's. The advantages I see is that
> it
> > is
> > > > > clear
> > > > > > > > > whether
> > > > > > > > > > something is from the Apache Kafka project and whether
> > > > something
> > > > > is
> > > > > > > > from
> > > > > > > > > > another org and related to Kafka. That said, nothing
> really
> > > > > > enforces
> > > > > > > > it.
> > > > > > > > > >
> > > > > > > > > > Futures: I'll see if I can create some examples to
> > > demonstrate
> > > > > > Future
> > > > > > > > > > making interop easier.
> > > > > > > > > >
> > > > > > > > > > Regards,
> > > > > > > > > > C
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > On Fri, Jan 24, 2014 at 4:36 PM, Jay Kreps <
> > > > jay.kreps@gmail.com>
> > > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > Hey Clark,
> > > > > > > > > > >
> > > > > > > > > > > - Serialization: Yes I agree with these though I don't
> > > > consider
> > > > > > the
> > > > > > > > > loss
> > > > > > > > > > of
> > > > > > > > > > > generics a big issue. I'll try to summarize what I
> would
> > > > > consider
> > > > > > > the
> > > > > > > > > > best
> > > > > > > > > > > alternative api with raw byte[].
> > > > > > > > > > >
> > > > > > > > > > > - Maven: We had this debate a few months back and the
> > > > consensus
> > > > > > was
> > > > > > > > > > gradle.
> > > > > > > > > > > Is there a specific issue with the poms gradle makes? I
> > am
> > > > > > > extremely
> > > > > > > > > > loath
> > > > > > > > > > > to revisit the issue as build issues are a recurring
> > thing
> > > > and
> > > > > no
> > > > > > > one
> > > > > > > > > > ever
> > > > > > > > > > > agrees and ultimately our build needs are very simple.
> > > > > > > > > > >
> > > > > > > > > > > - Config: I'm not sure if I follow the point. Are you
> > > saying
> > > > we
> > > > > > > > should
> > > > > > > > > > use
> > > > > > > > > > > something in dropwizard for config? One principle here
> is
> > > to
> > > > > try
> > > > > > to
> > > > > > > > > > remove
> > > > > > > > > > > as many client dependencies as possible as we
> inevitably
> > > run
> > > > > into
> > > > > > > > > > terrible
> > > > > > > > > > > compatibility issues with users who use the same
> library
> > or
> > > > its
> > > > > > > > > > > dependencies at different versions. Or are you talking
> > > about
> > > > > > > > > maintaining
> > > > > > > > > > > compatibility with existing config parameters? I think
> as
> > > > much
> > > > > > as a
> > > > > > > > > > config
> > > > > > > > > > > in the existing client makes sense it should have the
> > same
> > > > name
> > > > > > (I
> > > > > > > > was
> > > > > > > > > a
> > > > > > > > > > > bit sloppy about that so I'll fix any errors there).
> > There
> > > > are
> > > > > a
> > > > > > > few
> > > > > > > > > new
> > > > > > > > > > > things and we should give those reasonable defaults. I
> > > think
> > > > > > config
> > > > > > > > is
> > > > > > > > > > > important so I'll start a thread on the config package
> in
> > > > > there.
> > > > > > > > > > >
> > > > > > > > > > > - org.apache.kafka: We could do this. I always
> considered
> > > it
> > > > > kind
> > > > > > > of
> > > > > > > > an
> > > > > > > > > > odd
> > > > > > > > > > > thing Java programmers do that has no real motivation
> > (but
> > > I
> > > > > > could
> > > > > > > be
> > > > > > > > > > > re-educated!). I don't think it ends up reducing naming
> > > > > conflicts
> > > > > > > in
> > > > > > > > > > > practice and it adds a lot of noise and nested
> > directories.
> > > > Is
> > > > > > > there
> > > > > > > > a
> > > > > > > > > > > reason you prefer this or just to be more standard?
> > > > > > > > > > >
> > > > > > > > > > > - Future: Basically I didn't see any particular
> > advantage.
> > > > The
> > > > > > > > cancel()
> > > > > > > > > > > method doesn't really make sense so probably wouldn't
> > work.
> > > > > > > Likewise
> > > > > > > > I
> > > > > > > > > > > dislike the checked exceptions it requires. Basically I
> > > just
> > > > > > wrote
> > > > > > > > out
> > > > > > > > > > some
> > > > > > > > > > > code examples and it seemed cleaner with a special
> > purpose
> > > > > > object.
> > > > > > > I
> > > > > > > > > > wasn't
> > > > > > > > > > > actually aware of plans for improved futures in java 8
> or
> > > the
> > > > > > other
> > > > > > > > > > > integrations. Maybe you could elaborate on this a bit
> and
> > > > show
> > > > > > how
> > > > > > > it
> > > > > > > > > > would
> > > > > > > > > > > be used? Sounds promising, I just don't know a lot
> about
> > > it.
> > > > > > > > > > >
> > > > > > > > > > > -Jay
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > On Fri, Jan 24, 2014 at 3:30 PM, Clark Breyman <
> > > > > > clark@breyman.com>
> > > > > > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > Jay - Thanks for the call for comments. Here's some
> > > initial
> > > > > > > input:
> > > > > > > > > > > >
> > > > > > > > > > > > - Make message serialization a client responsibility
> > > > (making
> > > > > > all
> > > > > > > > > > messages
> > > > > > > > > > > > byte[]). Reflection-based loading makes it harder to
> > use
> > > > > > generic
> > > > > > > > > codecs
> > > > > > > > > > > > (e.g.  Envelope<PREFIX, DATA, SUFFIX>) or build up
> > codec
> > > > > > > > > > > programmatically.
> > > > > > > > > > > > Non-default partitioning should require an explicit
> > > > partition
> > > > > > > key.
> > > > > > > > > > > >
> > > > > > > > > > > > - I really like the fact that it will be native Java.
> > > > Please
> > > > > > > > consider
> > > > > > > > > > > using
> > > > > > > > > > > > native maven and not sbt, gradle, ivy, etc as they
> > don't
> > > > > > reliably
> > > > > > > > > play
> > > > > > > > > > > nice
> > > > > > > > > > > > in the maven ecosystem. A jar without a well-formed
> pom
> > > > > doesn't
> > > > > > > > feel
> > > > > > > > > > > like a
> > > > > > > > > > > > real artifact. The pom's generated by sbt et al. are
> > not
> > > > well
> > > > > > > > formed.
> > > > > > > > > > > Using
> > > > > > > > > > > > maven will make builds and IDE integration much
> > smoother.
> > > > > > > > > > > >
> > > > > > > > > > > > - Look at Nick Telford's dropwizard-extras package in
> > > which
> > > > > he
> > > > > > > > > defines
> > > > > > > > > > > some
> > > > > > > > > > > > Jackson-compatible POJO's for loading configuration.
> > > Seems
> > > > > like
> > > > > > > > your
> > > > > > > > > > > client
> > > > > > > > > > > > migration is similar. The config objects should have
> > > > > > constructors
> > > > > > > > or
> > > > > > > > > > > > factories that accept Map<String, String> and
> > Properties
> > > > for
> > > > > > ease
> > > > > > > > of
> > > > > > > > > > > > migration.
> > > > > > > > > > > >
> > > > > > > > > > > > - Would you consider using the org.apache.kafka
> package
> > > for
> > > > > the
> > > > > > > new
> > > > > > > > > API
> > > > > > > > > > > > (quibble)
> > > > > > > > > > > >
> > > > > > > > > > > > - Why create your own futures rather than use
> > > > > > > > > > > > java.util.concurrent.Future<Long> or similar?
> Standard
> > > > > futures
> > > > > > > will
> > > > > > > > > > play
> > > > > > > > > > > > nice with other reactive libs and things like J8's
> > > > > > > > ComposableFuture.
> > > > > > > > > > > >
> > > > > > > > > > > > Thanks again,
> > > > > > > > > > > > C
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > On Fri, Jan 24, 2014 at 2:46 PM, Roger Hoover <
> > > > > > > > > roger.hoover@gmail.com
> > > > > > > > > > > > >wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > > A couple comments:
> > > > > > > > > > > > >
> > > > > > > > > > > > > 1) Why does the config use a broker list instead of
> > > > > > discovering
> > > > > > > > the
> > > > > > > > > > > > brokers
> > > > > > > > > > > > > in ZooKeeper?  It doesn't match the
> HighLevelConsumer
> > > > API.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 2) It looks like broker connections are created on
> > > > demand.
> > > > > >  I'm
> > > > > > > > > > > wondering
> > > > > > > > > > > > > if sometimes you might want to flush out config or
> > > > network
> > > > > > > > > > connectivity
> > > > > > > > > > > > > issues before pushing the first message through.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Should there also be a KafkaProducer.connect() or
> > > .open()
> > > > > > > method
> > > > > > > > or
> > > > > > > > > > > > > connectAll()?  I guess it would try to connect to
> all
> > > > > brokers
> > > > > > > in
> > > > > > > > > the
> > > > > > > > > > > > > BROKER_LIST_CONFIG
> > > > > > > > > > > > >
> > > > > > > > > > > > > HTH,
> > > > > > > > > > > > >
> > > > > > > > > > > > > Roger
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Fri, Jan 24, 2014 at 11:54 AM, Jay Kreps <
> > > > > > > jay.kreps@gmail.com
> > > > > > > > >
> > > > > > > > > > > wrote:
> > > > > > > > > > > > >
> > > > > > > > > > > > > > As mentioned in a previous email we are working
> on
> > a
> > > > > > > > > > > re-implementation
> > > > > > > > > > > > of
> > > > > > > > > > > > > > the producer. I would like to use this email
> thread
> > > to
> > > > > > > discuss
> > > > > > > > > the
> > > > > > > > > > > > > details
> > > > > > > > > > > > > > of the public API and the configuration. I would
> > love
> > > > for
> > > > > > us
> > > > > > > to
> > > > > > > > > be
> > > > > > > > > > > > > > incredibly picky about this public api now so it
> is
> > > as
> > > > > good
> > > > > > > as
> > > > > > > > > > > possible
> > > > > > > > > > > > > and
> > > > > > > > > > > > > > we don't need to break it in the future.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > The best way to get a feel for the API is
> actually
> > to
> > > > > take
> > > > > > a
> > > > > > > > look
> > > > > > > > > > at
> > > > > > > > > > > > the
> > > > > > > > > > > > > > javadoc, my hope is to get the api docs good
> enough
> > > so
> > > > > that
> > > > > > > it
> > > > > > > > is
> > > > > > > > > > > > > > self-explanatory:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> http://empathybox.com/kafka-javadoc/index.html?kafka/clients/producer/KafkaProducer.html
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Please take a look at this API and give me any
> > > thoughts
> > > > > you
> > > > > > > may
> > > > > > > > > > have!
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > It may also be reasonable to take a look at the
> > > > configs:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> http://empathybox.com/kafka-javadoc/kafka/clients/producer/ProducerConfig.html
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > The actual code is posted here:
> > > > > > > > > > > > > > https://issues.apache.org/jira/browse/KAFKA-1227
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > A few questions or comments to kick things off:
> > > > > > > > > > > > > > 1. We need to make a decision on whether
> > > serialization
> > > > of
> > > > > > the
> > > > > > > > > > user's
> > > > > > > > > > > > key
> > > > > > > > > > > > > > and value should be done by the user (with our
> api
> > > just
> > > > > > > taking
> > > > > > > > > > > byte[])
> > > > > > > > > > > > or
> > > > > > > > > > > > > > if we should take an object and allow the user to
> > > > > > configure a
> > > > > > > > > > > > Serializer
> > > > > > > > > > > > > > class which we instantiate via reflection. We
> take
> > > the
> > > > > > later
> > > > > > > > > > approach
> > > > > > > > > > > > in
> > > > > > > > > > > > > > the current producer, and I have carried this
> > through
> > > > to
> > > > > > this
> > > > > > > > > > > > prototype.
> > > > > > > > > > > > > > The tradeoff I see is this: taking byte[] is
> > actually
> > > > > > > simpler,
> > > > > > > > > the
> > > > > > > > > > > user
> > > > > > > > > > > > > can
> > > > > > > > > > > > > > directly do whatever serialization they like. The
> > > > > > > complication
> > > > > > > > is
> > > > > > > > > > > > > actually
> > > > > > > > > > > > > > partitioning. Currently partitioning is done by a
> > > > similar
> > > > > > > > plug-in
> > > > > > > > > > api
> > > > > > > > > > > > > > (Partitioner) which the user can implement and
> > > > configure
> > > > > to
> > > > > > > > > > override
> > > > > > > > > > > > how
> > > > > > > > > > > > > > partitions are assigned. If we take byte[] as
> input
> > > > then
> > > > > we
> > > > > > > > have
> > > > > > > > > no
> > > > > > > > > > > > > access
> > > > > > > > > > > > > > to the original object and partitioning MUST be
> > done
> > > on
> > > > > the
> > > > > > > > > byte[].
> > > > > > > > > > > > This
> > > > > > > > > > > > > is
> > > > > > > > > > > > > > fine for hash partitioning. However for various
> > types
> > > > of
> > > > > > > > semantic
> > > > > > > > > > > > > > partitioning (range partitioning, or whatever)
> you
> > > > would
> > > > > > want
> > > > > > > > > > access
> > > > > > > > > > > to
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > original object. In the current approach a
> producer
> > > who
> > > > > > > wishes
> > > > > > > > to
> > > > > > > > > > > send
> > > > > > > > > > > > > > byte[] they have serialized in their own code can
> > > > > configure
> > > > > > > the
> > > > > > > > > > > > > > BytesSerialization we supply which is just a "no
> > op"
> > > > > > > > > serialization.
> > > > > > > > > > > > > > 2. We should obsess over naming and make sure
> each
> > of
> > > > the
> > > > > > > class
> > > > > > > > > > names
> > > > > > > > > > > > are
> > > > > > > > > > > > > > good.
> > > > > > > > > > > > > > 3. Jun has already pointed out that we need to
> > > include
> > > > > the
> > > > > > > > topic
> > > > > > > > > > and
> > > > > > > > > > > > > > partition in the response, which is absolutely
> > > right. I
> > > > > > > haven't
> > > > > > > > > > done
> > > > > > > > > > > > that
> > > > > > > > > > > > > > yet but that definitely needs to be there.
> > > > > > > > > > > > > > 4. Currently RecordSend.await will throw an
> > exception
> > > > if
> > > > > > the
> > > > > > > > > > request
> > > > > > > > > > > > > > failed. The intention here is that
> > > > > > > > producer.send(message).await()
> > > > > > > > > > > > exactly
> > > > > > > > > > > > > > simulates a synchronous call. Guozhang has noted
> > that
> > > > > this
> > > > > > > is a
> > > > > > > > > > > little
> > > > > > > > > > > > > > annoying since the user must then catch
> exceptions.
> > > > > However
> > > > > > > if
> > > > > > > > we
> > > > > > > > > > > > remove
> > > > > > > > > > > > > > this then if the user doesn't check for errors
> they
> > > > won't
> > > > > > > know
> > > > > > > > > one
> > > > > > > > > > > has
> > > > > > > > > > > > > > occurred, which I predict will be a common
> mistake.
> > > > > > > > > > > > > > 5. Perhaps there is more we could do to make the
> > > async
> > > > > > > > callbacks
> > > > > > > > > > and
> > > > > > > > > > > > > future
> > > > > > > > > > > > > > we give back intuitive and easy to program
> against?
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Some background info on implementation:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > At a high level the primary difference in this
> > > producer
> > > > > is
> > > > > > > that
> > > > > > > > > it
> > > > > > > > > > > > > removes
> > > > > > > > > > > > > > the distinction between the "sync" and "async"
> > > > producer.
> > > > > > > > > > Effectively
> > > > > > > > > > > > all
> > > > > > > > > > > > > > requests are sent asynchronously but always
> return
> > a
> > > > > future
> > > > > > > > > > response
> > > > > > > > > > > > > object
> > > > > > > > > > > > > > that gives the offset as well as any error that
> may
> > > > have
> > > > > > > > occurred
> > > > > > > > > > > when
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > request is complete. The batching that is done in
> > the
> > > > > async
> > > > > > > > > > producer
> > > > > > > > > > > > only
> > > > > > > > > > > > > > today is done whenever possible now. This means
> > that
> > > > the
> > > > > > sync
> > > > > > > > > > > producer,
> > > > > > > > > > > > > > under load, can get performance as good as the
> > async
> > > > > > producer
> > > > > > > > > > > > > (preliminary
> > > > > > > > > > > > > > results show the producer getting 1m
> messages/sec).
> > > > This
> > > > > > > works
> > > > > > > > > > > similar
> > > > > > > > > > > > to
> > > > > > > > > > > > > > group commit in databases but with respect to the
> > > > actual
> > > > > > > > network
> > > > > > > > > > > > > > transmission--any messages that arrive while a
> send
> > > is
> > > > in
> > > > > > > > > progress
> > > > > > > > > > > are
> > > > > > > > > > > > > > batched together. It is also possible to
> encourage
> > > > > batching
> > > > > > > > even
> > > > > > > > > > > under
> > > > > > > > > > > > > low
> > > > > > > > > > > > > > load to save server resources by introducing a
> > delay
> > > on
> > > > > the
> > > > > > > > send
> > > > > > > > > to
> > > > > > > > > > > > allow
> > > > > > > > > > > > > > more messages to accumulate; this is done using
> the
> > > > > > > > > linger.msconfig
> > > > > > > > > > > > > (this
> > > > > > > > > > > > > > is similar to Nagle's algorithm in TCP).
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > This producer does all network communication
> > > > > asynchronously
> > > > > > > and
> > > > > > > > > in
> > > > > > > > > > > > > parallel
> > > > > > > > > > > > > > to all servers so the performance penalty for
> > acks=-1
> > > > and
> > > > > > > > waiting
> > > > > > > > > > on
> > > > > > > > > > > > > > replication should be much reduced. I haven't
> done
> > > much
> > > > > > > > > > benchmarking
> > > > > > > > > > > on
> > > > > > > > > > > > > > this yet, though.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > The high level design is described a little here,
> > > > though
> > > > > > this
> > > > > > > > is
> > > > > > > > > > now
> > > > > > > > > > > a
> > > > > > > > > > > > > > little out of date:
> > > > > > > > > > > > > >
> > > > > > > >
> > https://cwiki.apache.org/confluence/display/KAFKA/Client+Rewrite
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > -Jay
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: New Producer Public API

Posted by Neha Narkhede <ne...@gmail.com>.
Here are more thoughts on the public APIs -

- I suggest we use java's Future instead of custom Future especially since
it is part of the public API

- Serialization: I like the simplicity of the producer APIs with the
absence of serialization where we just deal with byte arrays for keys and
values. What I don't like about this is the performance overhead on the
Partitioner for any kind of custom partitioning based on the partitionKey.
Since the only purpose of partitionKey is to do custom partitioning, why
can't we take it in directly as an integer and let the user figure out the
mapping from partition_key -> partition_id using the getCluster() API? If I
understand correctly, this is similar to what you suggested as part of
option 1A. I like this approach since it maintains the simplicity of APIs
by allowing us to deal with bytes and does not compromise performance in
the custom partitioning case.

Thanks,
Neha



On Tue, Jan 28, 2014 at 5:42 PM, Jay Kreps <ja...@gmail.com> wrote:

> Hey Tom,
>
> That sounds cool. How did you end up handling parallel I/O if you wrap the
> individual connections? Don't you need some selector that selects over all
> the connections?
>
> -Jay
>
>
> On Tue, Jan 28, 2014 at 2:31 PM, Tom Brown <to...@gmail.com> wrote:
>
> > I implemented a 0.7 client in pure java, and its API very closely
> resembled
> > this. (When multiple people independently engineer the same solution,
> it's
> > probably good... right?). However, there were a few architectural
> > differences with my client:
> >
> > 1. The basic client itself was just an asynchronous layer around the
> > different server functions. In and of itself it had no knowledge of
> > partitions, only servers (and maintained TCP connections to them).
> >
> > 2. The main producer was an additional layer that provided a high-level
> > interface that could batch individual messages based on partition.
> >
> > 3. Knowledge of partitioning was done via an interface so that different
> > strategies could be used.
> >
> > 4. Partitioning was done by the user, with knowledge of the available
> > partitions provided by #3.
> >
> > 5. Serialization was done by the user to simplify the API.
> >
> > 6. Futures were used to make asynchronous emulate synchronous calls.
> >
> >
> > The main benefit of this approach is flexibility. For example, since the
> > base client was just a managed connection (and not inherently a
> producer),
> > it was easy to composite a produce request and an offsets request
> together
> > into a confirmed produce request (officially not available in 0.7).
> >
> > Decoupling the basic client from partition management allowed the me to
> > implement zk discovery as a separate project so that the main project had
> > no complex dependencies. The same was true of decoupling serialization.
> > It's trivial to build an optional layer that adds those features in,
> while
> > allowing access to the base APIs for those that need it.
> >
> > Using standard Future objects was also beneficial, since I could combine
> > them with existing tools (such as guava).
> >
> > It may be too late to be of use, but I have been working with my
> company's
> > legal department to release the implementation I described above. If
> you're
> > interested in it, let me know.
> >
> >
> > To sum up my thoughts regarding the new API, I think it's a great start.
> I
> > would like to see a more layered approach so I can use the parts I want,
> > and adapt the other parts as needed. I would also like to see standard
> > interfaces (especially Future) used where they makes sense.
> >
> > --Tom
> >
> >
> >
> >
> >
> > On Tue, Jan 28, 2014 at 1:33 PM, Roger Hoover <roger.hoover@gmail.com
> > >wrote:
> >
> > > +1 ListenableFuture: If this works similar to Deferreds in Twisted
> Python
> > > or Promised IO in Javascript, I think this is a great pattern for
> > > decoupling your callback logic from the place where the Future is
> > > generated.  You can register as many callbacks as you like, each in the
> > > appropriate layer of the code and have each observer get notified when
> > the
> > > promised i/o is complete without any of them knowing about each other.
> > >
> > >
> > > On Tue, Jan 28, 2014 at 11:32 AM, Jay Kreps <ja...@gmail.com>
> wrote:
> > >
> > > > Hey Ross,
> > > >
> > > > - ListenableFuture: Interesting. That would be an alternative to the
> > > direct
> > > > callback support we provide. There could be pros to this, let me
> think
> > > > about it.
> > > > - We could provide layering, but I feel that the serialization is
> such
> > a
> > > > small thing we should just make a decision and chose one, it doesn't
> > seem
> > > > to me to justify a whole public facing layer.
> > > > - Yes, this is fairly esoteric, essentially I think it is fairly
> > similar
> > > to
> > > > databases like DynamoDB that allow you to specify two partition keys
> (I
> > > > think DynamoDB does this...). The reasoning is that in fact there are
> > > > several things you can use the key field for: (1) to compute the
> > > partition
> > > > to store the data in, (2) as a unique identifier to deduplicate that
> > > > partition's records within a log. These two things are almost always
> > the
> > > > same, but occationally may differ when you want to group data in a
> more
> > > > sophisticated way then just a hash of the primary key but still
> retain
> > > the
> > > > proper primary key for delivery to the consumer and log compaction.
> > > >
> > > > -Jay
> > > >
> > > >
> > > > On Tue, Jan 28, 2014 at 3:24 AM, Ross Black <ro...@gmail.com>
> > > > wrote:
> > > >
> > > > > Hi Jay,
> > > > >
> > > > > - Just to add some more info/confusion about possibly using Future
> > ...
> > > > >   If Kafka uses a JDK future, it plays nicely with other frameworks
> > as
> > > > > well.
> > > > >   Google Guava has a ListenableFuture that allows callback handling
> > to
> > > be
> > > > > added via the returned future, and allows the callbacks to be
> passed
> > > off
> > > > to
> > > > > a specified executor.
> > > > >
> > > > >
> > > > >
> > > >
> > >
> >
> http://docs.guava-libraries.googlecode.com/git-history/release/javadoc/com/google/common/util/concurrent/ListenableFuture.html
> > > > >   The JDK future can easily be converted to a listenable future.
> > > > >
> > > > > - On the question of byte[] vs Object, could this be solved by
> > layering
> > > > the
> > > > > API?  eg. a raw producer (use byte[] and specify the partition
> > number)
> > > > and
> > > > > a normal producer (use generic object and specify a Partitioner)?
> > > > >
> > > > > - I am confused by the keys in ProducerRecord and Partitioner.
>  What
> > is
> > > > the
> > > > > usage for both a key and a partition key? (I am not yet using 0.8)
> > > > >
> > > > >
> > > > > Thanks,
> > > > > Ross
> > > > >
> > > > >
> > > > >
> > > > > On 28 January 2014 05:00, Xavier Stevens <xa...@gaikai.com>
> wrote:
> > > > >
> > > > > > AutoCloseable would be nice for us as most of our code is using
> > Java
> > > 7
> > > > at
> > > > > > this point.
> > > > > >
> > > > > > I like Dropwizard's configuration mapping to POJOs via Jackson,
> but
> > > if
> > > > > you
> > > > > > wanted to stick with property maps I don't care enough to object.
> > > > > >
> > > > > > If the producer only dealt with bytes, is there a way we could
> > still
> > > > due
> > > > > > partition plugins without specifying the number explicitly? I
> would
> > > > > prefer
> > > > > > to be able to pass in field(s) that would be used by the
> > partitioner.
> > > > > > Obviously if this wasn't possible you could always deserialize
> the
> > > > object
> > > > > > in the partitioner and grab the fields you want, but that seems
> > > really
> > > > > > expensive to do on every message.
> > > > > >
> > > > > > It would also be nice to have a Java API Encoder constructor
> taking
> > > in
> > > > > > VerifiableProperties. Scala understands how to handle "props:
> > > > > > VerifiableProperties = null", but Java doesn't. So you don't run
> > into
> > > > > this
> > > > > > problem until runtime.
> > > > > >
> > > > > >
> > > > > > -Xavier
> > > > > >
> > > > > >
> > > > > > On Mon, Jan 27, 2014 at 9:37 AM, Clark Breyman <
> clark@breyman.com>
> > > > > wrote:
> > > > > >
> > > > > > > Jay -
> > > > > > >
> > > > > > > Config - your explanation makes sense. I'm just so accustomed
> to
> > > > having
> > > > > > > Jackson automatically map my configuration objects to POJOs
> that
> > > I've
> > > > > > > stopped using property files. They are lingua franca. The only
> > > > thought
> > > > > > > might be to separate the config interface from the
> implementation
> > > to
> > > > > > allow
> > > > > > > for alternatives, but that might undermine your point of "do it
> > > this
> > > > > way
> > > > > > so
> > > > > > > that everyone can find it where they expect it".
> > > > > > >
> > > > > > > Serialization: Of the options, I like 1A the best, though
> > possibly
> > > > with
> > > > > > > either an option to specify a partition key rather than ID or a
> > > > helper
> > > > > to
> > > > > > > translate an arbitrary byte[] or long into a partition number.
> > > > > > >
> > > > > > > Thanks
> > > > > > > Clark
> > > > > > >
> > > > > > >
> > > > > > > On Sun, Jan 26, 2014 at 9:13 PM, Jay Kreps <
> jay.kreps@gmail.com>
> > > > > wrote:
> > > > > > >
> > > > > > > > Thanks for the detailed thoughts. Let me elaborate on the
> > config
> > > > > thing.
> > > > > > > >
> > > > > > > > I agree that at first glance key-value strings don't seem
> like
> > a
> > > > very
> > > > > > > good
> > > > > > > > configuration api for a client. Surely a well-typed config
> > class
> > > > > would
> > > > > > be
> > > > > > > > better! I actually disagree and let me see if I can convince
> > you.
> > > > > > > >
> > > > > > > > My reasoning has nothing to do with the api and everything to
> > do
> > > > with
> > > > > > > > operations.
> > > > > > > >
> > > > > > > > Clients are embedded in applications which are themselves
> > > > configured.
> > > > > > In
> > > > > > > > any place that takes operations seriously the configuration
> for
> > > > these
> > > > > > > > applications will be version controlled and maintained
> through
> > > some
> > > > > > kind
> > > > > > > of
> > > > > > > > config management system. If we give a config class with
> > getters
> > > > and
> > > > > > > > setters the application has to expose those properties to its
> > > > > > > > configuration. What invariably happens is that the
> application
> > > > > exposes
> > > > > > > only
> > > > > > > > a choice few properties that they thought they would change.
> > > > > > Furthermore
> > > > > > > > the application will make up a name for these configs that
> > seems
> > > > > > > intuitive
> > > > > > > > at the time in the 2 seconds the engineer spends thinking
> about
> > > it.
> > > > > > > >
> > > > > > > > Now consider the result of this in the large. You end up with
> > > > dozens
> > > > > or
> > > > > > > > hundreds of applications that have the client embedded. Each
> > > > exposes
> > > > > a
> > > > > > > > different, inadequate subset of the possible configs, each
> with
> > > > > > different
> > > > > > > > names. It is a nightmare.
> > > > > > > >
> > > > > > > > If you use a string-string map the config system can directly
> > > get a
> > > > > > > bundle
> > > > > > > > of config key-value pairs and put them into the client. This
> > > means
> > > > > that
> > > > > > > all
> > > > > > > > configuration is automatically available with the name
> > documented
> > > > on
> > > > > > the
> > > > > > > > website in every application that does this. If you upgrade
> to
> > a
> > > > new
> > > > > > > kafka
> > > > > > > > version with more configs those will be exposed too. If you
> > > realize
> > > > > > that
> > > > > > > > you need to change a default you can just go through your
> > configs
> > > > and
> > > > > > > > change it everywhere as it will have the same name
> everywhere.
> > > > > > > >
> > > > > > > > -Jay
> > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > > On Sun, Jan 26, 2014 at 4:47 PM, Clark Breyman <
> > > clark@breyman.com>
> > > > > > > wrote:
> > > > > > > >
> > > > > > > > > Thanks Jay. I'll see if I can put together a more complete
> > > > > response,
> > > > > > > > > perhaps as separate threads so that topics don't get
> > entangled.
> > > > In
> > > > > > the
> > > > > > > > mean
> > > > > > > > > time, here's a couple responses:
> > > > > > > > >
> > > > > > > > > Serialization: you've broken out a sub-thread so i'll reply
> > > > there.
> > > > > My
> > > > > > > > bias
> > > > > > > > > is that I like generics (except for type-erasure) and in
> > > > particular
> > > > > > > they
> > > > > > > > > make it easy to compose serializers for compound payloads
> > (e.g.
> > > > > when
> > > > > > a
> > > > > > > > > common header wraps a payload of parameterized type). I'll
> > > > respond
> > > > > to
> > > > > > > > your
> > > > > > > > > 4-options message with an example.
> > > > > > > > >
> > > > > > > > > Build: I've seen a lot of "maven-compatible" build systems
> > > > produce
> > > > > > > > > "artifacts" that aren't really artifacts - no embedded POM
> > or,
> > > > > worst,
> > > > > > > > > malformed POM. I know the sbt-generated artifacts were this
> > > way -
> > > > > > onus
> > > > > > > is
> > > > > > > > > on me to see what gradle is spitting out and what a maven
> > build
> > > > > might
> > > > > > > > look
> > > > > > > > > like. Maven may be old and boring, but it gets out of the
> way
> > > and
> > > > > > > > > integrates really seamlessly with a lot of IDEs. When some
> > > scala
> > > > > > > > projects I
> > > > > > > > > was working on in the fall of 2011 switched from sbt to
> > maven,
> > > > > build
> > > > > > > > became
> > > > > > > > > a non-issue.
> > > > > > > > >
> > > > > > > > > Config: Not a big deal  and no, I don't think a dropwizard
> > > > > dependency
> > > > > > > is
> > > > > > > > > appropriate. I do like using simple entity beans (POJO's
> not
> > > > j2EE)
> > > > > > for
> > > > > > > > > configuration, especially if they can be marshalled without
> > > > > > annotation
> > > > > > > by
> > > > > > > > > Jackson. I only mentioned the dropwizard-extras  because it
> > has
> > > > > some
> > > > > > > > entity
> > > > > > > > > bean versions of the ZK and Kafka configs.
> > > > > > > > >
> > > > > > > > > Domain-packaging: Also not a big deal - it's what's
> expected
> > > and
> > > > > it's
> > > > > > > > > pretty free in most IDE's. The advantages I see is that it
> is
> > > > clear
> > > > > > > > whether
> > > > > > > > > something is from the Apache Kafka project and whether
> > > something
> > > > is
> > > > > > > from
> > > > > > > > > another org and related to Kafka. That said, nothing really
> > > > > enforces
> > > > > > > it.
> > > > > > > > >
> > > > > > > > > Futures: I'll see if I can create some examples to
> > demonstrate
> > > > > Future
> > > > > > > > > making interop easier.
> > > > > > > > >
> > > > > > > > > Regards,
> > > > > > > > > C
> > > > > > > > >
> > > > > > > > >
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > On Fri, Jan 24, 2014 at 4:36 PM, Jay Kreps <
> > > jay.kreps@gmail.com>
> > > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > Hey Clark,
> > > > > > > > > >
> > > > > > > > > > - Serialization: Yes I agree with these though I don't
> > > consider
> > > > > the
> > > > > > > > loss
> > > > > > > > > of
> > > > > > > > > > generics a big issue. I'll try to summarize what I would
> > > > consider
> > > > > > the
> > > > > > > > > best
> > > > > > > > > > alternative api with raw byte[].
> > > > > > > > > >
> > > > > > > > > > - Maven: We had this debate a few months back and the
> > > consensus
> > > > > was
> > > > > > > > > gradle.
> > > > > > > > > > Is there a specific issue with the poms gradle makes? I
> am
> > > > > > extremely
> > > > > > > > > loath
> > > > > > > > > > to revisit the issue as build issues are a recurring
> thing
> > > and
> > > > no
> > > > > > one
> > > > > > > > > ever
> > > > > > > > > > agrees and ultimately our build needs are very simple.
> > > > > > > > > >
> > > > > > > > > > - Config: I'm not sure if I follow the point. Are you
> > saying
> > > we
> > > > > > > should
> > > > > > > > > use
> > > > > > > > > > something in dropwizard for config? One principle here is
> > to
> > > > try
> > > > > to
> > > > > > > > > remove
> > > > > > > > > > as many client dependencies as possible as we inevitably
> > run
> > > > into
> > > > > > > > > terrible
> > > > > > > > > > compatibility issues with users who use the same library
> or
> > > its
> > > > > > > > > > dependencies at different versions. Or are you talking
> > about
> > > > > > > > maintaining
> > > > > > > > > > compatibility with existing config parameters? I think as
> > > much
> > > > > as a
> > > > > > > > > config
> > > > > > > > > > in the existing client makes sense it should have the
> same
> > > name
> > > > > (I
> > > > > > > was
> > > > > > > > a
> > > > > > > > > > bit sloppy about that so I'll fix any errors there).
> There
> > > are
> > > > a
> > > > > > few
> > > > > > > > new
> > > > > > > > > > things and we should give those reasonable defaults. I
> > think
> > > > > config
> > > > > > > is
> > > > > > > > > > important so I'll start a thread on the config package in
> > > > there.
> > > > > > > > > >
> > > > > > > > > > - org.apache.kafka: We could do this. I always considered
> > it
> > > > kind
> > > > > > of
> > > > > > > an
> > > > > > > > > odd
> > > > > > > > > > thing Java programmers do that has no real motivation
> (but
> > I
> > > > > could
> > > > > > be
> > > > > > > > > > re-educated!). I don't think it ends up reducing naming
> > > > conflicts
> > > > > > in
> > > > > > > > > > practice and it adds a lot of noise and nested
> directories.
> > > Is
> > > > > > there
> > > > > > > a
> > > > > > > > > > reason you prefer this or just to be more standard?
> > > > > > > > > >
> > > > > > > > > > - Future: Basically I didn't see any particular
> advantage.
> > > The
> > > > > > > cancel()
> > > > > > > > > > method doesn't really make sense so probably wouldn't
> work.
> > > > > > Likewise
> > > > > > > I
> > > > > > > > > > dislike the checked exceptions it requires. Basically I
> > just
> > > > > wrote
> > > > > > > out
> > > > > > > > > some
> > > > > > > > > > code examples and it seemed cleaner with a special
> purpose
> > > > > object.
> > > > > > I
> > > > > > > > > wasn't
> > > > > > > > > > actually aware of plans for improved futures in java 8 or
> > the
> > > > > other
> > > > > > > > > > integrations. Maybe you could elaborate on this a bit and
> > > show
> > > > > how
> > > > > > it
> > > > > > > > > would
> > > > > > > > > > be used? Sounds promising, I just don't know a lot about
> > it.
> > > > > > > > > >
> > > > > > > > > > -Jay
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > On Fri, Jan 24, 2014 at 3:30 PM, Clark Breyman <
> > > > > clark@breyman.com>
> > > > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > Jay - Thanks for the call for comments. Here's some
> > initial
> > > > > > input:
> > > > > > > > > > >
> > > > > > > > > > > - Make message serialization a client responsibility
> > > (making
> > > > > all
> > > > > > > > > messages
> > > > > > > > > > > byte[]). Reflection-based loading makes it harder to
> use
> > > > > generic
> > > > > > > > codecs
> > > > > > > > > > > (e.g.  Envelope<PREFIX, DATA, SUFFIX>) or build up
> codec
> > > > > > > > > > programmatically.
> > > > > > > > > > > Non-default partitioning should require an explicit
> > > partition
> > > > > > key.
> > > > > > > > > > >
> > > > > > > > > > > - I really like the fact that it will be native Java.
> > > Please
> > > > > > > consider
> > > > > > > > > > using
> > > > > > > > > > > native maven and not sbt, gradle, ivy, etc as they
> don't
> > > > > reliably
> > > > > > > > play
> > > > > > > > > > nice
> > > > > > > > > > > in the maven ecosystem. A jar without a well-formed pom
> > > > doesn't
> > > > > > > feel
> > > > > > > > > > like a
> > > > > > > > > > > real artifact. The pom's generated by sbt et al. are
> not
> > > well
> > > > > > > formed.
> > > > > > > > > > Using
> > > > > > > > > > > maven will make builds and IDE integration much
> smoother.
> > > > > > > > > > >
> > > > > > > > > > > - Look at Nick Telford's dropwizard-extras package in
> > which
> > > > he
> > > > > > > > defines
> > > > > > > > > > some
> > > > > > > > > > > Jackson-compatible POJO's for loading configuration.
> > Seems
> > > > like
> > > > > > > your
> > > > > > > > > > client
> > > > > > > > > > > migration is similar. The config objects should have
> > > > > constructors
> > > > > > > or
> > > > > > > > > > > factories that accept Map<String, String> and
> Properties
> > > for
> > > > > ease
> > > > > > > of
> > > > > > > > > > > migration.
> > > > > > > > > > >
> > > > > > > > > > > - Would you consider using the org.apache.kafka package
> > for
> > > > the
> > > > > > new
> > > > > > > > API
> > > > > > > > > > > (quibble)
> > > > > > > > > > >
> > > > > > > > > > > - Why create your own futures rather than use
> > > > > > > > > > > java.util.concurrent.Future<Long> or similar? Standard
> > > > futures
> > > > > > will
> > > > > > > > > play
> > > > > > > > > > > nice with other reactive libs and things like J8's
> > > > > > > ComposableFuture.
> > > > > > > > > > >
> > > > > > > > > > > Thanks again,
> > > > > > > > > > > C
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > On Fri, Jan 24, 2014 at 2:46 PM, Roger Hoover <
> > > > > > > > roger.hoover@gmail.com
> > > > > > > > > > > >wrote:
> > > > > > > > > > >
> > > > > > > > > > > > A couple comments:
> > > > > > > > > > > >
> > > > > > > > > > > > 1) Why does the config use a broker list instead of
> > > > > discovering
> > > > > > > the
> > > > > > > > > > > brokers
> > > > > > > > > > > > in ZooKeeper?  It doesn't match the HighLevelConsumer
> > > API.
> > > > > > > > > > > >
> > > > > > > > > > > > 2) It looks like broker connections are created on
> > > demand.
> > > > >  I'm
> > > > > > > > > > wondering
> > > > > > > > > > > > if sometimes you might want to flush out config or
> > > network
> > > > > > > > > connectivity
> > > > > > > > > > > > issues before pushing the first message through.
> > > > > > > > > > > >
> > > > > > > > > > > > Should there also be a KafkaProducer.connect() or
> > .open()
> > > > > > method
> > > > > > > or
> > > > > > > > > > > > connectAll()?  I guess it would try to connect to all
> > > > brokers
> > > > > > in
> > > > > > > > the
> > > > > > > > > > > > BROKER_LIST_CONFIG
> > > > > > > > > > > >
> > > > > > > > > > > > HTH,
> > > > > > > > > > > >
> > > > > > > > > > > > Roger
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > On Fri, Jan 24, 2014 at 11:54 AM, Jay Kreps <
> > > > > > jay.kreps@gmail.com
> > > > > > > >
> > > > > > > > > > wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > > As mentioned in a previous email we are working on
> a
> > > > > > > > > > re-implementation
> > > > > > > > > > > of
> > > > > > > > > > > > > the producer. I would like to use this email thread
> > to
> > > > > > discuss
> > > > > > > > the
> > > > > > > > > > > > details
> > > > > > > > > > > > > of the public API and the configuration. I would
> love
> > > for
> > > > > us
> > > > > > to
> > > > > > > > be
> > > > > > > > > > > > > incredibly picky about this public api now so it is
> > as
> > > > good
> > > > > > as
> > > > > > > > > > possible
> > > > > > > > > > > > and
> > > > > > > > > > > > > we don't need to break it in the future.
> > > > > > > > > > > > >
> > > > > > > > > > > > > The best way to get a feel for the API is actually
> to
> > > > take
> > > > > a
> > > > > > > look
> > > > > > > > > at
> > > > > > > > > > > the
> > > > > > > > > > > > > javadoc, my hope is to get the api docs good enough
> > so
> > > > that
> > > > > > it
> > > > > > > is
> > > > > > > > > > > > > self-explanatory:
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> http://empathybox.com/kafka-javadoc/index.html?kafka/clients/producer/KafkaProducer.html
> > > > > > > > > > > > >
> > > > > > > > > > > > > Please take a look at this API and give me any
> > thoughts
> > > > you
> > > > > > may
> > > > > > > > > have!
> > > > > > > > > > > > >
> > > > > > > > > > > > > It may also be reasonable to take a look at the
> > > configs:
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> http://empathybox.com/kafka-javadoc/kafka/clients/producer/ProducerConfig.html
> > > > > > > > > > > > >
> > > > > > > > > > > > > The actual code is posted here:
> > > > > > > > > > > > > https://issues.apache.org/jira/browse/KAFKA-1227
> > > > > > > > > > > > >
> > > > > > > > > > > > > A few questions or comments to kick things off:
> > > > > > > > > > > > > 1. We need to make a decision on whether
> > serialization
> > > of
> > > > > the
> > > > > > > > > user's
> > > > > > > > > > > key
> > > > > > > > > > > > > and value should be done by the user (with our api
> > just
> > > > > > taking
> > > > > > > > > > byte[])
> > > > > > > > > > > or
> > > > > > > > > > > > > if we should take an object and allow the user to
> > > > > configure a
> > > > > > > > > > > Serializer
> > > > > > > > > > > > > class which we instantiate via reflection. We take
> > the
> > > > > later
> > > > > > > > > approach
> > > > > > > > > > > in
> > > > > > > > > > > > > the current producer, and I have carried this
> through
> > > to
> > > > > this
> > > > > > > > > > > prototype.
> > > > > > > > > > > > > The tradeoff I see is this: taking byte[] is
> actually
> > > > > > simpler,
> > > > > > > > the
> > > > > > > > > > user
> > > > > > > > > > > > can
> > > > > > > > > > > > > directly do whatever serialization they like. The
> > > > > > complication
> > > > > > > is
> > > > > > > > > > > > actually
> > > > > > > > > > > > > partitioning. Currently partitioning is done by a
> > > similar
> > > > > > > plug-in
> > > > > > > > > api
> > > > > > > > > > > > > (Partitioner) which the user can implement and
> > > configure
> > > > to
> > > > > > > > > override
> > > > > > > > > > > how
> > > > > > > > > > > > > partitions are assigned. If we take byte[] as input
> > > then
> > > > we
> > > > > > > have
> > > > > > > > no
> > > > > > > > > > > > access
> > > > > > > > > > > > > to the original object and partitioning MUST be
> done
> > on
> > > > the
> > > > > > > > byte[].
> > > > > > > > > > > This
> > > > > > > > > > > > is
> > > > > > > > > > > > > fine for hash partitioning. However for various
> types
> > > of
> > > > > > > semantic
> > > > > > > > > > > > > partitioning (range partitioning, or whatever) you
> > > would
> > > > > want
> > > > > > > > > access
> > > > > > > > > > to
> > > > > > > > > > > > the
> > > > > > > > > > > > > original object. In the current approach a producer
> > who
> > > > > > wishes
> > > > > > > to
> > > > > > > > > > send
> > > > > > > > > > > > > byte[] they have serialized in their own code can
> > > > configure
> > > > > > the
> > > > > > > > > > > > > BytesSerialization we supply which is just a "no
> op"
> > > > > > > > serialization.
> > > > > > > > > > > > > 2. We should obsess over naming and make sure each
> of
> > > the
> > > > > > class
> > > > > > > > > names
> > > > > > > > > > > are
> > > > > > > > > > > > > good.
> > > > > > > > > > > > > 3. Jun has already pointed out that we need to
> > include
> > > > the
> > > > > > > topic
> > > > > > > > > and
> > > > > > > > > > > > > partition in the response, which is absolutely
> > right. I
> > > > > > haven't
> > > > > > > > > done
> > > > > > > > > > > that
> > > > > > > > > > > > > yet but that definitely needs to be there.
> > > > > > > > > > > > > 4. Currently RecordSend.await will throw an
> exception
> > > if
> > > > > the
> > > > > > > > > request
> > > > > > > > > > > > > failed. The intention here is that
> > > > > > > producer.send(message).await()
> > > > > > > > > > > exactly
> > > > > > > > > > > > > simulates a synchronous call. Guozhang has noted
> that
> > > > this
> > > > > > is a
> > > > > > > > > > little
> > > > > > > > > > > > > annoying since the user must then catch exceptions.
> > > > However
> > > > > > if
> > > > > > > we
> > > > > > > > > > > remove
> > > > > > > > > > > > > this then if the user doesn't check for errors they
> > > won't
> > > > > > know
> > > > > > > > one
> > > > > > > > > > has
> > > > > > > > > > > > > occurred, which I predict will be a common mistake.
> > > > > > > > > > > > > 5. Perhaps there is more we could do to make the
> > async
> > > > > > > callbacks
> > > > > > > > > and
> > > > > > > > > > > > future
> > > > > > > > > > > > > we give back intuitive and easy to program against?
> > > > > > > > > > > > >
> > > > > > > > > > > > > Some background info on implementation:
> > > > > > > > > > > > >
> > > > > > > > > > > > > At a high level the primary difference in this
> > producer
> > > > is
> > > > > > that
> > > > > > > > it
> > > > > > > > > > > > removes
> > > > > > > > > > > > > the distinction between the "sync" and "async"
> > > producer.
> > > > > > > > > Effectively
> > > > > > > > > > > all
> > > > > > > > > > > > > requests are sent asynchronously but always return
> a
> > > > future
> > > > > > > > > response
> > > > > > > > > > > > object
> > > > > > > > > > > > > that gives the offset as well as any error that may
> > > have
> > > > > > > occurred
> > > > > > > > > > when
> > > > > > > > > > > > the
> > > > > > > > > > > > > request is complete. The batching that is done in
> the
> > > > async
> > > > > > > > > producer
> > > > > > > > > > > only
> > > > > > > > > > > > > today is done whenever possible now. This means
> that
> > > the
> > > > > sync
> > > > > > > > > > producer,
> > > > > > > > > > > > > under load, can get performance as good as the
> async
> > > > > producer
> > > > > > > > > > > > (preliminary
> > > > > > > > > > > > > results show the producer getting 1m messages/sec).
> > > This
> > > > > > works
> > > > > > > > > > similar
> > > > > > > > > > > to
> > > > > > > > > > > > > group commit in databases but with respect to the
> > > actual
> > > > > > > network
> > > > > > > > > > > > > transmission--any messages that arrive while a send
> > is
> > > in
> > > > > > > > progress
> > > > > > > > > > are
> > > > > > > > > > > > > batched together. It is also possible to encourage
> > > > batching
> > > > > > > even
> > > > > > > > > > under
> > > > > > > > > > > > low
> > > > > > > > > > > > > load to save server resources by introducing a
> delay
> > on
> > > > the
> > > > > > > send
> > > > > > > > to
> > > > > > > > > > > allow
> > > > > > > > > > > > > more messages to accumulate; this is done using the
> > > > > > > > linger.msconfig
> > > > > > > > > > > > (this
> > > > > > > > > > > > > is similar to Nagle's algorithm in TCP).
> > > > > > > > > > > > >
> > > > > > > > > > > > > This producer does all network communication
> > > > asynchronously
> > > > > > and
> > > > > > > > in
> > > > > > > > > > > > parallel
> > > > > > > > > > > > > to all servers so the performance penalty for
> acks=-1
> > > and
> > > > > > > waiting
> > > > > > > > > on
> > > > > > > > > > > > > replication should be much reduced. I haven't done
> > much
> > > > > > > > > benchmarking
> > > > > > > > > > on
> > > > > > > > > > > > > this yet, though.
> > > > > > > > > > > > >
> > > > > > > > > > > > > The high level design is described a little here,
> > > though
> > > > > this
> > > > > > > is
> > > > > > > > > now
> > > > > > > > > > a
> > > > > > > > > > > > > little out of date:
> > > > > > > > > > > > >
> > > > > > >
> https://cwiki.apache.org/confluence/display/KAFKA/Client+Rewrite
> > > > > > > > > > > > >
> > > > > > > > > > > > > -Jay
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: New Producer Public API

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

That sounds cool. How did you end up handling parallel I/O if you wrap the
individual connections? Don't you need some selector that selects over all
the connections?

-Jay


On Tue, Jan 28, 2014 at 2:31 PM, Tom Brown <to...@gmail.com> wrote:

> I implemented a 0.7 client in pure java, and its API very closely resembled
> this. (When multiple people independently engineer the same solution, it's
> probably good... right?). However, there were a few architectural
> differences with my client:
>
> 1. The basic client itself was just an asynchronous layer around the
> different server functions. In and of itself it had no knowledge of
> partitions, only servers (and maintained TCP connections to them).
>
> 2. The main producer was an additional layer that provided a high-level
> interface that could batch individual messages based on partition.
>
> 3. Knowledge of partitioning was done via an interface so that different
> strategies could be used.
>
> 4. Partitioning was done by the user, with knowledge of the available
> partitions provided by #3.
>
> 5. Serialization was done by the user to simplify the API.
>
> 6. Futures were used to make asynchronous emulate synchronous calls.
>
>
> The main benefit of this approach is flexibility. For example, since the
> base client was just a managed connection (and not inherently a producer),
> it was easy to composite a produce request and an offsets request together
> into a confirmed produce request (officially not available in 0.7).
>
> Decoupling the basic client from partition management allowed the me to
> implement zk discovery as a separate project so that the main project had
> no complex dependencies. The same was true of decoupling serialization.
> It's trivial to build an optional layer that adds those features in, while
> allowing access to the base APIs for those that need it.
>
> Using standard Future objects was also beneficial, since I could combine
> them with existing tools (such as guava).
>
> It may be too late to be of use, but I have been working with my company's
> legal department to release the implementation I described above. If you're
> interested in it, let me know.
>
>
> To sum up my thoughts regarding the new API, I think it's a great start. I
> would like to see a more layered approach so I can use the parts I want,
> and adapt the other parts as needed. I would also like to see standard
> interfaces (especially Future) used where they makes sense.
>
> --Tom
>
>
>
>
>
> On Tue, Jan 28, 2014 at 1:33 PM, Roger Hoover <roger.hoover@gmail.com
> >wrote:
>
> > +1 ListenableFuture: If this works similar to Deferreds in Twisted Python
> > or Promised IO in Javascript, I think this is a great pattern for
> > decoupling your callback logic from the place where the Future is
> > generated.  You can register as many callbacks as you like, each in the
> > appropriate layer of the code and have each observer get notified when
> the
> > promised i/o is complete without any of them knowing about each other.
> >
> >
> > On Tue, Jan 28, 2014 at 11:32 AM, Jay Kreps <ja...@gmail.com> wrote:
> >
> > > Hey Ross,
> > >
> > > - ListenableFuture: Interesting. That would be an alternative to the
> > direct
> > > callback support we provide. There could be pros to this, let me think
> > > about it.
> > > - We could provide layering, but I feel that the serialization is such
> a
> > > small thing we should just make a decision and chose one, it doesn't
> seem
> > > to me to justify a whole public facing layer.
> > > - Yes, this is fairly esoteric, essentially I think it is fairly
> similar
> > to
> > > databases like DynamoDB that allow you to specify two partition keys (I
> > > think DynamoDB does this...). The reasoning is that in fact there are
> > > several things you can use the key field for: (1) to compute the
> > partition
> > > to store the data in, (2) as a unique identifier to deduplicate that
> > > partition's records within a log. These two things are almost always
> the
> > > same, but occationally may differ when you want to group data in a more
> > > sophisticated way then just a hash of the primary key but still retain
> > the
> > > proper primary key for delivery to the consumer and log compaction.
> > >
> > > -Jay
> > >
> > >
> > > On Tue, Jan 28, 2014 at 3:24 AM, Ross Black <ro...@gmail.com>
> > > wrote:
> > >
> > > > Hi Jay,
> > > >
> > > > - Just to add some more info/confusion about possibly using Future
> ...
> > > >   If Kafka uses a JDK future, it plays nicely with other frameworks
> as
> > > > well.
> > > >   Google Guava has a ListenableFuture that allows callback handling
> to
> > be
> > > > added via the returned future, and allows the callbacks to be passed
> > off
> > > to
> > > > a specified executor.
> > > >
> > > >
> > > >
> > >
> >
> http://docs.guava-libraries.googlecode.com/git-history/release/javadoc/com/google/common/util/concurrent/ListenableFuture.html
> > > >   The JDK future can easily be converted to a listenable future.
> > > >
> > > > - On the question of byte[] vs Object, could this be solved by
> layering
> > > the
> > > > API?  eg. a raw producer (use byte[] and specify the partition
> number)
> > > and
> > > > a normal producer (use generic object and specify a Partitioner)?
> > > >
> > > > - I am confused by the keys in ProducerRecord and Partitioner.  What
> is
> > > the
> > > > usage for both a key and a partition key? (I am not yet using 0.8)
> > > >
> > > >
> > > > Thanks,
> > > > Ross
> > > >
> > > >
> > > >
> > > > On 28 January 2014 05:00, Xavier Stevens <xa...@gaikai.com> wrote:
> > > >
> > > > > AutoCloseable would be nice for us as most of our code is using
> Java
> > 7
> > > at
> > > > > this point.
> > > > >
> > > > > I like Dropwizard's configuration mapping to POJOs via Jackson, but
> > if
> > > > you
> > > > > wanted to stick with property maps I don't care enough to object.
> > > > >
> > > > > If the producer only dealt with bytes, is there a way we could
> still
> > > due
> > > > > partition plugins without specifying the number explicitly? I would
> > > > prefer
> > > > > to be able to pass in field(s) that would be used by the
> partitioner.
> > > > > Obviously if this wasn't possible you could always deserialize the
> > > object
> > > > > in the partitioner and grab the fields you want, but that seems
> > really
> > > > > expensive to do on every message.
> > > > >
> > > > > It would also be nice to have a Java API Encoder constructor taking
> > in
> > > > > VerifiableProperties. Scala understands how to handle "props:
> > > > > VerifiableProperties = null", but Java doesn't. So you don't run
> into
> > > > this
> > > > > problem until runtime.
> > > > >
> > > > >
> > > > > -Xavier
> > > > >
> > > > >
> > > > > On Mon, Jan 27, 2014 at 9:37 AM, Clark Breyman <cl...@breyman.com>
> > > > wrote:
> > > > >
> > > > > > Jay -
> > > > > >
> > > > > > Config - your explanation makes sense. I'm just so accustomed to
> > > having
> > > > > > Jackson automatically map my configuration objects to POJOs that
> > I've
> > > > > > stopped using property files. They are lingua franca. The only
> > > thought
> > > > > > might be to separate the config interface from the implementation
> > to
> > > > > allow
> > > > > > for alternatives, but that might undermine your point of "do it
> > this
> > > > way
> > > > > so
> > > > > > that everyone can find it where they expect it".
> > > > > >
> > > > > > Serialization: Of the options, I like 1A the best, though
> possibly
> > > with
> > > > > > either an option to specify a partition key rather than ID or a
> > > helper
> > > > to
> > > > > > translate an arbitrary byte[] or long into a partition number.
> > > > > >
> > > > > > Thanks
> > > > > > Clark
> > > > > >
> > > > > >
> > > > > > On Sun, Jan 26, 2014 at 9:13 PM, Jay Kreps <ja...@gmail.com>
> > > > wrote:
> > > > > >
> > > > > > > Thanks for the detailed thoughts. Let me elaborate on the
> config
> > > > thing.
> > > > > > >
> > > > > > > I agree that at first glance key-value strings don't seem like
> a
> > > very
> > > > > > good
> > > > > > > configuration api for a client. Surely a well-typed config
> class
> > > > would
> > > > > be
> > > > > > > better! I actually disagree and let me see if I can convince
> you.
> > > > > > >
> > > > > > > My reasoning has nothing to do with the api and everything to
> do
> > > with
> > > > > > > operations.
> > > > > > >
> > > > > > > Clients are embedded in applications which are themselves
> > > configured.
> > > > > In
> > > > > > > any place that takes operations seriously the configuration for
> > > these
> > > > > > > applications will be version controlled and maintained through
> > some
> > > > > kind
> > > > > > of
> > > > > > > config management system. If we give a config class with
> getters
> > > and
> > > > > > > setters the application has to expose those properties to its
> > > > > > > configuration. What invariably happens is that the application
> > > > exposes
> > > > > > only
> > > > > > > a choice few properties that they thought they would change.
> > > > > Furthermore
> > > > > > > the application will make up a name for these configs that
> seems
> > > > > > intuitive
> > > > > > > at the time in the 2 seconds the engineer spends thinking about
> > it.
> > > > > > >
> > > > > > > Now consider the result of this in the large. You end up with
> > > dozens
> > > > or
> > > > > > > hundreds of applications that have the client embedded. Each
> > > exposes
> > > > a
> > > > > > > different, inadequate subset of the possible configs, each with
> > > > > different
> > > > > > > names. It is a nightmare.
> > > > > > >
> > > > > > > If you use a string-string map the config system can directly
> > get a
> > > > > > bundle
> > > > > > > of config key-value pairs and put them into the client. This
> > means
> > > > that
> > > > > > all
> > > > > > > configuration is automatically available with the name
> documented
> > > on
> > > > > the
> > > > > > > website in every application that does this. If you upgrade to
> a
> > > new
> > > > > > kafka
> > > > > > > version with more configs those will be exposed too. If you
> > realize
> > > > > that
> > > > > > > you need to change a default you can just go through your
> configs
> > > and
> > > > > > > change it everywhere as it will have the same name everywhere.
> > > > > > >
> > > > > > > -Jay
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > On Sun, Jan 26, 2014 at 4:47 PM, Clark Breyman <
> > clark@breyman.com>
> > > > > > wrote:
> > > > > > >
> > > > > > > > Thanks Jay. I'll see if I can put together a more complete
> > > > response,
> > > > > > > > perhaps as separate threads so that topics don't get
> entangled.
> > > In
> > > > > the
> > > > > > > mean
> > > > > > > > time, here's a couple responses:
> > > > > > > >
> > > > > > > > Serialization: you've broken out a sub-thread so i'll reply
> > > there.
> > > > My
> > > > > > > bias
> > > > > > > > is that I like generics (except for type-erasure) and in
> > > particular
> > > > > > they
> > > > > > > > make it easy to compose serializers for compound payloads
> (e.g.
> > > > when
> > > > > a
> > > > > > > > common header wraps a payload of parameterized type). I'll
> > > respond
> > > > to
> > > > > > > your
> > > > > > > > 4-options message with an example.
> > > > > > > >
> > > > > > > > Build: I've seen a lot of "maven-compatible" build systems
> > > produce
> > > > > > > > "artifacts" that aren't really artifacts - no embedded POM
> or,
> > > > worst,
> > > > > > > > malformed POM. I know the sbt-generated artifacts were this
> > way -
> > > > > onus
> > > > > > is
> > > > > > > > on me to see what gradle is spitting out and what a maven
> build
> > > > might
> > > > > > > look
> > > > > > > > like. Maven may be old and boring, but it gets out of the way
> > and
> > > > > > > > integrates really seamlessly with a lot of IDEs. When some
> > scala
> > > > > > > projects I
> > > > > > > > was working on in the fall of 2011 switched from sbt to
> maven,
> > > > build
> > > > > > > became
> > > > > > > > a non-issue.
> > > > > > > >
> > > > > > > > Config: Not a big deal  and no, I don't think a dropwizard
> > > > dependency
> > > > > > is
> > > > > > > > appropriate. I do like using simple entity beans (POJO's not
> > > j2EE)
> > > > > for
> > > > > > > > configuration, especially if they can be marshalled without
> > > > > annotation
> > > > > > by
> > > > > > > > Jackson. I only mentioned the dropwizard-extras  because it
> has
> > > > some
> > > > > > > entity
> > > > > > > > bean versions of the ZK and Kafka configs.
> > > > > > > >
> > > > > > > > Domain-packaging: Also not a big deal - it's what's expected
> > and
> > > > it's
> > > > > > > > pretty free in most IDE's. The advantages I see is that it is
> > > clear
> > > > > > > whether
> > > > > > > > something is from the Apache Kafka project and whether
> > something
> > > is
> > > > > > from
> > > > > > > > another org and related to Kafka. That said, nothing really
> > > > enforces
> > > > > > it.
> > > > > > > >
> > > > > > > > Futures: I'll see if I can create some examples to
> demonstrate
> > > > Future
> > > > > > > > making interop easier.
> > > > > > > >
> > > > > > > > Regards,
> > > > > > > > C
> > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > > On Fri, Jan 24, 2014 at 4:36 PM, Jay Kreps <
> > jay.kreps@gmail.com>
> > > > > > wrote:
> > > > > > > >
> > > > > > > > > Hey Clark,
> > > > > > > > >
> > > > > > > > > - Serialization: Yes I agree with these though I don't
> > consider
> > > > the
> > > > > > > loss
> > > > > > > > of
> > > > > > > > > generics a big issue. I'll try to summarize what I would
> > > consider
> > > > > the
> > > > > > > > best
> > > > > > > > > alternative api with raw byte[].
> > > > > > > > >
> > > > > > > > > - Maven: We had this debate a few months back and the
> > consensus
> > > > was
> > > > > > > > gradle.
> > > > > > > > > Is there a specific issue with the poms gradle makes? I am
> > > > > extremely
> > > > > > > > loath
> > > > > > > > > to revisit the issue as build issues are a recurring thing
> > and
> > > no
> > > > > one
> > > > > > > > ever
> > > > > > > > > agrees and ultimately our build needs are very simple.
> > > > > > > > >
> > > > > > > > > - Config: I'm not sure if I follow the point. Are you
> saying
> > we
> > > > > > should
> > > > > > > > use
> > > > > > > > > something in dropwizard for config? One principle here is
> to
> > > try
> > > > to
> > > > > > > > remove
> > > > > > > > > as many client dependencies as possible as we inevitably
> run
> > > into
> > > > > > > > terrible
> > > > > > > > > compatibility issues with users who use the same library or
> > its
> > > > > > > > > dependencies at different versions. Or are you talking
> about
> > > > > > > maintaining
> > > > > > > > > compatibility with existing config parameters? I think as
> > much
> > > > as a
> > > > > > > > config
> > > > > > > > > in the existing client makes sense it should have the same
> > name
> > > > (I
> > > > > > was
> > > > > > > a
> > > > > > > > > bit sloppy about that so I'll fix any errors there). There
> > are
> > > a
> > > > > few
> > > > > > > new
> > > > > > > > > things and we should give those reasonable defaults. I
> think
> > > > config
> > > > > > is
> > > > > > > > > important so I'll start a thread on the config package in
> > > there.
> > > > > > > > >
> > > > > > > > > - org.apache.kafka: We could do this. I always considered
> it
> > > kind
> > > > > of
> > > > > > an
> > > > > > > > odd
> > > > > > > > > thing Java programmers do that has no real motivation (but
> I
> > > > could
> > > > > be
> > > > > > > > > re-educated!). I don't think it ends up reducing naming
> > > conflicts
> > > > > in
> > > > > > > > > practice and it adds a lot of noise and nested directories.
> > Is
> > > > > there
> > > > > > a
> > > > > > > > > reason you prefer this or just to be more standard?
> > > > > > > > >
> > > > > > > > > - Future: Basically I didn't see any particular advantage.
> > The
> > > > > > cancel()
> > > > > > > > > method doesn't really make sense so probably wouldn't work.
> > > > > Likewise
> > > > > > I
> > > > > > > > > dislike the checked exceptions it requires. Basically I
> just
> > > > wrote
> > > > > > out
> > > > > > > > some
> > > > > > > > > code examples and it seemed cleaner with a special purpose
> > > > object.
> > > > > I
> > > > > > > > wasn't
> > > > > > > > > actually aware of plans for improved futures in java 8 or
> the
> > > > other
> > > > > > > > > integrations. Maybe you could elaborate on this a bit and
> > show
> > > > how
> > > > > it
> > > > > > > > would
> > > > > > > > > be used? Sounds promising, I just don't know a lot about
> it.
> > > > > > > > >
> > > > > > > > > -Jay
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > On Fri, Jan 24, 2014 at 3:30 PM, Clark Breyman <
> > > > clark@breyman.com>
> > > > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > Jay - Thanks for the call for comments. Here's some
> initial
> > > > > input:
> > > > > > > > > >
> > > > > > > > > > - Make message serialization a client responsibility
> > (making
> > > > all
> > > > > > > > messages
> > > > > > > > > > byte[]). Reflection-based loading makes it harder to use
> > > > generic
> > > > > > > codecs
> > > > > > > > > > (e.g.  Envelope<PREFIX, DATA, SUFFIX>) or build up codec
> > > > > > > > > programmatically.
> > > > > > > > > > Non-default partitioning should require an explicit
> > partition
> > > > > key.
> > > > > > > > > >
> > > > > > > > > > - I really like the fact that it will be native Java.
> > Please
> > > > > > consider
> > > > > > > > > using
> > > > > > > > > > native maven and not sbt, gradle, ivy, etc as they don't
> > > > reliably
> > > > > > > play
> > > > > > > > > nice
> > > > > > > > > > in the maven ecosystem. A jar without a well-formed pom
> > > doesn't
> > > > > > feel
> > > > > > > > > like a
> > > > > > > > > > real artifact. The pom's generated by sbt et al. are not
> > well
> > > > > > formed.
> > > > > > > > > Using
> > > > > > > > > > maven will make builds and IDE integration much smoother.
> > > > > > > > > >
> > > > > > > > > > - Look at Nick Telford's dropwizard-extras package in
> which
> > > he
> > > > > > > defines
> > > > > > > > > some
> > > > > > > > > > Jackson-compatible POJO's for loading configuration.
> Seems
> > > like
> > > > > > your
> > > > > > > > > client
> > > > > > > > > > migration is similar. The config objects should have
> > > > constructors
> > > > > > or
> > > > > > > > > > factories that accept Map<String, String> and Properties
> > for
> > > > ease
> > > > > > of
> > > > > > > > > > migration.
> > > > > > > > > >
> > > > > > > > > > - Would you consider using the org.apache.kafka package
> for
> > > the
> > > > > new
> > > > > > > API
> > > > > > > > > > (quibble)
> > > > > > > > > >
> > > > > > > > > > - Why create your own futures rather than use
> > > > > > > > > > java.util.concurrent.Future<Long> or similar? Standard
> > > futures
> > > > > will
> > > > > > > > play
> > > > > > > > > > nice with other reactive libs and things like J8's
> > > > > > ComposableFuture.
> > > > > > > > > >
> > > > > > > > > > Thanks again,
> > > > > > > > > > C
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > On Fri, Jan 24, 2014 at 2:46 PM, Roger Hoover <
> > > > > > > roger.hoover@gmail.com
> > > > > > > > > > >wrote:
> > > > > > > > > >
> > > > > > > > > > > A couple comments:
> > > > > > > > > > >
> > > > > > > > > > > 1) Why does the config use a broker list instead of
> > > > discovering
> > > > > > the
> > > > > > > > > > brokers
> > > > > > > > > > > in ZooKeeper?  It doesn't match the HighLevelConsumer
> > API.
> > > > > > > > > > >
> > > > > > > > > > > 2) It looks like broker connections are created on
> > demand.
> > > >  I'm
> > > > > > > > > wondering
> > > > > > > > > > > if sometimes you might want to flush out config or
> > network
> > > > > > > > connectivity
> > > > > > > > > > > issues before pushing the first message through.
> > > > > > > > > > >
> > > > > > > > > > > Should there also be a KafkaProducer.connect() or
> .open()
> > > > > method
> > > > > > or
> > > > > > > > > > > connectAll()?  I guess it would try to connect to all
> > > brokers
> > > > > in
> > > > > > > the
> > > > > > > > > > > BROKER_LIST_CONFIG
> > > > > > > > > > >
> > > > > > > > > > > HTH,
> > > > > > > > > > >
> > > > > > > > > > > Roger
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > On Fri, Jan 24, 2014 at 11:54 AM, Jay Kreps <
> > > > > jay.kreps@gmail.com
> > > > > > >
> > > > > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > As mentioned in a previous email we are working on a
> > > > > > > > > re-implementation
> > > > > > > > > > of
> > > > > > > > > > > > the producer. I would like to use this email thread
> to
> > > > > discuss
> > > > > > > the
> > > > > > > > > > > details
> > > > > > > > > > > > of the public API and the configuration. I would love
> > for
> > > > us
> > > > > to
> > > > > > > be
> > > > > > > > > > > > incredibly picky about this public api now so it is
> as
> > > good
> > > > > as
> > > > > > > > > possible
> > > > > > > > > > > and
> > > > > > > > > > > > we don't need to break it in the future.
> > > > > > > > > > > >
> > > > > > > > > > > > The best way to get a feel for the API is actually to
> > > take
> > > > a
> > > > > > look
> > > > > > > > at
> > > > > > > > > > the
> > > > > > > > > > > > javadoc, my hope is to get the api docs good enough
> so
> > > that
> > > > > it
> > > > > > is
> > > > > > > > > > > > self-explanatory:
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> http://empathybox.com/kafka-javadoc/index.html?kafka/clients/producer/KafkaProducer.html
> > > > > > > > > > > >
> > > > > > > > > > > > Please take a look at this API and give me any
> thoughts
> > > you
> > > > > may
> > > > > > > > have!
> > > > > > > > > > > >
> > > > > > > > > > > > It may also be reasonable to take a look at the
> > configs:
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> http://empathybox.com/kafka-javadoc/kafka/clients/producer/ProducerConfig.html
> > > > > > > > > > > >
> > > > > > > > > > > > The actual code is posted here:
> > > > > > > > > > > > https://issues.apache.org/jira/browse/KAFKA-1227
> > > > > > > > > > > >
> > > > > > > > > > > > A few questions or comments to kick things off:
> > > > > > > > > > > > 1. We need to make a decision on whether
> serialization
> > of
> > > > the
> > > > > > > > user's
> > > > > > > > > > key
> > > > > > > > > > > > and value should be done by the user (with our api
> just
> > > > > taking
> > > > > > > > > byte[])
> > > > > > > > > > or
> > > > > > > > > > > > if we should take an object and allow the user to
> > > > configure a
> > > > > > > > > > Serializer
> > > > > > > > > > > > class which we instantiate via reflection. We take
> the
> > > > later
> > > > > > > > approach
> > > > > > > > > > in
> > > > > > > > > > > > the current producer, and I have carried this through
> > to
> > > > this
> > > > > > > > > > prototype.
> > > > > > > > > > > > The tradeoff I see is this: taking byte[] is actually
> > > > > simpler,
> > > > > > > the
> > > > > > > > > user
> > > > > > > > > > > can
> > > > > > > > > > > > directly do whatever serialization they like. The
> > > > > complication
> > > > > > is
> > > > > > > > > > > actually
> > > > > > > > > > > > partitioning. Currently partitioning is done by a
> > similar
> > > > > > plug-in
> > > > > > > > api
> > > > > > > > > > > > (Partitioner) which the user can implement and
> > configure
> > > to
> > > > > > > > override
> > > > > > > > > > how
> > > > > > > > > > > > partitions are assigned. If we take byte[] as input
> > then
> > > we
> > > > > > have
> > > > > > > no
> > > > > > > > > > > access
> > > > > > > > > > > > to the original object and partitioning MUST be done
> on
> > > the
> > > > > > > byte[].
> > > > > > > > > > This
> > > > > > > > > > > is
> > > > > > > > > > > > fine for hash partitioning. However for various types
> > of
> > > > > > semantic
> > > > > > > > > > > > partitioning (range partitioning, or whatever) you
> > would
> > > > want
> > > > > > > > access
> > > > > > > > > to
> > > > > > > > > > > the
> > > > > > > > > > > > original object. In the current approach a producer
> who
> > > > > wishes
> > > > > > to
> > > > > > > > > send
> > > > > > > > > > > > byte[] they have serialized in their own code can
> > > configure
> > > > > the
> > > > > > > > > > > > BytesSerialization we supply which is just a "no op"
> > > > > > > serialization.
> > > > > > > > > > > > 2. We should obsess over naming and make sure each of
> > the
> > > > > class
> > > > > > > > names
> > > > > > > > > > are
> > > > > > > > > > > > good.
> > > > > > > > > > > > 3. Jun has already pointed out that we need to
> include
> > > the
> > > > > > topic
> > > > > > > > and
> > > > > > > > > > > > partition in the response, which is absolutely
> right. I
> > > > > haven't
> > > > > > > > done
> > > > > > > > > > that
> > > > > > > > > > > > yet but that definitely needs to be there.
> > > > > > > > > > > > 4. Currently RecordSend.await will throw an exception
> > if
> > > > the
> > > > > > > > request
> > > > > > > > > > > > failed. The intention here is that
> > > > > > producer.send(message).await()
> > > > > > > > > > exactly
> > > > > > > > > > > > simulates a synchronous call. Guozhang has noted that
> > > this
> > > > > is a
> > > > > > > > > little
> > > > > > > > > > > > annoying since the user must then catch exceptions.
> > > However
> > > > > if
> > > > > > we
> > > > > > > > > > remove
> > > > > > > > > > > > this then if the user doesn't check for errors they
> > won't
> > > > > know
> > > > > > > one
> > > > > > > > > has
> > > > > > > > > > > > occurred, which I predict will be a common mistake.
> > > > > > > > > > > > 5. Perhaps there is more we could do to make the
> async
> > > > > > callbacks
> > > > > > > > and
> > > > > > > > > > > future
> > > > > > > > > > > > we give back intuitive and easy to program against?
> > > > > > > > > > > >
> > > > > > > > > > > > Some background info on implementation:
> > > > > > > > > > > >
> > > > > > > > > > > > At a high level the primary difference in this
> producer
> > > is
> > > > > that
> > > > > > > it
> > > > > > > > > > > removes
> > > > > > > > > > > > the distinction between the "sync" and "async"
> > producer.
> > > > > > > > Effectively
> > > > > > > > > > all
> > > > > > > > > > > > requests are sent asynchronously but always return a
> > > future
> > > > > > > > response
> > > > > > > > > > > object
> > > > > > > > > > > > that gives the offset as well as any error that may
> > have
> > > > > > occurred
> > > > > > > > > when
> > > > > > > > > > > the
> > > > > > > > > > > > request is complete. The batching that is done in the
> > > async
> > > > > > > > producer
> > > > > > > > > > only
> > > > > > > > > > > > today is done whenever possible now. This means that
> > the
> > > > sync
> > > > > > > > > producer,
> > > > > > > > > > > > under load, can get performance as good as the async
> > > > producer
> > > > > > > > > > > (preliminary
> > > > > > > > > > > > results show the producer getting 1m messages/sec).
> > This
> > > > > works
> > > > > > > > > similar
> > > > > > > > > > to
> > > > > > > > > > > > group commit in databases but with respect to the
> > actual
> > > > > > network
> > > > > > > > > > > > transmission--any messages that arrive while a send
> is
> > in
> > > > > > > progress
> > > > > > > > > are
> > > > > > > > > > > > batched together. It is also possible to encourage
> > > batching
> > > > > > even
> > > > > > > > > under
> > > > > > > > > > > low
> > > > > > > > > > > > load to save server resources by introducing a delay
> on
> > > the
> > > > > > send
> > > > > > > to
> > > > > > > > > > allow
> > > > > > > > > > > > more messages to accumulate; this is done using the
> > > > > > > linger.msconfig
> > > > > > > > > > > (this
> > > > > > > > > > > > is similar to Nagle's algorithm in TCP).
> > > > > > > > > > > >
> > > > > > > > > > > > This producer does all network communication
> > > asynchronously
> > > > > and
> > > > > > > in
> > > > > > > > > > > parallel
> > > > > > > > > > > > to all servers so the performance penalty for acks=-1
> > and
> > > > > > waiting
> > > > > > > > on
> > > > > > > > > > > > replication should be much reduced. I haven't done
> much
> > > > > > > > benchmarking
> > > > > > > > > on
> > > > > > > > > > > > this yet, though.
> > > > > > > > > > > >
> > > > > > > > > > > > The high level design is described a little here,
> > though
> > > > this
> > > > > > is
> > > > > > > > now
> > > > > > > > > a
> > > > > > > > > > > > little out of date:
> > > > > > > > > > > >
> > > > > > https://cwiki.apache.org/confluence/display/KAFKA/Client+Rewrite
> > > > > > > > > > > >
> > > > > > > > > > > > -Jay
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: New Producer Public API

Posted by Tom Brown <to...@gmail.com>.
I implemented a 0.7 client in pure java, and its API very closely resembled
this. (When multiple people independently engineer the same solution, it's
probably good... right?). However, there were a few architectural
differences with my client:

1. The basic client itself was just an asynchronous layer around the
different server functions. In and of itself it had no knowledge of
partitions, only servers (and maintained TCP connections to them).

2. The main producer was an additional layer that provided a high-level
interface that could batch individual messages based on partition.

3. Knowledge of partitioning was done via an interface so that different
strategies could be used.

4. Partitioning was done by the user, with knowledge of the available
partitions provided by #3.

5. Serialization was done by the user to simplify the API.

6. Futures were used to make asynchronous emulate synchronous calls.


The main benefit of this approach is flexibility. For example, since the
base client was just a managed connection (and not inherently a producer),
it was easy to composite a produce request and an offsets request together
into a confirmed produce request (officially not available in 0.7).

Decoupling the basic client from partition management allowed the me to
implement zk discovery as a separate project so that the main project had
no complex dependencies. The same was true of decoupling serialization.
It's trivial to build an optional layer that adds those features in, while
allowing access to the base APIs for those that need it.

Using standard Future objects was also beneficial, since I could combine
them with existing tools (such as guava).

It may be too late to be of use, but I have been working with my company's
legal department to release the implementation I described above. If you're
interested in it, let me know.


To sum up my thoughts regarding the new API, I think it's a great start. I
would like to see a more layered approach so I can use the parts I want,
and adapt the other parts as needed. I would also like to see standard
interfaces (especially Future) used where they makes sense.

--Tom





On Tue, Jan 28, 2014 at 1:33 PM, Roger Hoover <ro...@gmail.com>wrote:

> +1 ListenableFuture: If this works similar to Deferreds in Twisted Python
> or Promised IO in Javascript, I think this is a great pattern for
> decoupling your callback logic from the place where the Future is
> generated.  You can register as many callbacks as you like, each in the
> appropriate layer of the code and have each observer get notified when the
> promised i/o is complete without any of them knowing about each other.
>
>
> On Tue, Jan 28, 2014 at 11:32 AM, Jay Kreps <ja...@gmail.com> wrote:
>
> > Hey Ross,
> >
> > - ListenableFuture: Interesting. That would be an alternative to the
> direct
> > callback support we provide. There could be pros to this, let me think
> > about it.
> > - We could provide layering, but I feel that the serialization is such a
> > small thing we should just make a decision and chose one, it doesn't seem
> > to me to justify a whole public facing layer.
> > - Yes, this is fairly esoteric, essentially I think it is fairly similar
> to
> > databases like DynamoDB that allow you to specify two partition keys (I
> > think DynamoDB does this...). The reasoning is that in fact there are
> > several things you can use the key field for: (1) to compute the
> partition
> > to store the data in, (2) as a unique identifier to deduplicate that
> > partition's records within a log. These two things are almost always the
> > same, but occationally may differ when you want to group data in a more
> > sophisticated way then just a hash of the primary key but still retain
> the
> > proper primary key for delivery to the consumer and log compaction.
> >
> > -Jay
> >
> >
> > On Tue, Jan 28, 2014 at 3:24 AM, Ross Black <ro...@gmail.com>
> > wrote:
> >
> > > Hi Jay,
> > >
> > > - Just to add some more info/confusion about possibly using Future ...
> > >   If Kafka uses a JDK future, it plays nicely with other frameworks as
> > > well.
> > >   Google Guava has a ListenableFuture that allows callback handling to
> be
> > > added via the returned future, and allows the callbacks to be passed
> off
> > to
> > > a specified executor.
> > >
> > >
> > >
> >
> http://docs.guava-libraries.googlecode.com/git-history/release/javadoc/com/google/common/util/concurrent/ListenableFuture.html
> > >   The JDK future can easily be converted to a listenable future.
> > >
> > > - On the question of byte[] vs Object, could this be solved by layering
> > the
> > > API?  eg. a raw producer (use byte[] and specify the partition number)
> > and
> > > a normal producer (use generic object and specify a Partitioner)?
> > >
> > > - I am confused by the keys in ProducerRecord and Partitioner.  What is
> > the
> > > usage for both a key and a partition key? (I am not yet using 0.8)
> > >
> > >
> > > Thanks,
> > > Ross
> > >
> > >
> > >
> > > On 28 January 2014 05:00, Xavier Stevens <xa...@gaikai.com> wrote:
> > >
> > > > AutoCloseable would be nice for us as most of our code is using Java
> 7
> > at
> > > > this point.
> > > >
> > > > I like Dropwizard's configuration mapping to POJOs via Jackson, but
> if
> > > you
> > > > wanted to stick with property maps I don't care enough to object.
> > > >
> > > > If the producer only dealt with bytes, is there a way we could still
> > due
> > > > partition plugins without specifying the number explicitly? I would
> > > prefer
> > > > to be able to pass in field(s) that would be used by the partitioner.
> > > > Obviously if this wasn't possible you could always deserialize the
> > object
> > > > in the partitioner and grab the fields you want, but that seems
> really
> > > > expensive to do on every message.
> > > >
> > > > It would also be nice to have a Java API Encoder constructor taking
> in
> > > > VerifiableProperties. Scala understands how to handle "props:
> > > > VerifiableProperties = null", but Java doesn't. So you don't run into
> > > this
> > > > problem until runtime.
> > > >
> > > >
> > > > -Xavier
> > > >
> > > >
> > > > On Mon, Jan 27, 2014 at 9:37 AM, Clark Breyman <cl...@breyman.com>
> > > wrote:
> > > >
> > > > > Jay -
> > > > >
> > > > > Config - your explanation makes sense. I'm just so accustomed to
> > having
> > > > > Jackson automatically map my configuration objects to POJOs that
> I've
> > > > > stopped using property files. They are lingua franca. The only
> > thought
> > > > > might be to separate the config interface from the implementation
> to
> > > > allow
> > > > > for alternatives, but that might undermine your point of "do it
> this
> > > way
> > > > so
> > > > > that everyone can find it where they expect it".
> > > > >
> > > > > Serialization: Of the options, I like 1A the best, though possibly
> > with
> > > > > either an option to specify a partition key rather than ID or a
> > helper
> > > to
> > > > > translate an arbitrary byte[] or long into a partition number.
> > > > >
> > > > > Thanks
> > > > > Clark
> > > > >
> > > > >
> > > > > On Sun, Jan 26, 2014 at 9:13 PM, Jay Kreps <ja...@gmail.com>
> > > wrote:
> > > > >
> > > > > > Thanks for the detailed thoughts. Let me elaborate on the config
> > > thing.
> > > > > >
> > > > > > I agree that at first glance key-value strings don't seem like a
> > very
> > > > > good
> > > > > > configuration api for a client. Surely a well-typed config class
> > > would
> > > > be
> > > > > > better! I actually disagree and let me see if I can convince you.
> > > > > >
> > > > > > My reasoning has nothing to do with the api and everything to do
> > with
> > > > > > operations.
> > > > > >
> > > > > > Clients are embedded in applications which are themselves
> > configured.
> > > > In
> > > > > > any place that takes operations seriously the configuration for
> > these
> > > > > > applications will be version controlled and maintained through
> some
> > > > kind
> > > > > of
> > > > > > config management system. If we give a config class with getters
> > and
> > > > > > setters the application has to expose those properties to its
> > > > > > configuration. What invariably happens is that the application
> > > exposes
> > > > > only
> > > > > > a choice few properties that they thought they would change.
> > > > Furthermore
> > > > > > the application will make up a name for these configs that seems
> > > > > intuitive
> > > > > > at the time in the 2 seconds the engineer spends thinking about
> it.
> > > > > >
> > > > > > Now consider the result of this in the large. You end up with
> > dozens
> > > or
> > > > > > hundreds of applications that have the client embedded. Each
> > exposes
> > > a
> > > > > > different, inadequate subset of the possible configs, each with
> > > > different
> > > > > > names. It is a nightmare.
> > > > > >
> > > > > > If you use a string-string map the config system can directly
> get a
> > > > > bundle
> > > > > > of config key-value pairs and put them into the client. This
> means
> > > that
> > > > > all
> > > > > > configuration is automatically available with the name documented
> > on
> > > > the
> > > > > > website in every application that does this. If you upgrade to a
> > new
> > > > > kafka
> > > > > > version with more configs those will be exposed too. If you
> realize
> > > > that
> > > > > > you need to change a default you can just go through your configs
> > and
> > > > > > change it everywhere as it will have the same name everywhere.
> > > > > >
> > > > > > -Jay
> > > > > >
> > > > > >
> > > > > >
> > > > > >
> > > > > > On Sun, Jan 26, 2014 at 4:47 PM, Clark Breyman <
> clark@breyman.com>
> > > > > wrote:
> > > > > >
> > > > > > > Thanks Jay. I'll see if I can put together a more complete
> > > response,
> > > > > > > perhaps as separate threads so that topics don't get entangled.
> > In
> > > > the
> > > > > > mean
> > > > > > > time, here's a couple responses:
> > > > > > >
> > > > > > > Serialization: you've broken out a sub-thread so i'll reply
> > there.
> > > My
> > > > > > bias
> > > > > > > is that I like generics (except for type-erasure) and in
> > particular
> > > > > they
> > > > > > > make it easy to compose serializers for compound payloads (e.g.
> > > when
> > > > a
> > > > > > > common header wraps a payload of parameterized type). I'll
> > respond
> > > to
> > > > > > your
> > > > > > > 4-options message with an example.
> > > > > > >
> > > > > > > Build: I've seen a lot of "maven-compatible" build systems
> > produce
> > > > > > > "artifacts" that aren't really artifacts - no embedded POM or,
> > > worst,
> > > > > > > malformed POM. I know the sbt-generated artifacts were this
> way -
> > > > onus
> > > > > is
> > > > > > > on me to see what gradle is spitting out and what a maven build
> > > might
> > > > > > look
> > > > > > > like. Maven may be old and boring, but it gets out of the way
> and
> > > > > > > integrates really seamlessly with a lot of IDEs. When some
> scala
> > > > > > projects I
> > > > > > > was working on in the fall of 2011 switched from sbt to maven,
> > > build
> > > > > > became
> > > > > > > a non-issue.
> > > > > > >
> > > > > > > Config: Not a big deal  and no, I don't think a dropwizard
> > > dependency
> > > > > is
> > > > > > > appropriate. I do like using simple entity beans (POJO's not
> > j2EE)
> > > > for
> > > > > > > configuration, especially if they can be marshalled without
> > > > annotation
> > > > > by
> > > > > > > Jackson. I only mentioned the dropwizard-extras  because it has
> > > some
> > > > > > entity
> > > > > > > bean versions of the ZK and Kafka configs.
> > > > > > >
> > > > > > > Domain-packaging: Also not a big deal - it's what's expected
> and
> > > it's
> > > > > > > pretty free in most IDE's. The advantages I see is that it is
> > clear
> > > > > > whether
> > > > > > > something is from the Apache Kafka project and whether
> something
> > is
> > > > > from
> > > > > > > another org and related to Kafka. That said, nothing really
> > > enforces
> > > > > it.
> > > > > > >
> > > > > > > Futures: I'll see if I can create some examples to demonstrate
> > > Future
> > > > > > > making interop easier.
> > > > > > >
> > > > > > > Regards,
> > > > > > > C
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > On Fri, Jan 24, 2014 at 4:36 PM, Jay Kreps <
> jay.kreps@gmail.com>
> > > > > wrote:
> > > > > > >
> > > > > > > > Hey Clark,
> > > > > > > >
> > > > > > > > - Serialization: Yes I agree with these though I don't
> consider
> > > the
> > > > > > loss
> > > > > > > of
> > > > > > > > generics a big issue. I'll try to summarize what I would
> > consider
> > > > the
> > > > > > > best
> > > > > > > > alternative api with raw byte[].
> > > > > > > >
> > > > > > > > - Maven: We had this debate a few months back and the
> consensus
> > > was
> > > > > > > gradle.
> > > > > > > > Is there a specific issue with the poms gradle makes? I am
> > > > extremely
> > > > > > > loath
> > > > > > > > to revisit the issue as build issues are a recurring thing
> and
> > no
> > > > one
> > > > > > > ever
> > > > > > > > agrees and ultimately our build needs are very simple.
> > > > > > > >
> > > > > > > > - Config: I'm not sure if I follow the point. Are you saying
> we
> > > > > should
> > > > > > > use
> > > > > > > > something in dropwizard for config? One principle here is to
> > try
> > > to
> > > > > > > remove
> > > > > > > > as many client dependencies as possible as we inevitably run
> > into
> > > > > > > terrible
> > > > > > > > compatibility issues with users who use the same library or
> its
> > > > > > > > dependencies at different versions. Or are you talking about
> > > > > > maintaining
> > > > > > > > compatibility with existing config parameters? I think as
> much
> > > as a
> > > > > > > config
> > > > > > > > in the existing client makes sense it should have the same
> name
> > > (I
> > > > > was
> > > > > > a
> > > > > > > > bit sloppy about that so I'll fix any errors there). There
> are
> > a
> > > > few
> > > > > > new
> > > > > > > > things and we should give those reasonable defaults. I think
> > > config
> > > > > is
> > > > > > > > important so I'll start a thread on the config package in
> > there.
> > > > > > > >
> > > > > > > > - org.apache.kafka: We could do this. I always considered it
> > kind
> > > > of
> > > > > an
> > > > > > > odd
> > > > > > > > thing Java programmers do that has no real motivation (but I
> > > could
> > > > be
> > > > > > > > re-educated!). I don't think it ends up reducing naming
> > conflicts
> > > > in
> > > > > > > > practice and it adds a lot of noise and nested directories.
> Is
> > > > there
> > > > > a
> > > > > > > > reason you prefer this or just to be more standard?
> > > > > > > >
> > > > > > > > - Future: Basically I didn't see any particular advantage.
> The
> > > > > cancel()
> > > > > > > > method doesn't really make sense so probably wouldn't work.
> > > > Likewise
> > > > > I
> > > > > > > > dislike the checked exceptions it requires. Basically I just
> > > wrote
> > > > > out
> > > > > > > some
> > > > > > > > code examples and it seemed cleaner with a special purpose
> > > object.
> > > > I
> > > > > > > wasn't
> > > > > > > > actually aware of plans for improved futures in java 8 or the
> > > other
> > > > > > > > integrations. Maybe you could elaborate on this a bit and
> show
> > > how
> > > > it
> > > > > > > would
> > > > > > > > be used? Sounds promising, I just don't know a lot about it.
> > > > > > > >
> > > > > > > > -Jay
> > > > > > > >
> > > > > > > >
> > > > > > > > On Fri, Jan 24, 2014 at 3:30 PM, Clark Breyman <
> > > clark@breyman.com>
> > > > > > > wrote:
> > > > > > > >
> > > > > > > > > Jay - Thanks for the call for comments. Here's some initial
> > > > input:
> > > > > > > > >
> > > > > > > > > - Make message serialization a client responsibility
> (making
> > > all
> > > > > > > messages
> > > > > > > > > byte[]). Reflection-based loading makes it harder to use
> > > generic
> > > > > > codecs
> > > > > > > > > (e.g.  Envelope<PREFIX, DATA, SUFFIX>) or build up codec
> > > > > > > > programmatically.
> > > > > > > > > Non-default partitioning should require an explicit
> partition
> > > > key.
> > > > > > > > >
> > > > > > > > > - I really like the fact that it will be native Java.
> Please
> > > > > consider
> > > > > > > > using
> > > > > > > > > native maven and not sbt, gradle, ivy, etc as they don't
> > > reliably
> > > > > > play
> > > > > > > > nice
> > > > > > > > > in the maven ecosystem. A jar without a well-formed pom
> > doesn't
> > > > > feel
> > > > > > > > like a
> > > > > > > > > real artifact. The pom's generated by sbt et al. are not
> well
> > > > > formed.
> > > > > > > > Using
> > > > > > > > > maven will make builds and IDE integration much smoother.
> > > > > > > > >
> > > > > > > > > - Look at Nick Telford's dropwizard-extras package in which
> > he
> > > > > > defines
> > > > > > > > some
> > > > > > > > > Jackson-compatible POJO's for loading configuration. Seems
> > like
> > > > > your
> > > > > > > > client
> > > > > > > > > migration is similar. The config objects should have
> > > constructors
> > > > > or
> > > > > > > > > factories that accept Map<String, String> and Properties
> for
> > > ease
> > > > > of
> > > > > > > > > migration.
> > > > > > > > >
> > > > > > > > > - Would you consider using the org.apache.kafka package for
> > the
> > > > new
> > > > > > API
> > > > > > > > > (quibble)
> > > > > > > > >
> > > > > > > > > - Why create your own futures rather than use
> > > > > > > > > java.util.concurrent.Future<Long> or similar? Standard
> > futures
> > > > will
> > > > > > > play
> > > > > > > > > nice with other reactive libs and things like J8's
> > > > > ComposableFuture.
> > > > > > > > >
> > > > > > > > > Thanks again,
> > > > > > > > > C
> > > > > > > > >
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > On Fri, Jan 24, 2014 at 2:46 PM, Roger Hoover <
> > > > > > roger.hoover@gmail.com
> > > > > > > > > >wrote:
> > > > > > > > >
> > > > > > > > > > A couple comments:
> > > > > > > > > >
> > > > > > > > > > 1) Why does the config use a broker list instead of
> > > discovering
> > > > > the
> > > > > > > > > brokers
> > > > > > > > > > in ZooKeeper?  It doesn't match the HighLevelConsumer
> API.
> > > > > > > > > >
> > > > > > > > > > 2) It looks like broker connections are created on
> demand.
> > >  I'm
> > > > > > > > wondering
> > > > > > > > > > if sometimes you might want to flush out config or
> network
> > > > > > > connectivity
> > > > > > > > > > issues before pushing the first message through.
> > > > > > > > > >
> > > > > > > > > > Should there also be a KafkaProducer.connect() or .open()
> > > > method
> > > > > or
> > > > > > > > > > connectAll()?  I guess it would try to connect to all
> > brokers
> > > > in
> > > > > > the
> > > > > > > > > > BROKER_LIST_CONFIG
> > > > > > > > > >
> > > > > > > > > > HTH,
> > > > > > > > > >
> > > > > > > > > > Roger
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > On Fri, Jan 24, 2014 at 11:54 AM, Jay Kreps <
> > > > jay.kreps@gmail.com
> > > > > >
> > > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > As mentioned in a previous email we are working on a
> > > > > > > > re-implementation
> > > > > > > > > of
> > > > > > > > > > > the producer. I would like to use this email thread to
> > > > discuss
> > > > > > the
> > > > > > > > > > details
> > > > > > > > > > > of the public API and the configuration. I would love
> for
> > > us
> > > > to
> > > > > > be
> > > > > > > > > > > incredibly picky about this public api now so it is as
> > good
> > > > as
> > > > > > > > possible
> > > > > > > > > > and
> > > > > > > > > > > we don't need to break it in the future.
> > > > > > > > > > >
> > > > > > > > > > > The best way to get a feel for the API is actually to
> > take
> > > a
> > > > > look
> > > > > > > at
> > > > > > > > > the
> > > > > > > > > > > javadoc, my hope is to get the api docs good enough so
> > that
> > > > it
> > > > > is
> > > > > > > > > > > self-explanatory:
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> http://empathybox.com/kafka-javadoc/index.html?kafka/clients/producer/KafkaProducer.html
> > > > > > > > > > >
> > > > > > > > > > > Please take a look at this API and give me any thoughts
> > you
> > > > may
> > > > > > > have!
> > > > > > > > > > >
> > > > > > > > > > > It may also be reasonable to take a look at the
> configs:
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> http://empathybox.com/kafka-javadoc/kafka/clients/producer/ProducerConfig.html
> > > > > > > > > > >
> > > > > > > > > > > The actual code is posted here:
> > > > > > > > > > > https://issues.apache.org/jira/browse/KAFKA-1227
> > > > > > > > > > >
> > > > > > > > > > > A few questions or comments to kick things off:
> > > > > > > > > > > 1. We need to make a decision on whether serialization
> of
> > > the
> > > > > > > user's
> > > > > > > > > key
> > > > > > > > > > > and value should be done by the user (with our api just
> > > > taking
> > > > > > > > byte[])
> > > > > > > > > or
> > > > > > > > > > > if we should take an object and allow the user to
> > > configure a
> > > > > > > > > Serializer
> > > > > > > > > > > class which we instantiate via reflection. We take the
> > > later
> > > > > > > approach
> > > > > > > > > in
> > > > > > > > > > > the current producer, and I have carried this through
> to
> > > this
> > > > > > > > > prototype.
> > > > > > > > > > > The tradeoff I see is this: taking byte[] is actually
> > > > simpler,
> > > > > > the
> > > > > > > > user
> > > > > > > > > > can
> > > > > > > > > > > directly do whatever serialization they like. The
> > > > complication
> > > > > is
> > > > > > > > > > actually
> > > > > > > > > > > partitioning. Currently partitioning is done by a
> similar
> > > > > plug-in
> > > > > > > api
> > > > > > > > > > > (Partitioner) which the user can implement and
> configure
> > to
> > > > > > > override
> > > > > > > > > how
> > > > > > > > > > > partitions are assigned. If we take byte[] as input
> then
> > we
> > > > > have
> > > > > > no
> > > > > > > > > > access
> > > > > > > > > > > to the original object and partitioning MUST be done on
> > the
> > > > > > byte[].
> > > > > > > > > This
> > > > > > > > > > is
> > > > > > > > > > > fine for hash partitioning. However for various types
> of
> > > > > semantic
> > > > > > > > > > > partitioning (range partitioning, or whatever) you
> would
> > > want
> > > > > > > access
> > > > > > > > to
> > > > > > > > > > the
> > > > > > > > > > > original object. In the current approach a producer who
> > > > wishes
> > > > > to
> > > > > > > > send
> > > > > > > > > > > byte[] they have serialized in their own code can
> > configure
> > > > the
> > > > > > > > > > > BytesSerialization we supply which is just a "no op"
> > > > > > serialization.
> > > > > > > > > > > 2. We should obsess over naming and make sure each of
> the
> > > > class
> > > > > > > names
> > > > > > > > > are
> > > > > > > > > > > good.
> > > > > > > > > > > 3. Jun has already pointed out that we need to include
> > the
> > > > > topic
> > > > > > > and
> > > > > > > > > > > partition in the response, which is absolutely right. I
> > > > haven't
> > > > > > > done
> > > > > > > > > that
> > > > > > > > > > > yet but that definitely needs to be there.
> > > > > > > > > > > 4. Currently RecordSend.await will throw an exception
> if
> > > the
> > > > > > > request
> > > > > > > > > > > failed. The intention here is that
> > > > > producer.send(message).await()
> > > > > > > > > exactly
> > > > > > > > > > > simulates a synchronous call. Guozhang has noted that
> > this
> > > > is a
> > > > > > > > little
> > > > > > > > > > > annoying since the user must then catch exceptions.
> > However
> > > > if
> > > > > we
> > > > > > > > > remove
> > > > > > > > > > > this then if the user doesn't check for errors they
> won't
> > > > know
> > > > > > one
> > > > > > > > has
> > > > > > > > > > > occurred, which I predict will be a common mistake.
> > > > > > > > > > > 5. Perhaps there is more we could do to make the async
> > > > > callbacks
> > > > > > > and
> > > > > > > > > > future
> > > > > > > > > > > we give back intuitive and easy to program against?
> > > > > > > > > > >
> > > > > > > > > > > Some background info on implementation:
> > > > > > > > > > >
> > > > > > > > > > > At a high level the primary difference in this producer
> > is
> > > > that
> > > > > > it
> > > > > > > > > > removes
> > > > > > > > > > > the distinction between the "sync" and "async"
> producer.
> > > > > > > Effectively
> > > > > > > > > all
> > > > > > > > > > > requests are sent asynchronously but always return a
> > future
> > > > > > > response
> > > > > > > > > > object
> > > > > > > > > > > that gives the offset as well as any error that may
> have
> > > > > occurred
> > > > > > > > when
> > > > > > > > > > the
> > > > > > > > > > > request is complete. The batching that is done in the
> > async
> > > > > > > producer
> > > > > > > > > only
> > > > > > > > > > > today is done whenever possible now. This means that
> the
> > > sync
> > > > > > > > producer,
> > > > > > > > > > > under load, can get performance as good as the async
> > > producer
> > > > > > > > > > (preliminary
> > > > > > > > > > > results show the producer getting 1m messages/sec).
> This
> > > > works
> > > > > > > > similar
> > > > > > > > > to
> > > > > > > > > > > group commit in databases but with respect to the
> actual
> > > > > network
> > > > > > > > > > > transmission--any messages that arrive while a send is
> in
> > > > > > progress
> > > > > > > > are
> > > > > > > > > > > batched together. It is also possible to encourage
> > batching
> > > > > even
> > > > > > > > under
> > > > > > > > > > low
> > > > > > > > > > > load to save server resources by introducing a delay on
> > the
> > > > > send
> > > > > > to
> > > > > > > > > allow
> > > > > > > > > > > more messages to accumulate; this is done using the
> > > > > > linger.msconfig
> > > > > > > > > > (this
> > > > > > > > > > > is similar to Nagle's algorithm in TCP).
> > > > > > > > > > >
> > > > > > > > > > > This producer does all network communication
> > asynchronously
> > > > and
> > > > > > in
> > > > > > > > > > parallel
> > > > > > > > > > > to all servers so the performance penalty for acks=-1
> and
> > > > > waiting
> > > > > > > on
> > > > > > > > > > > replication should be much reduced. I haven't done much
> > > > > > > benchmarking
> > > > > > > > on
> > > > > > > > > > > this yet, though.
> > > > > > > > > > >
> > > > > > > > > > > The high level design is described a little here,
> though
> > > this
> > > > > is
> > > > > > > now
> > > > > > > > a
> > > > > > > > > > > little out of date:
> > > > > > > > > > >
> > > > > https://cwiki.apache.org/confluence/display/KAFKA/Client+Rewrite
> > > > > > > > > > >
> > > > > > > > > > > -Jay
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: New Producer Public API

Posted by Roger Hoover <ro...@gmail.com>.
That makes sense.  Thanks, Jay.


On Tue, Jan 28, 2014 at 4:38 PM, Jay Kreps <ja...@gmail.com> wrote:

> Hey Roger,
>
> We really can't use ListenableFuture directly though I agree it is nice. We
> have had some previous experience with embedding google collection classes
> in public apis, and it was quite the disaster. The problem has been that
> the google guys regularly go on a refactoring binge for no apparent reason.
> The result of this is that we end up having to code to a particular version
> of guava, but users use every possible version and hence any version we
> pick is incompatible for a large segment of people. I've been through many
> iterations of this first with google collections then with Guava. The worst
> part is that this incompatibility is just to get some fairly trivial
> helper/utility class.
>
> Basically I think guava is a must have for application code but highly
> shared libraries like clients really need to optimize for the end user, not
> the convenience of using pre-existing helper code.
>
> So that means either Future or the custom return object, I think.
>
> I did consider doing multiple callbacks per call and adding this to the
> send rather than the method invocation, but both added some complexity and
> it seemed both could be implemented using the api provided without too much
> trouble.
>
> -Jay
>
>
> On Tue, Jan 28, 2014 at 12:33 PM, Roger Hoover <roger.hoover@gmail.com
> >wrote:
>
> > +1 ListenableFuture: If this works similar to Deferreds in Twisted Python
> > or Promised IO in Javascript, I think this is a great pattern for
> > decoupling your callback logic from the place where the Future is
> > generated.  You can register as many callbacks as you like, each in the
> > appropriate layer of the code and have each observer get notified when
> the
> > promised i/o is complete without any of them knowing about each other.
> >
> >
> > On Tue, Jan 28, 2014 at 11:32 AM, Jay Kreps <ja...@gmail.com> wrote:
> >
> > > Hey Ross,
> > >
> > > - ListenableFuture: Interesting. That would be an alternative to the
> > direct
> > > callback support we provide. There could be pros to this, let me think
> > > about it.
> > > - We could provide layering, but I feel that the serialization is such
> a
> > > small thing we should just make a decision and chose one, it doesn't
> seem
> > > to me to justify a whole public facing layer.
> > > - Yes, this is fairly esoteric, essentially I think it is fairly
> similar
> > to
> > > databases like DynamoDB that allow you to specify two partition keys (I
> > > think DynamoDB does this...). The reasoning is that in fact there are
> > > several things you can use the key field for: (1) to compute the
> > partition
> > > to store the data in, (2) as a unique identifier to deduplicate that
> > > partition's records within a log. These two things are almost always
> the
> > > same, but occationally may differ when you want to group data in a more
> > > sophisticated way then just a hash of the primary key but still retain
> > the
> > > proper primary key for delivery to the consumer and log compaction.
> > >
> > > -Jay
> > >
> > >
> > > On Tue, Jan 28, 2014 at 3:24 AM, Ross Black <ro...@gmail.com>
> > > wrote:
> > >
> > > > Hi Jay,
> > > >
> > > > - Just to add some more info/confusion about possibly using Future
> ...
> > > >   If Kafka uses a JDK future, it plays nicely with other frameworks
> as
> > > > well.
> > > >   Google Guava has a ListenableFuture that allows callback handling
> to
> > be
> > > > added via the returned future, and allows the callbacks to be passed
> > off
> > > to
> > > > a specified executor.
> > > >
> > > >
> > > >
> > >
> >
> http://docs.guava-libraries.googlecode.com/git-history/release/javadoc/com/google/common/util/concurrent/ListenableFuture.html
> > > >   The JDK future can easily be converted to a listenable future.
> > > >
> > > > - On the question of byte[] vs Object, could this be solved by
> layering
> > > the
> > > > API?  eg. a raw producer (use byte[] and specify the partition
> number)
> > > and
> > > > a normal producer (use generic object and specify a Partitioner)?
> > > >
> > > > - I am confused by the keys in ProducerRecord and Partitioner.  What
> is
> > > the
> > > > usage for both a key and a partition key? (I am not yet using 0.8)
> > > >
> > > >
> > > > Thanks,
> > > > Ross
> > > >
> > > >
> > > >
> > > > On 28 January 2014 05:00, Xavier Stevens <xa...@gaikai.com> wrote:
> > > >
> > > > > AutoCloseable would be nice for us as most of our code is using
> Java
> > 7
> > > at
> > > > > this point.
> > > > >
> > > > > I like Dropwizard's configuration mapping to POJOs via Jackson, but
> > if
> > > > you
> > > > > wanted to stick with property maps I don't care enough to object.
> > > > >
> > > > > If the producer only dealt with bytes, is there a way we could
> still
> > > due
> > > > > partition plugins without specifying the number explicitly? I would
> > > > prefer
> > > > > to be able to pass in field(s) that would be used by the
> partitioner.
> > > > > Obviously if this wasn't possible you could always deserialize the
> > > object
> > > > > in the partitioner and grab the fields you want, but that seems
> > really
> > > > > expensive to do on every message.
> > > > >
> > > > > It would also be nice to have a Java API Encoder constructor taking
> > in
> > > > > VerifiableProperties. Scala understands how to handle "props:
> > > > > VerifiableProperties = null", but Java doesn't. So you don't run
> into
> > > > this
> > > > > problem until runtime.
> > > > >
> > > > >
> > > > > -Xavier
> > > > >
> > > > >
> > > > > On Mon, Jan 27, 2014 at 9:37 AM, Clark Breyman <cl...@breyman.com>
> > > > wrote:
> > > > >
> > > > > > Jay -
> > > > > >
> > > > > > Config - your explanation makes sense. I'm just so accustomed to
> > > having
> > > > > > Jackson automatically map my configuration objects to POJOs that
> > I've
> > > > > > stopped using property files. They are lingua franca. The only
> > > thought
> > > > > > might be to separate the config interface from the implementation
> > to
> > > > > allow
> > > > > > for alternatives, but that might undermine your point of "do it
> > this
> > > > way
> > > > > so
> > > > > > that everyone can find it where they expect it".
> > > > > >
> > > > > > Serialization: Of the options, I like 1A the best, though
> possibly
> > > with
> > > > > > either an option to specify a partition key rather than ID or a
> > > helper
> > > > to
> > > > > > translate an arbitrary byte[] or long into a partition number.
> > > > > >
> > > > > > Thanks
> > > > > > Clark
> > > > > >
> > > > > >
> > > > > > On Sun, Jan 26, 2014 at 9:13 PM, Jay Kreps <ja...@gmail.com>
> > > > wrote:
> > > > > >
> > > > > > > Thanks for the detailed thoughts. Let me elaborate on the
> config
> > > > thing.
> > > > > > >
> > > > > > > I agree that at first glance key-value strings don't seem like
> a
> > > very
> > > > > > good
> > > > > > > configuration api for a client. Surely a well-typed config
> class
> > > > would
> > > > > be
> > > > > > > better! I actually disagree and let me see if I can convince
> you.
> > > > > > >
> > > > > > > My reasoning has nothing to do with the api and everything to
> do
> > > with
> > > > > > > operations.
> > > > > > >
> > > > > > > Clients are embedded in applications which are themselves
> > > configured.
> > > > > In
> > > > > > > any place that takes operations seriously the configuration for
> > > these
> > > > > > > applications will be version controlled and maintained through
> > some
> > > > > kind
> > > > > > of
> > > > > > > config management system. If we give a config class with
> getters
> > > and
> > > > > > > setters the application has to expose those properties to its
> > > > > > > configuration. What invariably happens is that the application
> > > > exposes
> > > > > > only
> > > > > > > a choice few properties that they thought they would change.
> > > > > Furthermore
> > > > > > > the application will make up a name for these configs that
> seems
> > > > > > intuitive
> > > > > > > at the time in the 2 seconds the engineer spends thinking about
> > it.
> > > > > > >
> > > > > > > Now consider the result of this in the large. You end up with
> > > dozens
> > > > or
> > > > > > > hundreds of applications that have the client embedded. Each
> > > exposes
> > > > a
> > > > > > > different, inadequate subset of the possible configs, each with
> > > > > different
> > > > > > > names. It is a nightmare.
> > > > > > >
> > > > > > > If you use a string-string map the config system can directly
> > get a
> > > > > > bundle
> > > > > > > of config key-value pairs and put them into the client. This
> > means
> > > > that
> > > > > > all
> > > > > > > configuration is automatically available with the name
> documented
> > > on
> > > > > the
> > > > > > > website in every application that does this. If you upgrade to
> a
> > > new
> > > > > > kafka
> > > > > > > version with more configs those will be exposed too. If you
> > realize
> > > > > that
> > > > > > > you need to change a default you can just go through your
> configs
> > > and
> > > > > > > change it everywhere as it will have the same name everywhere.
> > > > > > >
> > > > > > > -Jay
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > On Sun, Jan 26, 2014 at 4:47 PM, Clark Breyman <
> > clark@breyman.com>
> > > > > > wrote:
> > > > > > >
> > > > > > > > Thanks Jay. I'll see if I can put together a more complete
> > > > response,
> > > > > > > > perhaps as separate threads so that topics don't get
> entangled.
> > > In
> > > > > the
> > > > > > > mean
> > > > > > > > time, here's a couple responses:
> > > > > > > >
> > > > > > > > Serialization: you've broken out a sub-thread so i'll reply
> > > there.
> > > > My
> > > > > > > bias
> > > > > > > > is that I like generics (except for type-erasure) and in
> > > particular
> > > > > > they
> > > > > > > > make it easy to compose serializers for compound payloads
> (e.g.
> > > > when
> > > > > a
> > > > > > > > common header wraps a payload of parameterized type). I'll
> > > respond
> > > > to
> > > > > > > your
> > > > > > > > 4-options message with an example.
> > > > > > > >
> > > > > > > > Build: I've seen a lot of "maven-compatible" build systems
> > > produce
> > > > > > > > "artifacts" that aren't really artifacts - no embedded POM
> or,
> > > > worst,
> > > > > > > > malformed POM. I know the sbt-generated artifacts were this
> > way -
> > > > > onus
> > > > > > is
> > > > > > > > on me to see what gradle is spitting out and what a maven
> build
> > > > might
> > > > > > > look
> > > > > > > > like. Maven may be old and boring, but it gets out of the way
> > and
> > > > > > > > integrates really seamlessly with a lot of IDEs. When some
> > scala
> > > > > > > projects I
> > > > > > > > was working on in the fall of 2011 switched from sbt to
> maven,
> > > > build
> > > > > > > became
> > > > > > > > a non-issue.
> > > > > > > >
> > > > > > > > Config: Not a big deal  and no, I don't think a dropwizard
> > > > dependency
> > > > > > is
> > > > > > > > appropriate. I do like using simple entity beans (POJO's not
> > > j2EE)
> > > > > for
> > > > > > > > configuration, especially if they can be marshalled without
> > > > > annotation
> > > > > > by
> > > > > > > > Jackson. I only mentioned the dropwizard-extras  because it
> has
> > > > some
> > > > > > > entity
> > > > > > > > bean versions of the ZK and Kafka configs.
> > > > > > > >
> > > > > > > > Domain-packaging: Also not a big deal - it's what's expected
> > and
> > > > it's
> > > > > > > > pretty free in most IDE's. The advantages I see is that it is
> > > clear
> > > > > > > whether
> > > > > > > > something is from the Apache Kafka project and whether
> > something
> > > is
> > > > > > from
> > > > > > > > another org and related to Kafka. That said, nothing really
> > > > enforces
> > > > > > it.
> > > > > > > >
> > > > > > > > Futures: I'll see if I can create some examples to
> demonstrate
> > > > Future
> > > > > > > > making interop easier.
> > > > > > > >
> > > > > > > > Regards,
> > > > > > > > C
> > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > > On Fri, Jan 24, 2014 at 4:36 PM, Jay Kreps <
> > jay.kreps@gmail.com>
> > > > > > wrote:
> > > > > > > >
> > > > > > > > > Hey Clark,
> > > > > > > > >
> > > > > > > > > - Serialization: Yes I agree with these though I don't
> > consider
> > > > the
> > > > > > > loss
> > > > > > > > of
> > > > > > > > > generics a big issue. I'll try to summarize what I would
> > > consider
> > > > > the
> > > > > > > > best
> > > > > > > > > alternative api with raw byte[].
> > > > > > > > >
> > > > > > > > > - Maven: We had this debate a few months back and the
> > consensus
> > > > was
> > > > > > > > gradle.
> > > > > > > > > Is there a specific issue with the poms gradle makes? I am
> > > > > extremely
> > > > > > > > loath
> > > > > > > > > to revisit the issue as build issues are a recurring thing
> > and
> > > no
> > > > > one
> > > > > > > > ever
> > > > > > > > > agrees and ultimately our build needs are very simple.
> > > > > > > > >
> > > > > > > > > - Config: I'm not sure if I follow the point. Are you
> saying
> > we
> > > > > > should
> > > > > > > > use
> > > > > > > > > something in dropwizard for config? One principle here is
> to
> > > try
> > > > to
> > > > > > > > remove
> > > > > > > > > as many client dependencies as possible as we inevitably
> run
> > > into
> > > > > > > > terrible
> > > > > > > > > compatibility issues with users who use the same library or
> > its
> > > > > > > > > dependencies at different versions. Or are you talking
> about
> > > > > > > maintaining
> > > > > > > > > compatibility with existing config parameters? I think as
> > much
> > > > as a
> > > > > > > > config
> > > > > > > > > in the existing client makes sense it should have the same
> > name
> > > > (I
> > > > > > was
> > > > > > > a
> > > > > > > > > bit sloppy about that so I'll fix any errors there). There
> > are
> > > a
> > > > > few
> > > > > > > new
> > > > > > > > > things and we should give those reasonable defaults. I
> think
> > > > config
> > > > > > is
> > > > > > > > > important so I'll start a thread on the config package in
> > > there.
> > > > > > > > >
> > > > > > > > > - org.apache.kafka: We could do this. I always considered
> it
> > > kind
> > > > > of
> > > > > > an
> > > > > > > > odd
> > > > > > > > > thing Java programmers do that has no real motivation (but
> I
> > > > could
> > > > > be
> > > > > > > > > re-educated!). I don't think it ends up reducing naming
> > > conflicts
> > > > > in
> > > > > > > > > practice and it adds a lot of noise and nested directories.
> > Is
> > > > > there
> > > > > > a
> > > > > > > > > reason you prefer this or just to be more standard?
> > > > > > > > >
> > > > > > > > > - Future: Basically I didn't see any particular advantage.
> > The
> > > > > > cancel()
> > > > > > > > > method doesn't really make sense so probably wouldn't work.
> > > > > Likewise
> > > > > > I
> > > > > > > > > dislike the checked exceptions it requires. Basically I
> just
> > > > wrote
> > > > > > out
> > > > > > > > some
> > > > > > > > > code examples and it seemed cleaner with a special purpose
> > > > object.
> > > > > I
> > > > > > > > wasn't
> > > > > > > > > actually aware of plans for improved futures in java 8 or
> the
> > > > other
> > > > > > > > > integrations. Maybe you could elaborate on this a bit and
> > show
> > > > how
> > > > > it
> > > > > > > > would
> > > > > > > > > be used? Sounds promising, I just don't know a lot about
> it.
> > > > > > > > >
> > > > > > > > > -Jay
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > On Fri, Jan 24, 2014 at 3:30 PM, Clark Breyman <
> > > > clark@breyman.com>
> > > > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > Jay - Thanks for the call for comments. Here's some
> initial
> > > > > input:
> > > > > > > > > >
> > > > > > > > > > - Make message serialization a client responsibility
> > (making
> > > > all
> > > > > > > > messages
> > > > > > > > > > byte[]). Reflection-based loading makes it harder to use
> > > > generic
> > > > > > > codecs
> > > > > > > > > > (e.g.  Envelope<PREFIX, DATA, SUFFIX>) or build up codec
> > > > > > > > > programmatically.
> > > > > > > > > > Non-default partitioning should require an explicit
> > partition
> > > > > key.
> > > > > > > > > >
> > > > > > > > > > - I really like the fact that it will be native Java.
> > Please
> > > > > > consider
> > > > > > > > > using
> > > > > > > > > > native maven and not sbt, gradle, ivy, etc as they don't
> > > > reliably
> > > > > > > play
> > > > > > > > > nice
> > > > > > > > > > in the maven ecosystem. A jar without a well-formed pom
> > > doesn't
> > > > > > feel
> > > > > > > > > like a
> > > > > > > > > > real artifact. The pom's generated by sbt et al. are not
> > well
> > > > > > formed.
> > > > > > > > > Using
> > > > > > > > > > maven will make builds and IDE integration much smoother.
> > > > > > > > > >
> > > > > > > > > > - Look at Nick Telford's dropwizard-extras package in
> which
> > > he
> > > > > > > defines
> > > > > > > > > some
> > > > > > > > > > Jackson-compatible POJO's for loading configuration.
> Seems
> > > like
> > > > > > your
> > > > > > > > > client
> > > > > > > > > > migration is similar. The config objects should have
> > > > constructors
> > > > > > or
> > > > > > > > > > factories that accept Map<String, String> and Properties
> > for
> > > > ease
> > > > > > of
> > > > > > > > > > migration.
> > > > > > > > > >
> > > > > > > > > > - Would you consider using the org.apache.kafka package
> for
> > > the
> > > > > new
> > > > > > > API
> > > > > > > > > > (quibble)
> > > > > > > > > >
> > > > > > > > > > - Why create your own futures rather than use
> > > > > > > > > > java.util.concurrent.Future<Long> or similar? Standard
> > > futures
> > > > > will
> > > > > > > > play
> > > > > > > > > > nice with other reactive libs and things like J8's
> > > > > > ComposableFuture.
> > > > > > > > > >
> > > > > > > > > > Thanks again,
> > > > > > > > > > C
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > On Fri, Jan 24, 2014 at 2:46 PM, Roger Hoover <
> > > > > > > roger.hoover@gmail.com
> > > > > > > > > > >wrote:
> > > > > > > > > >
> > > > > > > > > > > A couple comments:
> > > > > > > > > > >
> > > > > > > > > > > 1) Why does the config use a broker list instead of
> > > > discovering
> > > > > > the
> > > > > > > > > > brokers
> > > > > > > > > > > in ZooKeeper?  It doesn't match the HighLevelConsumer
> > API.
> > > > > > > > > > >
> > > > > > > > > > > 2) It looks like broker connections are created on
> > demand.
> > > >  I'm
> > > > > > > > > wondering
> > > > > > > > > > > if sometimes you might want to flush out config or
> > network
> > > > > > > > connectivity
> > > > > > > > > > > issues before pushing the first message through.
> > > > > > > > > > >
> > > > > > > > > > > Should there also be a KafkaProducer.connect() or
> .open()
> > > > > method
> > > > > > or
> > > > > > > > > > > connectAll()?  I guess it would try to connect to all
> > > brokers
> > > > > in
> > > > > > > the
> > > > > > > > > > > BROKER_LIST_CONFIG
> > > > > > > > > > >
> > > > > > > > > > > HTH,
> > > > > > > > > > >
> > > > > > > > > > > Roger
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > On Fri, Jan 24, 2014 at 11:54 AM, Jay Kreps <
> > > > > jay.kreps@gmail.com
> > > > > > >
> > > > > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > As mentioned in a previous email we are working on a
> > > > > > > > > re-implementation
> > > > > > > > > > of
> > > > > > > > > > > > the producer. I would like to use this email thread
> to
> > > > > discuss
> > > > > > > the
> > > > > > > > > > > details
> > > > > > > > > > > > of the public API and the configuration. I would love
> > for
> > > > us
> > > > > to
> > > > > > > be
> > > > > > > > > > > > incredibly picky about this public api now so it is
> as
> > > good
> > > > > as
> > > > > > > > > possible
> > > > > > > > > > > and
> > > > > > > > > > > > we don't need to break it in the future.
> > > > > > > > > > > >
> > > > > > > > > > > > The best way to get a feel for the API is actually to
> > > take
> > > > a
> > > > > > look
> > > > > > > > at
> > > > > > > > > > the
> > > > > > > > > > > > javadoc, my hope is to get the api docs good enough
> so
> > > that
> > > > > it
> > > > > > is
> > > > > > > > > > > > self-explanatory:
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> http://empathybox.com/kafka-javadoc/index.html?kafka/clients/producer/KafkaProducer.html
> > > > > > > > > > > >
> > > > > > > > > > > > Please take a look at this API and give me any
> thoughts
> > > you
> > > > > may
> > > > > > > > have!
> > > > > > > > > > > >
> > > > > > > > > > > > It may also be reasonable to take a look at the
> > configs:
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> http://empathybox.com/kafka-javadoc/kafka/clients/producer/ProducerConfig.html
> > > > > > > > > > > >
> > > > > > > > > > > > The actual code is posted here:
> > > > > > > > > > > > https://issues.apache.org/jira/browse/KAFKA-1227
> > > > > > > > > > > >
> > > > > > > > > > > > A few questions or comments to kick things off:
> > > > > > > > > > > > 1. We need to make a decision on whether
> serialization
> > of
> > > > the
> > > > > > > > user's
> > > > > > > > > > key
> > > > > > > > > > > > and value should be done by the user (with our api
> just
> > > > > taking
> > > > > > > > > byte[])
> > > > > > > > > > or
> > > > > > > > > > > > if we should take an object and allow the user to
> > > > configure a
> > > > > > > > > > Serializer
> > > > > > > > > > > > class which we instantiate via reflection. We take
> the
> > > > later
> > > > > > > > approach
> > > > > > > > > > in
> > > > > > > > > > > > the current producer, and I have carried this through
> > to
> > > > this
> > > > > > > > > > prototype.
> > > > > > > > > > > > The tradeoff I see is this: taking byte[] is actually
> > > > > simpler,
> > > > > > > the
> > > > > > > > > user
> > > > > > > > > > > can
> > > > > > > > > > > > directly do whatever serialization they like. The
> > > > > complication
> > > > > > is
> > > > > > > > > > > actually
> > > > > > > > > > > > partitioning. Currently partitioning is done by a
> > similar
> > > > > > plug-in
> > > > > > > > api
> > > > > > > > > > > > (Partitioner) which the user can implement and
> > configure
> > > to
> > > > > > > > override
> > > > > > > > > > how
> > > > > > > > > > > > partitions are assigned. If we take byte[] as input
> > then
> > > we
> > > > > > have
> > > > > > > no
> > > > > > > > > > > access
> > > > > > > > > > > > to the original object and partitioning MUST be done
> on
> > > the
> > > > > > > byte[].
> > > > > > > > > > This
> > > > > > > > > > > is
> > > > > > > > > > > > fine for hash partitioning. However for various types
> > of
> > > > > > semantic
> > > > > > > > > > > > partitioning (range partitioning, or whatever) you
> > would
> > > > want
> > > > > > > > access
> > > > > > > > > to
> > > > > > > > > > > the
> > > > > > > > > > > > original object. In the current approach a producer
> who
> > > > > wishes
> > > > > > to
> > > > > > > > > send
> > > > > > > > > > > > byte[] they have serialized in their own code can
> > > configure
> > > > > the
> > > > > > > > > > > > BytesSerialization we supply which is just a "no op"
> > > > > > > serialization.
> > > > > > > > > > > > 2. We should obsess over naming and make sure each of
> > the
> > > > > class
> > > > > > > > names
> > > > > > > > > > are
> > > > > > > > > > > > good.
> > > > > > > > > > > > 3. Jun has already pointed out that we need to
> include
> > > the
> > > > > > topic
> > > > > > > > and
> > > > > > > > > > > > partition in the response, which is absolutely
> right. I
> > > > > haven't
> > > > > > > > done
> > > > > > > > > > that
> > > > > > > > > > > > yet but that definitely needs to be there.
> > > > > > > > > > > > 4. Currently RecordSend.await will throw an exception
> > if
> > > > the
> > > > > > > > request
> > > > > > > > > > > > failed. The intention here is that
> > > > > > producer.send(message).await()
> > > > > > > > > > exactly
> > > > > > > > > > > > simulates a synchronous call. Guozhang has noted that
> > > this
> > > > > is a
> > > > > > > > > little
> > > > > > > > > > > > annoying since the user must then catch exceptions.
> > > However
> > > > > if
> > > > > > we
> > > > > > > > > > remove
> > > > > > > > > > > > this then if the user doesn't check for errors they
> > won't
> > > > > know
> > > > > > > one
> > > > > > > > > has
> > > > > > > > > > > > occurred, which I predict will be a common mistake.
> > > > > > > > > > > > 5. Perhaps there is more we could do to make the
> async
> > > > > > callbacks
> > > > > > > > and
> > > > > > > > > > > future
> > > > > > > > > > > > we give back intuitive and easy to program against?
> > > > > > > > > > > >
> > > > > > > > > > > > Some background info on implementation:
> > > > > > > > > > > >
> > > > > > > > > > > > At a high level the primary difference in this
> producer
> > > is
> > > > > that
> > > > > > > it
> > > > > > > > > > > removes
> > > > > > > > > > > > the distinction between the "sync" and "async"
> > producer.
> > > > > > > > Effectively
> > > > > > > > > > all
> > > > > > > > > > > > requests are sent asynchronously but always return a
> > > future
> > > > > > > > response
> > > > > > > > > > > object
> > > > > > > > > > > > that gives the offset as well as any error that may
> > have
> > > > > > occurred
> > > > > > > > > when
> > > > > > > > > > > the
> > > > > > > > > > > > request is complete. The batching that is done in the
> > > async
> > > > > > > > producer
> > > > > > > > > > only
> > > > > > > > > > > > today is done whenever possible now. This means that
> > the
> > > > sync
> > > > > > > > > producer,
> > > > > > > > > > > > under load, can get performance as good as the async
> > > > producer
> > > > > > > > > > > (preliminary
> > > > > > > > > > > > results show the producer getting 1m messages/sec).
> > This
> > > > > works
> > > > > > > > > similar
> > > > > > > > > > to
> > > > > > > > > > > > group commit in databases but with respect to the
> > actual
> > > > > > network
> > > > > > > > > > > > transmission--any messages that arrive while a send
> is
> > in
> > > > > > > progress
> > > > > > > > > are
> > > > > > > > > > > > batched together. It is also possible to encourage
> > > batching
> > > > > > even
> > > > > > > > > under
> > > > > > > > > > > low
> > > > > > > > > > > > load to save server resources by introducing a delay
> on
> > > the
> > > > > > send
> > > > > > > to
> > > > > > > > > > allow
> > > > > > > > > > > > more messages to accumulate; this is done using the
> > > > > > > linger.msconfig
> > > > > > > > > > > (this
> > > > > > > > > > > > is similar to Nagle's algorithm in TCP).
> > > > > > > > > > > >
> > > > > > > > > > > > This producer does all network communication
> > > asynchronously
> > > > > and
> > > > > > > in
> > > > > > > > > > > parallel
> > > > > > > > > > > > to all servers so the performance penalty for acks=-1
> > and
> > > > > > waiting
> > > > > > > > on
> > > > > > > > > > > > replication should be much reduced. I haven't done
> much
> > > > > > > > benchmarking
> > > > > > > > > on
> > > > > > > > > > > > this yet, though.
> > > > > > > > > > > >
> > > > > > > > > > > > The high level design is described a little here,
> > though
> > > > this
> > > > > > is
> > > > > > > > now
> > > > > > > > > a
> > > > > > > > > > > > little out of date:
> > > > > > > > > > > >
> > > > > > https://cwiki.apache.org/confluence/display/KAFKA/Client+Rewrite
> > > > > > > > > > > >
> > > > > > > > > > > > -Jay
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: New Producer Public API

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

We really can't use ListenableFuture directly though I agree it is nice. We
have had some previous experience with embedding google collection classes
in public apis, and it was quite the disaster. The problem has been that
the google guys regularly go on a refactoring binge for no apparent reason.
The result of this is that we end up having to code to a particular version
of guava, but users use every possible version and hence any version we
pick is incompatible for a large segment of people. I've been through many
iterations of this first with google collections then with Guava. The worst
part is that this incompatibility is just to get some fairly trivial
helper/utility class.

Basically I think guava is a must have for application code but highly
shared libraries like clients really need to optimize for the end user, not
the convenience of using pre-existing helper code.

So that means either Future or the custom return object, I think.

I did consider doing multiple callbacks per call and adding this to the
send rather than the method invocation, but both added some complexity and
it seemed both could be implemented using the api provided without too much
trouble.

-Jay


On Tue, Jan 28, 2014 at 12:33 PM, Roger Hoover <ro...@gmail.com>wrote:

> +1 ListenableFuture: If this works similar to Deferreds in Twisted Python
> or Promised IO in Javascript, I think this is a great pattern for
> decoupling your callback logic from the place where the Future is
> generated.  You can register as many callbacks as you like, each in the
> appropriate layer of the code and have each observer get notified when the
> promised i/o is complete without any of them knowing about each other.
>
>
> On Tue, Jan 28, 2014 at 11:32 AM, Jay Kreps <ja...@gmail.com> wrote:
>
> > Hey Ross,
> >
> > - ListenableFuture: Interesting. That would be an alternative to the
> direct
> > callback support we provide. There could be pros to this, let me think
> > about it.
> > - We could provide layering, but I feel that the serialization is such a
> > small thing we should just make a decision and chose one, it doesn't seem
> > to me to justify a whole public facing layer.
> > - Yes, this is fairly esoteric, essentially I think it is fairly similar
> to
> > databases like DynamoDB that allow you to specify two partition keys (I
> > think DynamoDB does this...). The reasoning is that in fact there are
> > several things you can use the key field for: (1) to compute the
> partition
> > to store the data in, (2) as a unique identifier to deduplicate that
> > partition's records within a log. These two things are almost always the
> > same, but occationally may differ when you want to group data in a more
> > sophisticated way then just a hash of the primary key but still retain
> the
> > proper primary key for delivery to the consumer and log compaction.
> >
> > -Jay
> >
> >
> > On Tue, Jan 28, 2014 at 3:24 AM, Ross Black <ro...@gmail.com>
> > wrote:
> >
> > > Hi Jay,
> > >
> > > - Just to add some more info/confusion about possibly using Future ...
> > >   If Kafka uses a JDK future, it plays nicely with other frameworks as
> > > well.
> > >   Google Guava has a ListenableFuture that allows callback handling to
> be
> > > added via the returned future, and allows the callbacks to be passed
> off
> > to
> > > a specified executor.
> > >
> > >
> > >
> >
> http://docs.guava-libraries.googlecode.com/git-history/release/javadoc/com/google/common/util/concurrent/ListenableFuture.html
> > >   The JDK future can easily be converted to a listenable future.
> > >
> > > - On the question of byte[] vs Object, could this be solved by layering
> > the
> > > API?  eg. a raw producer (use byte[] and specify the partition number)
> > and
> > > a normal producer (use generic object and specify a Partitioner)?
> > >
> > > - I am confused by the keys in ProducerRecord and Partitioner.  What is
> > the
> > > usage for both a key and a partition key? (I am not yet using 0.8)
> > >
> > >
> > > Thanks,
> > > Ross
> > >
> > >
> > >
> > > On 28 January 2014 05:00, Xavier Stevens <xa...@gaikai.com> wrote:
> > >
> > > > AutoCloseable would be nice for us as most of our code is using Java
> 7
> > at
> > > > this point.
> > > >
> > > > I like Dropwizard's configuration mapping to POJOs via Jackson, but
> if
> > > you
> > > > wanted to stick with property maps I don't care enough to object.
> > > >
> > > > If the producer only dealt with bytes, is there a way we could still
> > due
> > > > partition plugins without specifying the number explicitly? I would
> > > prefer
> > > > to be able to pass in field(s) that would be used by the partitioner.
> > > > Obviously if this wasn't possible you could always deserialize the
> > object
> > > > in the partitioner and grab the fields you want, but that seems
> really
> > > > expensive to do on every message.
> > > >
> > > > It would also be nice to have a Java API Encoder constructor taking
> in
> > > > VerifiableProperties. Scala understands how to handle "props:
> > > > VerifiableProperties = null", but Java doesn't. So you don't run into
> > > this
> > > > problem until runtime.
> > > >
> > > >
> > > > -Xavier
> > > >
> > > >
> > > > On Mon, Jan 27, 2014 at 9:37 AM, Clark Breyman <cl...@breyman.com>
> > > wrote:
> > > >
> > > > > Jay -
> > > > >
> > > > > Config - your explanation makes sense. I'm just so accustomed to
> > having
> > > > > Jackson automatically map my configuration objects to POJOs that
> I've
> > > > > stopped using property files. They are lingua franca. The only
> > thought
> > > > > might be to separate the config interface from the implementation
> to
> > > > allow
> > > > > for alternatives, but that might undermine your point of "do it
> this
> > > way
> > > > so
> > > > > that everyone can find it where they expect it".
> > > > >
> > > > > Serialization: Of the options, I like 1A the best, though possibly
> > with
> > > > > either an option to specify a partition key rather than ID or a
> > helper
> > > to
> > > > > translate an arbitrary byte[] or long into a partition number.
> > > > >
> > > > > Thanks
> > > > > Clark
> > > > >
> > > > >
> > > > > On Sun, Jan 26, 2014 at 9:13 PM, Jay Kreps <ja...@gmail.com>
> > > wrote:
> > > > >
> > > > > > Thanks for the detailed thoughts. Let me elaborate on the config
> > > thing.
> > > > > >
> > > > > > I agree that at first glance key-value strings don't seem like a
> > very
> > > > > good
> > > > > > configuration api for a client. Surely a well-typed config class
> > > would
> > > > be
> > > > > > better! I actually disagree and let me see if I can convince you.
> > > > > >
> > > > > > My reasoning has nothing to do with the api and everything to do
> > with
> > > > > > operations.
> > > > > >
> > > > > > Clients are embedded in applications which are themselves
> > configured.
> > > > In
> > > > > > any place that takes operations seriously the configuration for
> > these
> > > > > > applications will be version controlled and maintained through
> some
> > > > kind
> > > > > of
> > > > > > config management system. If we give a config class with getters
> > and
> > > > > > setters the application has to expose those properties to its
> > > > > > configuration. What invariably happens is that the application
> > > exposes
> > > > > only
> > > > > > a choice few properties that they thought they would change.
> > > > Furthermore
> > > > > > the application will make up a name for these configs that seems
> > > > > intuitive
> > > > > > at the time in the 2 seconds the engineer spends thinking about
> it.
> > > > > >
> > > > > > Now consider the result of this in the large. You end up with
> > dozens
> > > or
> > > > > > hundreds of applications that have the client embedded. Each
> > exposes
> > > a
> > > > > > different, inadequate subset of the possible configs, each with
> > > > different
> > > > > > names. It is a nightmare.
> > > > > >
> > > > > > If you use a string-string map the config system can directly
> get a
> > > > > bundle
> > > > > > of config key-value pairs and put them into the client. This
> means
> > > that
> > > > > all
> > > > > > configuration is automatically available with the name documented
> > on
> > > > the
> > > > > > website in every application that does this. If you upgrade to a
> > new
> > > > > kafka
> > > > > > version with more configs those will be exposed too. If you
> realize
> > > > that
> > > > > > you need to change a default you can just go through your configs
> > and
> > > > > > change it everywhere as it will have the same name everywhere.
> > > > > >
> > > > > > -Jay
> > > > > >
> > > > > >
> > > > > >
> > > > > >
> > > > > > On Sun, Jan 26, 2014 at 4:47 PM, Clark Breyman <
> clark@breyman.com>
> > > > > wrote:
> > > > > >
> > > > > > > Thanks Jay. I'll see if I can put together a more complete
> > > response,
> > > > > > > perhaps as separate threads so that topics don't get entangled.
> > In
> > > > the
> > > > > > mean
> > > > > > > time, here's a couple responses:
> > > > > > >
> > > > > > > Serialization: you've broken out a sub-thread so i'll reply
> > there.
> > > My
> > > > > > bias
> > > > > > > is that I like generics (except for type-erasure) and in
> > particular
> > > > > they
> > > > > > > make it easy to compose serializers for compound payloads (e.g.
> > > when
> > > > a
> > > > > > > common header wraps a payload of parameterized type). I'll
> > respond
> > > to
> > > > > > your
> > > > > > > 4-options message with an example.
> > > > > > >
> > > > > > > Build: I've seen a lot of "maven-compatible" build systems
> > produce
> > > > > > > "artifacts" that aren't really artifacts - no embedded POM or,
> > > worst,
> > > > > > > malformed POM. I know the sbt-generated artifacts were this
> way -
> > > > onus
> > > > > is
> > > > > > > on me to see what gradle is spitting out and what a maven build
> > > might
> > > > > > look
> > > > > > > like. Maven may be old and boring, but it gets out of the way
> and
> > > > > > > integrates really seamlessly with a lot of IDEs. When some
> scala
> > > > > > projects I
> > > > > > > was working on in the fall of 2011 switched from sbt to maven,
> > > build
> > > > > > became
> > > > > > > a non-issue.
> > > > > > >
> > > > > > > Config: Not a big deal  and no, I don't think a dropwizard
> > > dependency
> > > > > is
> > > > > > > appropriate. I do like using simple entity beans (POJO's not
> > j2EE)
> > > > for
> > > > > > > configuration, especially if they can be marshalled without
> > > > annotation
> > > > > by
> > > > > > > Jackson. I only mentioned the dropwizard-extras  because it has
> > > some
> > > > > > entity
> > > > > > > bean versions of the ZK and Kafka configs.
> > > > > > >
> > > > > > > Domain-packaging: Also not a big deal - it's what's expected
> and
> > > it's
> > > > > > > pretty free in most IDE's. The advantages I see is that it is
> > clear
> > > > > > whether
> > > > > > > something is from the Apache Kafka project and whether
> something
> > is
> > > > > from
> > > > > > > another org and related to Kafka. That said, nothing really
> > > enforces
> > > > > it.
> > > > > > >
> > > > > > > Futures: I'll see if I can create some examples to demonstrate
> > > Future
> > > > > > > making interop easier.
> > > > > > >
> > > > > > > Regards,
> > > > > > > C
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > On Fri, Jan 24, 2014 at 4:36 PM, Jay Kreps <
> jay.kreps@gmail.com>
> > > > > wrote:
> > > > > > >
> > > > > > > > Hey Clark,
> > > > > > > >
> > > > > > > > - Serialization: Yes I agree with these though I don't
> consider
> > > the
> > > > > > loss
> > > > > > > of
> > > > > > > > generics a big issue. I'll try to summarize what I would
> > consider
> > > > the
> > > > > > > best
> > > > > > > > alternative api with raw byte[].
> > > > > > > >
> > > > > > > > - Maven: We had this debate a few months back and the
> consensus
> > > was
> > > > > > > gradle.
> > > > > > > > Is there a specific issue with the poms gradle makes? I am
> > > > extremely
> > > > > > > loath
> > > > > > > > to revisit the issue as build issues are a recurring thing
> and
> > no
> > > > one
> > > > > > > ever
> > > > > > > > agrees and ultimately our build needs are very simple.
> > > > > > > >
> > > > > > > > - Config: I'm not sure if I follow the point. Are you saying
> we
> > > > > should
> > > > > > > use
> > > > > > > > something in dropwizard for config? One principle here is to
> > try
> > > to
> > > > > > > remove
> > > > > > > > as many client dependencies as possible as we inevitably run
> > into
> > > > > > > terrible
> > > > > > > > compatibility issues with users who use the same library or
> its
> > > > > > > > dependencies at different versions. Or are you talking about
> > > > > > maintaining
> > > > > > > > compatibility with existing config parameters? I think as
> much
> > > as a
> > > > > > > config
> > > > > > > > in the existing client makes sense it should have the same
> name
> > > (I
> > > > > was
> > > > > > a
> > > > > > > > bit sloppy about that so I'll fix any errors there). There
> are
> > a
> > > > few
> > > > > > new
> > > > > > > > things and we should give those reasonable defaults. I think
> > > config
> > > > > is
> > > > > > > > important so I'll start a thread on the config package in
> > there.
> > > > > > > >
> > > > > > > > - org.apache.kafka: We could do this. I always considered it
> > kind
> > > > of
> > > > > an
> > > > > > > odd
> > > > > > > > thing Java programmers do that has no real motivation (but I
> > > could
> > > > be
> > > > > > > > re-educated!). I don't think it ends up reducing naming
> > conflicts
> > > > in
> > > > > > > > practice and it adds a lot of noise and nested directories.
> Is
> > > > there
> > > > > a
> > > > > > > > reason you prefer this or just to be more standard?
> > > > > > > >
> > > > > > > > - Future: Basically I didn't see any particular advantage.
> The
> > > > > cancel()
> > > > > > > > method doesn't really make sense so probably wouldn't work.
> > > > Likewise
> > > > > I
> > > > > > > > dislike the checked exceptions it requires. Basically I just
> > > wrote
> > > > > out
> > > > > > > some
> > > > > > > > code examples and it seemed cleaner with a special purpose
> > > object.
> > > > I
> > > > > > > wasn't
> > > > > > > > actually aware of plans for improved futures in java 8 or the
> > > other
> > > > > > > > integrations. Maybe you could elaborate on this a bit and
> show
> > > how
> > > > it
> > > > > > > would
> > > > > > > > be used? Sounds promising, I just don't know a lot about it.
> > > > > > > >
> > > > > > > > -Jay
> > > > > > > >
> > > > > > > >
> > > > > > > > On Fri, Jan 24, 2014 at 3:30 PM, Clark Breyman <
> > > clark@breyman.com>
> > > > > > > wrote:
> > > > > > > >
> > > > > > > > > Jay - Thanks for the call for comments. Here's some initial
> > > > input:
> > > > > > > > >
> > > > > > > > > - Make message serialization a client responsibility
> (making
> > > all
> > > > > > > messages
> > > > > > > > > byte[]). Reflection-based loading makes it harder to use
> > > generic
> > > > > > codecs
> > > > > > > > > (e.g.  Envelope<PREFIX, DATA, SUFFIX>) or build up codec
> > > > > > > > programmatically.
> > > > > > > > > Non-default partitioning should require an explicit
> partition
> > > > key.
> > > > > > > > >
> > > > > > > > > - I really like the fact that it will be native Java.
> Please
> > > > > consider
> > > > > > > > using
> > > > > > > > > native maven and not sbt, gradle, ivy, etc as they don't
> > > reliably
> > > > > > play
> > > > > > > > nice
> > > > > > > > > in the maven ecosystem. A jar without a well-formed pom
> > doesn't
> > > > > feel
> > > > > > > > like a
> > > > > > > > > real artifact. The pom's generated by sbt et al. are not
> well
> > > > > formed.
> > > > > > > > Using
> > > > > > > > > maven will make builds and IDE integration much smoother.
> > > > > > > > >
> > > > > > > > > - Look at Nick Telford's dropwizard-extras package in which
> > he
> > > > > > defines
> > > > > > > > some
> > > > > > > > > Jackson-compatible POJO's for loading configuration. Seems
> > like
> > > > > your
> > > > > > > > client
> > > > > > > > > migration is similar. The config objects should have
> > > constructors
> > > > > or
> > > > > > > > > factories that accept Map<String, String> and Properties
> for
> > > ease
> > > > > of
> > > > > > > > > migration.
> > > > > > > > >
> > > > > > > > > - Would you consider using the org.apache.kafka package for
> > the
> > > > new
> > > > > > API
> > > > > > > > > (quibble)
> > > > > > > > >
> > > > > > > > > - Why create your own futures rather than use
> > > > > > > > > java.util.concurrent.Future<Long> or similar? Standard
> > futures
> > > > will
> > > > > > > play
> > > > > > > > > nice with other reactive libs and things like J8's
> > > > > ComposableFuture.
> > > > > > > > >
> > > > > > > > > Thanks again,
> > > > > > > > > C
> > > > > > > > >
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > On Fri, Jan 24, 2014 at 2:46 PM, Roger Hoover <
> > > > > > roger.hoover@gmail.com
> > > > > > > > > >wrote:
> > > > > > > > >
> > > > > > > > > > A couple comments:
> > > > > > > > > >
> > > > > > > > > > 1) Why does the config use a broker list instead of
> > > discovering
> > > > > the
> > > > > > > > > brokers
> > > > > > > > > > in ZooKeeper?  It doesn't match the HighLevelConsumer
> API.
> > > > > > > > > >
> > > > > > > > > > 2) It looks like broker connections are created on
> demand.
> > >  I'm
> > > > > > > > wondering
> > > > > > > > > > if sometimes you might want to flush out config or
> network
> > > > > > > connectivity
> > > > > > > > > > issues before pushing the first message through.
> > > > > > > > > >
> > > > > > > > > > Should there also be a KafkaProducer.connect() or .open()
> > > > method
> > > > > or
> > > > > > > > > > connectAll()?  I guess it would try to connect to all
> > brokers
> > > > in
> > > > > > the
> > > > > > > > > > BROKER_LIST_CONFIG
> > > > > > > > > >
> > > > > > > > > > HTH,
> > > > > > > > > >
> > > > > > > > > > Roger
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > On Fri, Jan 24, 2014 at 11:54 AM, Jay Kreps <
> > > > jay.kreps@gmail.com
> > > > > >
> > > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > As mentioned in a previous email we are working on a
> > > > > > > > re-implementation
> > > > > > > > > of
> > > > > > > > > > > the producer. I would like to use this email thread to
> > > > discuss
> > > > > > the
> > > > > > > > > > details
> > > > > > > > > > > of the public API and the configuration. I would love
> for
> > > us
> > > > to
> > > > > > be
> > > > > > > > > > > incredibly picky about this public api now so it is as
> > good
> > > > as
> > > > > > > > possible
> > > > > > > > > > and
> > > > > > > > > > > we don't need to break it in the future.
> > > > > > > > > > >
> > > > > > > > > > > The best way to get a feel for the API is actually to
> > take
> > > a
> > > > > look
> > > > > > > at
> > > > > > > > > the
> > > > > > > > > > > javadoc, my hope is to get the api docs good enough so
> > that
> > > > it
> > > > > is
> > > > > > > > > > > self-explanatory:
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> http://empathybox.com/kafka-javadoc/index.html?kafka/clients/producer/KafkaProducer.html
> > > > > > > > > > >
> > > > > > > > > > > Please take a look at this API and give me any thoughts
> > you
> > > > may
> > > > > > > have!
> > > > > > > > > > >
> > > > > > > > > > > It may also be reasonable to take a look at the
> configs:
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> http://empathybox.com/kafka-javadoc/kafka/clients/producer/ProducerConfig.html
> > > > > > > > > > >
> > > > > > > > > > > The actual code is posted here:
> > > > > > > > > > > https://issues.apache.org/jira/browse/KAFKA-1227
> > > > > > > > > > >
> > > > > > > > > > > A few questions or comments to kick things off:
> > > > > > > > > > > 1. We need to make a decision on whether serialization
> of
> > > the
> > > > > > > user's
> > > > > > > > > key
> > > > > > > > > > > and value should be done by the user (with our api just
> > > > taking
> > > > > > > > byte[])
> > > > > > > > > or
> > > > > > > > > > > if we should take an object and allow the user to
> > > configure a
> > > > > > > > > Serializer
> > > > > > > > > > > class which we instantiate via reflection. We take the
> > > later
> > > > > > > approach
> > > > > > > > > in
> > > > > > > > > > > the current producer, and I have carried this through
> to
> > > this
> > > > > > > > > prototype.
> > > > > > > > > > > The tradeoff I see is this: taking byte[] is actually
> > > > simpler,
> > > > > > the
> > > > > > > > user
> > > > > > > > > > can
> > > > > > > > > > > directly do whatever serialization they like. The
> > > > complication
> > > > > is
> > > > > > > > > > actually
> > > > > > > > > > > partitioning. Currently partitioning is done by a
> similar
> > > > > plug-in
> > > > > > > api
> > > > > > > > > > > (Partitioner) which the user can implement and
> configure
> > to
> > > > > > > override
> > > > > > > > > how
> > > > > > > > > > > partitions are assigned. If we take byte[] as input
> then
> > we
> > > > > have
> > > > > > no
> > > > > > > > > > access
> > > > > > > > > > > to the original object and partitioning MUST be done on
> > the
> > > > > > byte[].
> > > > > > > > > This
> > > > > > > > > > is
> > > > > > > > > > > fine for hash partitioning. However for various types
> of
> > > > > semantic
> > > > > > > > > > > partitioning (range partitioning, or whatever) you
> would
> > > want
> > > > > > > access
> > > > > > > > to
> > > > > > > > > > the
> > > > > > > > > > > original object. In the current approach a producer who
> > > > wishes
> > > > > to
> > > > > > > > send
> > > > > > > > > > > byte[] they have serialized in their own code can
> > configure
> > > > the
> > > > > > > > > > > BytesSerialization we supply which is just a "no op"
> > > > > > serialization.
> > > > > > > > > > > 2. We should obsess over naming and make sure each of
> the
> > > > class
> > > > > > > names
> > > > > > > > > are
> > > > > > > > > > > good.
> > > > > > > > > > > 3. Jun has already pointed out that we need to include
> > the
> > > > > topic
> > > > > > > and
> > > > > > > > > > > partition in the response, which is absolutely right. I
> > > > haven't
> > > > > > > done
> > > > > > > > > that
> > > > > > > > > > > yet but that definitely needs to be there.
> > > > > > > > > > > 4. Currently RecordSend.await will throw an exception
> if
> > > the
> > > > > > > request
> > > > > > > > > > > failed. The intention here is that
> > > > > producer.send(message).await()
> > > > > > > > > exactly
> > > > > > > > > > > simulates a synchronous call. Guozhang has noted that
> > this
> > > > is a
> > > > > > > > little
> > > > > > > > > > > annoying since the user must then catch exceptions.
> > However
> > > > if
> > > > > we
> > > > > > > > > remove
> > > > > > > > > > > this then if the user doesn't check for errors they
> won't
> > > > know
> > > > > > one
> > > > > > > > has
> > > > > > > > > > > occurred, which I predict will be a common mistake.
> > > > > > > > > > > 5. Perhaps there is more we could do to make the async
> > > > > callbacks
> > > > > > > and
> > > > > > > > > > future
> > > > > > > > > > > we give back intuitive and easy to program against?
> > > > > > > > > > >
> > > > > > > > > > > Some background info on implementation:
> > > > > > > > > > >
> > > > > > > > > > > At a high level the primary difference in this producer
> > is
> > > > that
> > > > > > it
> > > > > > > > > > removes
> > > > > > > > > > > the distinction between the "sync" and "async"
> producer.
> > > > > > > Effectively
> > > > > > > > > all
> > > > > > > > > > > requests are sent asynchronously but always return a
> > future
> > > > > > > response
> > > > > > > > > > object
> > > > > > > > > > > that gives the offset as well as any error that may
> have
> > > > > occurred
> > > > > > > > when
> > > > > > > > > > the
> > > > > > > > > > > request is complete. The batching that is done in the
> > async
> > > > > > > producer
> > > > > > > > > only
> > > > > > > > > > > today is done whenever possible now. This means that
> the
> > > sync
> > > > > > > > producer,
> > > > > > > > > > > under load, can get performance as good as the async
> > > producer
> > > > > > > > > > (preliminary
> > > > > > > > > > > results show the producer getting 1m messages/sec).
> This
> > > > works
> > > > > > > > similar
> > > > > > > > > to
> > > > > > > > > > > group commit in databases but with respect to the
> actual
> > > > > network
> > > > > > > > > > > transmission--any messages that arrive while a send is
> in
> > > > > > progress
> > > > > > > > are
> > > > > > > > > > > batched together. It is also possible to encourage
> > batching
> > > > > even
> > > > > > > > under
> > > > > > > > > > low
> > > > > > > > > > > load to save server resources by introducing a delay on
> > the
> > > > > send
> > > > > > to
> > > > > > > > > allow
> > > > > > > > > > > more messages to accumulate; this is done using the
> > > > > > linger.msconfig
> > > > > > > > > > (this
> > > > > > > > > > > is similar to Nagle's algorithm in TCP).
> > > > > > > > > > >
> > > > > > > > > > > This producer does all network communication
> > asynchronously
> > > > and
> > > > > > in
> > > > > > > > > > parallel
> > > > > > > > > > > to all servers so the performance penalty for acks=-1
> and
> > > > > waiting
> > > > > > > on
> > > > > > > > > > > replication should be much reduced. I haven't done much
> > > > > > > benchmarking
> > > > > > > > on
> > > > > > > > > > > this yet, though.
> > > > > > > > > > >
> > > > > > > > > > > The high level design is described a little here,
> though
> > > this
> > > > > is
> > > > > > > now
> > > > > > > > a
> > > > > > > > > > > little out of date:
> > > > > > > > > > >
> > > > > https://cwiki.apache.org/confluence/display/KAFKA/Client+Rewrite
> > > > > > > > > > >
> > > > > > > > > > > -Jay
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: New Producer Public API

Posted by Roger Hoover <ro...@gmail.com>.
+1 ListenableFuture: If this works similar to Deferreds in Twisted Python
or Promised IO in Javascript, I think this is a great pattern for
decoupling your callback logic from the place where the Future is
generated.  You can register as many callbacks as you like, each in the
appropriate layer of the code and have each observer get notified when the
promised i/o is complete without any of them knowing about each other.


On Tue, Jan 28, 2014 at 11:32 AM, Jay Kreps <ja...@gmail.com> wrote:

> Hey Ross,
>
> - ListenableFuture: Interesting. That would be an alternative to the direct
> callback support we provide. There could be pros to this, let me think
> about it.
> - We could provide layering, but I feel that the serialization is such a
> small thing we should just make a decision and chose one, it doesn't seem
> to me to justify a whole public facing layer.
> - Yes, this is fairly esoteric, essentially I think it is fairly similar to
> databases like DynamoDB that allow you to specify two partition keys (I
> think DynamoDB does this...). The reasoning is that in fact there are
> several things you can use the key field for: (1) to compute the partition
> to store the data in, (2) as a unique identifier to deduplicate that
> partition's records within a log. These two things are almost always the
> same, but occationally may differ when you want to group data in a more
> sophisticated way then just a hash of the primary key but still retain the
> proper primary key for delivery to the consumer and log compaction.
>
> -Jay
>
>
> On Tue, Jan 28, 2014 at 3:24 AM, Ross Black <ro...@gmail.com>
> wrote:
>
> > Hi Jay,
> >
> > - Just to add some more info/confusion about possibly using Future ...
> >   If Kafka uses a JDK future, it plays nicely with other frameworks as
> > well.
> >   Google Guava has a ListenableFuture that allows callback handling to be
> > added via the returned future, and allows the callbacks to be passed off
> to
> > a specified executor.
> >
> >
> >
> http://docs.guava-libraries.googlecode.com/git-history/release/javadoc/com/google/common/util/concurrent/ListenableFuture.html
> >   The JDK future can easily be converted to a listenable future.
> >
> > - On the question of byte[] vs Object, could this be solved by layering
> the
> > API?  eg. a raw producer (use byte[] and specify the partition number)
> and
> > a normal producer (use generic object and specify a Partitioner)?
> >
> > - I am confused by the keys in ProducerRecord and Partitioner.  What is
> the
> > usage for both a key and a partition key? (I am not yet using 0.8)
> >
> >
> > Thanks,
> > Ross
> >
> >
> >
> > On 28 January 2014 05:00, Xavier Stevens <xa...@gaikai.com> wrote:
> >
> > > AutoCloseable would be nice for us as most of our code is using Java 7
> at
> > > this point.
> > >
> > > I like Dropwizard's configuration mapping to POJOs via Jackson, but if
> > you
> > > wanted to stick with property maps I don't care enough to object.
> > >
> > > If the producer only dealt with bytes, is there a way we could still
> due
> > > partition plugins without specifying the number explicitly? I would
> > prefer
> > > to be able to pass in field(s) that would be used by the partitioner.
> > > Obviously if this wasn't possible you could always deserialize the
> object
> > > in the partitioner and grab the fields you want, but that seems really
> > > expensive to do on every message.
> > >
> > > It would also be nice to have a Java API Encoder constructor taking in
> > > VerifiableProperties. Scala understands how to handle "props:
> > > VerifiableProperties = null", but Java doesn't. So you don't run into
> > this
> > > problem until runtime.
> > >
> > >
> > > -Xavier
> > >
> > >
> > > On Mon, Jan 27, 2014 at 9:37 AM, Clark Breyman <cl...@breyman.com>
> > wrote:
> > >
> > > > Jay -
> > > >
> > > > Config - your explanation makes sense. I'm just so accustomed to
> having
> > > > Jackson automatically map my configuration objects to POJOs that I've
> > > > stopped using property files. They are lingua franca. The only
> thought
> > > > might be to separate the config interface from the implementation to
> > > allow
> > > > for alternatives, but that might undermine your point of "do it this
> > way
> > > so
> > > > that everyone can find it where they expect it".
> > > >
> > > > Serialization: Of the options, I like 1A the best, though possibly
> with
> > > > either an option to specify a partition key rather than ID or a
> helper
> > to
> > > > translate an arbitrary byte[] or long into a partition number.
> > > >
> > > > Thanks
> > > > Clark
> > > >
> > > >
> > > > On Sun, Jan 26, 2014 at 9:13 PM, Jay Kreps <ja...@gmail.com>
> > wrote:
> > > >
> > > > > Thanks for the detailed thoughts. Let me elaborate on the config
> > thing.
> > > > >
> > > > > I agree that at first glance key-value strings don't seem like a
> very
> > > > good
> > > > > configuration api for a client. Surely a well-typed config class
> > would
> > > be
> > > > > better! I actually disagree and let me see if I can convince you.
> > > > >
> > > > > My reasoning has nothing to do with the api and everything to do
> with
> > > > > operations.
> > > > >
> > > > > Clients are embedded in applications which are themselves
> configured.
> > > In
> > > > > any place that takes operations seriously the configuration for
> these
> > > > > applications will be version controlled and maintained through some
> > > kind
> > > > of
> > > > > config management system. If we give a config class with getters
> and
> > > > > setters the application has to expose those properties to its
> > > > > configuration. What invariably happens is that the application
> > exposes
> > > > only
> > > > > a choice few properties that they thought they would change.
> > > Furthermore
> > > > > the application will make up a name for these configs that seems
> > > > intuitive
> > > > > at the time in the 2 seconds the engineer spends thinking about it.
> > > > >
> > > > > Now consider the result of this in the large. You end up with
> dozens
> > or
> > > > > hundreds of applications that have the client embedded. Each
> exposes
> > a
> > > > > different, inadequate subset of the possible configs, each with
> > > different
> > > > > names. It is a nightmare.
> > > > >
> > > > > If you use a string-string map the config system can directly get a
> > > > bundle
> > > > > of config key-value pairs and put them into the client. This means
> > that
> > > > all
> > > > > configuration is automatically available with the name documented
> on
> > > the
> > > > > website in every application that does this. If you upgrade to a
> new
> > > > kafka
> > > > > version with more configs those will be exposed too. If you realize
> > > that
> > > > > you need to change a default you can just go through your configs
> and
> > > > > change it everywhere as it will have the same name everywhere.
> > > > >
> > > > > -Jay
> > > > >
> > > > >
> > > > >
> > > > >
> > > > > On Sun, Jan 26, 2014 at 4:47 PM, Clark Breyman <cl...@breyman.com>
> > > > wrote:
> > > > >
> > > > > > Thanks Jay. I'll see if I can put together a more complete
> > response,
> > > > > > perhaps as separate threads so that topics don't get entangled.
> In
> > > the
> > > > > mean
> > > > > > time, here's a couple responses:
> > > > > >
> > > > > > Serialization: you've broken out a sub-thread so i'll reply
> there.
> > My
> > > > > bias
> > > > > > is that I like generics (except for type-erasure) and in
> particular
> > > > they
> > > > > > make it easy to compose serializers for compound payloads (e.g.
> > when
> > > a
> > > > > > common header wraps a payload of parameterized type). I'll
> respond
> > to
> > > > > your
> > > > > > 4-options message with an example.
> > > > > >
> > > > > > Build: I've seen a lot of "maven-compatible" build systems
> produce
> > > > > > "artifacts" that aren't really artifacts - no embedded POM or,
> > worst,
> > > > > > malformed POM. I know the sbt-generated artifacts were this way -
> > > onus
> > > > is
> > > > > > on me to see what gradle is spitting out and what a maven build
> > might
> > > > > look
> > > > > > like. Maven may be old and boring, but it gets out of the way and
> > > > > > integrates really seamlessly with a lot of IDEs. When some scala
> > > > > projects I
> > > > > > was working on in the fall of 2011 switched from sbt to maven,
> > build
> > > > > became
> > > > > > a non-issue.
> > > > > >
> > > > > > Config: Not a big deal  and no, I don't think a dropwizard
> > dependency
> > > > is
> > > > > > appropriate. I do like using simple entity beans (POJO's not
> j2EE)
> > > for
> > > > > > configuration, especially if they can be marshalled without
> > > annotation
> > > > by
> > > > > > Jackson. I only mentioned the dropwizard-extras  because it has
> > some
> > > > > entity
> > > > > > bean versions of the ZK and Kafka configs.
> > > > > >
> > > > > > Domain-packaging: Also not a big deal - it's what's expected and
> > it's
> > > > > > pretty free in most IDE's. The advantages I see is that it is
> clear
> > > > > whether
> > > > > > something is from the Apache Kafka project and whether something
> is
> > > > from
> > > > > > another org and related to Kafka. That said, nothing really
> > enforces
> > > > it.
> > > > > >
> > > > > > Futures: I'll see if I can create some examples to demonstrate
> > Future
> > > > > > making interop easier.
> > > > > >
> > > > > > Regards,
> > > > > > C
> > > > > >
> > > > > >
> > > > > >
> > > > > >
> > > > > > On Fri, Jan 24, 2014 at 4:36 PM, Jay Kreps <ja...@gmail.com>
> > > > wrote:
> > > > > >
> > > > > > > Hey Clark,
> > > > > > >
> > > > > > > - Serialization: Yes I agree with these though I don't consider
> > the
> > > > > loss
> > > > > > of
> > > > > > > generics a big issue. I'll try to summarize what I would
> consider
> > > the
> > > > > > best
> > > > > > > alternative api with raw byte[].
> > > > > > >
> > > > > > > - Maven: We had this debate a few months back and the consensus
> > was
> > > > > > gradle.
> > > > > > > Is there a specific issue with the poms gradle makes? I am
> > > extremely
> > > > > > loath
> > > > > > > to revisit the issue as build issues are a recurring thing and
> no
> > > one
> > > > > > ever
> > > > > > > agrees and ultimately our build needs are very simple.
> > > > > > >
> > > > > > > - Config: I'm not sure if I follow the point. Are you saying we
> > > > should
> > > > > > use
> > > > > > > something in dropwizard for config? One principle here is to
> try
> > to
> > > > > > remove
> > > > > > > as many client dependencies as possible as we inevitably run
> into
> > > > > > terrible
> > > > > > > compatibility issues with users who use the same library or its
> > > > > > > dependencies at different versions. Or are you talking about
> > > > > maintaining
> > > > > > > compatibility with existing config parameters? I think as much
> > as a
> > > > > > config
> > > > > > > in the existing client makes sense it should have the same name
> > (I
> > > > was
> > > > > a
> > > > > > > bit sloppy about that so I'll fix any errors there). There are
> a
> > > few
> > > > > new
> > > > > > > things and we should give those reasonable defaults. I think
> > config
> > > > is
> > > > > > > important so I'll start a thread on the config package in
> there.
> > > > > > >
> > > > > > > - org.apache.kafka: We could do this. I always considered it
> kind
> > > of
> > > > an
> > > > > > odd
> > > > > > > thing Java programmers do that has no real motivation (but I
> > could
> > > be
> > > > > > > re-educated!). I don't think it ends up reducing naming
> conflicts
> > > in
> > > > > > > practice and it adds a lot of noise and nested directories. Is
> > > there
> > > > a
> > > > > > > reason you prefer this or just to be more standard?
> > > > > > >
> > > > > > > - Future: Basically I didn't see any particular advantage. The
> > > > cancel()
> > > > > > > method doesn't really make sense so probably wouldn't work.
> > > Likewise
> > > > I
> > > > > > > dislike the checked exceptions it requires. Basically I just
> > wrote
> > > > out
> > > > > > some
> > > > > > > code examples and it seemed cleaner with a special purpose
> > object.
> > > I
> > > > > > wasn't
> > > > > > > actually aware of plans for improved futures in java 8 or the
> > other
> > > > > > > integrations. Maybe you could elaborate on this a bit and show
> > how
> > > it
> > > > > > would
> > > > > > > be used? Sounds promising, I just don't know a lot about it.
> > > > > > >
> > > > > > > -Jay
> > > > > > >
> > > > > > >
> > > > > > > On Fri, Jan 24, 2014 at 3:30 PM, Clark Breyman <
> > clark@breyman.com>
> > > > > > wrote:
> > > > > > >
> > > > > > > > Jay - Thanks for the call for comments. Here's some initial
> > > input:
> > > > > > > >
> > > > > > > > - Make message serialization a client responsibility (making
> > all
> > > > > > messages
> > > > > > > > byte[]). Reflection-based loading makes it harder to use
> > generic
> > > > > codecs
> > > > > > > > (e.g.  Envelope<PREFIX, DATA, SUFFIX>) or build up codec
> > > > > > > programmatically.
> > > > > > > > Non-default partitioning should require an explicit partition
> > > key.
> > > > > > > >
> > > > > > > > - I really like the fact that it will be native Java. Please
> > > > consider
> > > > > > > using
> > > > > > > > native maven and not sbt, gradle, ivy, etc as they don't
> > reliably
> > > > > play
> > > > > > > nice
> > > > > > > > in the maven ecosystem. A jar without a well-formed pom
> doesn't
> > > > feel
> > > > > > > like a
> > > > > > > > real artifact. The pom's generated by sbt et al. are not well
> > > > formed.
> > > > > > > Using
> > > > > > > > maven will make builds and IDE integration much smoother.
> > > > > > > >
> > > > > > > > - Look at Nick Telford's dropwizard-extras package in which
> he
> > > > > defines
> > > > > > > some
> > > > > > > > Jackson-compatible POJO's for loading configuration. Seems
> like
> > > > your
> > > > > > > client
> > > > > > > > migration is similar. The config objects should have
> > constructors
> > > > or
> > > > > > > > factories that accept Map<String, String> and Properties for
> > ease
> > > > of
> > > > > > > > migration.
> > > > > > > >
> > > > > > > > - Would you consider using the org.apache.kafka package for
> the
> > > new
> > > > > API
> > > > > > > > (quibble)
> > > > > > > >
> > > > > > > > - Why create your own futures rather than use
> > > > > > > > java.util.concurrent.Future<Long> or similar? Standard
> futures
> > > will
> > > > > > play
> > > > > > > > nice with other reactive libs and things like J8's
> > > > ComposableFuture.
> > > > > > > >
> > > > > > > > Thanks again,
> > > > > > > > C
> > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > > On Fri, Jan 24, 2014 at 2:46 PM, Roger Hoover <
> > > > > roger.hoover@gmail.com
> > > > > > > > >wrote:
> > > > > > > >
> > > > > > > > > A couple comments:
> > > > > > > > >
> > > > > > > > > 1) Why does the config use a broker list instead of
> > discovering
> > > > the
> > > > > > > > brokers
> > > > > > > > > in ZooKeeper?  It doesn't match the HighLevelConsumer API.
> > > > > > > > >
> > > > > > > > > 2) It looks like broker connections are created on demand.
> >  I'm
> > > > > > > wondering
> > > > > > > > > if sometimes you might want to flush out config or network
> > > > > > connectivity
> > > > > > > > > issues before pushing the first message through.
> > > > > > > > >
> > > > > > > > > Should there also be a KafkaProducer.connect() or .open()
> > > method
> > > > or
> > > > > > > > > connectAll()?  I guess it would try to connect to all
> brokers
> > > in
> > > > > the
> > > > > > > > > BROKER_LIST_CONFIG
> > > > > > > > >
> > > > > > > > > HTH,
> > > > > > > > >
> > > > > > > > > Roger
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > On Fri, Jan 24, 2014 at 11:54 AM, Jay Kreps <
> > > jay.kreps@gmail.com
> > > > >
> > > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > As mentioned in a previous email we are working on a
> > > > > > > re-implementation
> > > > > > > > of
> > > > > > > > > > the producer. I would like to use this email thread to
> > > discuss
> > > > > the
> > > > > > > > > details
> > > > > > > > > > of the public API and the configuration. I would love for
> > us
> > > to
> > > > > be
> > > > > > > > > > incredibly picky about this public api now so it is as
> good
> > > as
> > > > > > > possible
> > > > > > > > > and
> > > > > > > > > > we don't need to break it in the future.
> > > > > > > > > >
> > > > > > > > > > The best way to get a feel for the API is actually to
> take
> > a
> > > > look
> > > > > > at
> > > > > > > > the
> > > > > > > > > > javadoc, my hope is to get the api docs good enough so
> that
> > > it
> > > > is
> > > > > > > > > > self-explanatory:
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> http://empathybox.com/kafka-javadoc/index.html?kafka/clients/producer/KafkaProducer.html
> > > > > > > > > >
> > > > > > > > > > Please take a look at this API and give me any thoughts
> you
> > > may
> > > > > > have!
> > > > > > > > > >
> > > > > > > > > > It may also be reasonable to take a look at the configs:
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> http://empathybox.com/kafka-javadoc/kafka/clients/producer/ProducerConfig.html
> > > > > > > > > >
> > > > > > > > > > The actual code is posted here:
> > > > > > > > > > https://issues.apache.org/jira/browse/KAFKA-1227
> > > > > > > > > >
> > > > > > > > > > A few questions or comments to kick things off:
> > > > > > > > > > 1. We need to make a decision on whether serialization of
> > the
> > > > > > user's
> > > > > > > > key
> > > > > > > > > > and value should be done by the user (with our api just
> > > taking
> > > > > > > byte[])
> > > > > > > > or
> > > > > > > > > > if we should take an object and allow the user to
> > configure a
> > > > > > > > Serializer
> > > > > > > > > > class which we instantiate via reflection. We take the
> > later
> > > > > > approach
> > > > > > > > in
> > > > > > > > > > the current producer, and I have carried this through to
> > this
> > > > > > > > prototype.
> > > > > > > > > > The tradeoff I see is this: taking byte[] is actually
> > > simpler,
> > > > > the
> > > > > > > user
> > > > > > > > > can
> > > > > > > > > > directly do whatever serialization they like. The
> > > complication
> > > > is
> > > > > > > > > actually
> > > > > > > > > > partitioning. Currently partitioning is done by a similar
> > > > plug-in
> > > > > > api
> > > > > > > > > > (Partitioner) which the user can implement and configure
> to
> > > > > > override
> > > > > > > > how
> > > > > > > > > > partitions are assigned. If we take byte[] as input then
> we
> > > > have
> > > > > no
> > > > > > > > > access
> > > > > > > > > > to the original object and partitioning MUST be done on
> the
> > > > > byte[].
> > > > > > > > This
> > > > > > > > > is
> > > > > > > > > > fine for hash partitioning. However for various types of
> > > > semantic
> > > > > > > > > > partitioning (range partitioning, or whatever) you would
> > want
> > > > > > access
> > > > > > > to
> > > > > > > > > the
> > > > > > > > > > original object. In the current approach a producer who
> > > wishes
> > > > to
> > > > > > > send
> > > > > > > > > > byte[] they have serialized in their own code can
> configure
> > > the
> > > > > > > > > > BytesSerialization we supply which is just a "no op"
> > > > > serialization.
> > > > > > > > > > 2. We should obsess over naming and make sure each of the
> > > class
> > > > > > names
> > > > > > > > are
> > > > > > > > > > good.
> > > > > > > > > > 3. Jun has already pointed out that we need to include
> the
> > > > topic
> > > > > > and
> > > > > > > > > > partition in the response, which is absolutely right. I
> > > haven't
> > > > > > done
> > > > > > > > that
> > > > > > > > > > yet but that definitely needs to be there.
> > > > > > > > > > 4. Currently RecordSend.await will throw an exception if
> > the
> > > > > > request
> > > > > > > > > > failed. The intention here is that
> > > > producer.send(message).await()
> > > > > > > > exactly
> > > > > > > > > > simulates a synchronous call. Guozhang has noted that
> this
> > > is a
> > > > > > > little
> > > > > > > > > > annoying since the user must then catch exceptions.
> However
> > > if
> > > > we
> > > > > > > > remove
> > > > > > > > > > this then if the user doesn't check for errors they won't
> > > know
> > > > > one
> > > > > > > has
> > > > > > > > > > occurred, which I predict will be a common mistake.
> > > > > > > > > > 5. Perhaps there is more we could do to make the async
> > > > callbacks
> > > > > > and
> > > > > > > > > future
> > > > > > > > > > we give back intuitive and easy to program against?
> > > > > > > > > >
> > > > > > > > > > Some background info on implementation:
> > > > > > > > > >
> > > > > > > > > > At a high level the primary difference in this producer
> is
> > > that
> > > > > it
> > > > > > > > > removes
> > > > > > > > > > the distinction between the "sync" and "async" producer.
> > > > > > Effectively
> > > > > > > > all
> > > > > > > > > > requests are sent asynchronously but always return a
> future
> > > > > > response
> > > > > > > > > object
> > > > > > > > > > that gives the offset as well as any error that may have
> > > > occurred
> > > > > > > when
> > > > > > > > > the
> > > > > > > > > > request is complete. The batching that is done in the
> async
> > > > > > producer
> > > > > > > > only
> > > > > > > > > > today is done whenever possible now. This means that the
> > sync
> > > > > > > producer,
> > > > > > > > > > under load, can get performance as good as the async
> > producer
> > > > > > > > > (preliminary
> > > > > > > > > > results show the producer getting 1m messages/sec). This
> > > works
> > > > > > > similar
> > > > > > > > to
> > > > > > > > > > group commit in databases but with respect to the actual
> > > > network
> > > > > > > > > > transmission--any messages that arrive while a send is in
> > > > > progress
> > > > > > > are
> > > > > > > > > > batched together. It is also possible to encourage
> batching
> > > > even
> > > > > > > under
> > > > > > > > > low
> > > > > > > > > > load to save server resources by introducing a delay on
> the
> > > > send
> > > > > to
> > > > > > > > allow
> > > > > > > > > > more messages to accumulate; this is done using the
> > > > > linger.msconfig
> > > > > > > > > (this
> > > > > > > > > > is similar to Nagle's algorithm in TCP).
> > > > > > > > > >
> > > > > > > > > > This producer does all network communication
> asynchronously
> > > and
> > > > > in
> > > > > > > > > parallel
> > > > > > > > > > to all servers so the performance penalty for acks=-1 and
> > > > waiting
> > > > > > on
> > > > > > > > > > replication should be much reduced. I haven't done much
> > > > > > benchmarking
> > > > > > > on
> > > > > > > > > > this yet, though.
> > > > > > > > > >
> > > > > > > > > > The high level design is described a little here, though
> > this
> > > > is
> > > > > > now
> > > > > > > a
> > > > > > > > > > little out of date:
> > > > > > > > > >
> > > > https://cwiki.apache.org/confluence/display/KAFKA/Client+Rewrite
> > > > > > > > > >
> > > > > > > > > > -Jay
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: New Producer Public API

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

- ListenableFuture: Interesting. That would be an alternative to the direct
callback support we provide. There could be pros to this, let me think
about it.
- We could provide layering, but I feel that the serialization is such a
small thing we should just make a decision and chose one, it doesn't seem
to me to justify a whole public facing layer.
- Yes, this is fairly esoteric, essentially I think it is fairly similar to
databases like DynamoDB that allow you to specify two partition keys (I
think DynamoDB does this...). The reasoning is that in fact there are
several things you can use the key field for: (1) to compute the partition
to store the data in, (2) as a unique identifier to deduplicate that
partition's records within a log. These two things are almost always the
same, but occationally may differ when you want to group data in a more
sophisticated way then just a hash of the primary key but still retain the
proper primary key for delivery to the consumer and log compaction.

-Jay


On Tue, Jan 28, 2014 at 3:24 AM, Ross Black <ro...@gmail.com> wrote:

> Hi Jay,
>
> - Just to add some more info/confusion about possibly using Future ...
>   If Kafka uses a JDK future, it plays nicely with other frameworks as
> well.
>   Google Guava has a ListenableFuture that allows callback handling to be
> added via the returned future, and allows the callbacks to be passed off to
> a specified executor.
>
>
> http://docs.guava-libraries.googlecode.com/git-history/release/javadoc/com/google/common/util/concurrent/ListenableFuture.html
>   The JDK future can easily be converted to a listenable future.
>
> - On the question of byte[] vs Object, could this be solved by layering the
> API?  eg. a raw producer (use byte[] and specify the partition number) and
> a normal producer (use generic object and specify a Partitioner)?
>
> - I am confused by the keys in ProducerRecord and Partitioner.  What is the
> usage for both a key and a partition key? (I am not yet using 0.8)
>
>
> Thanks,
> Ross
>
>
>
> On 28 January 2014 05:00, Xavier Stevens <xa...@gaikai.com> wrote:
>
> > AutoCloseable would be nice for us as most of our code is using Java 7 at
> > this point.
> >
> > I like Dropwizard's configuration mapping to POJOs via Jackson, but if
> you
> > wanted to stick with property maps I don't care enough to object.
> >
> > If the producer only dealt with bytes, is there a way we could still due
> > partition plugins without specifying the number explicitly? I would
> prefer
> > to be able to pass in field(s) that would be used by the partitioner.
> > Obviously if this wasn't possible you could always deserialize the object
> > in the partitioner and grab the fields you want, but that seems really
> > expensive to do on every message.
> >
> > It would also be nice to have a Java API Encoder constructor taking in
> > VerifiableProperties. Scala understands how to handle "props:
> > VerifiableProperties = null", but Java doesn't. So you don't run into
> this
> > problem until runtime.
> >
> >
> > -Xavier
> >
> >
> > On Mon, Jan 27, 2014 at 9:37 AM, Clark Breyman <cl...@breyman.com>
> wrote:
> >
> > > Jay -
> > >
> > > Config - your explanation makes sense. I'm just so accustomed to having
> > > Jackson automatically map my configuration objects to POJOs that I've
> > > stopped using property files. They are lingua franca. The only thought
> > > might be to separate the config interface from the implementation to
> > allow
> > > for alternatives, but that might undermine your point of "do it this
> way
> > so
> > > that everyone can find it where they expect it".
> > >
> > > Serialization: Of the options, I like 1A the best, though possibly with
> > > either an option to specify a partition key rather than ID or a helper
> to
> > > translate an arbitrary byte[] or long into a partition number.
> > >
> > > Thanks
> > > Clark
> > >
> > >
> > > On Sun, Jan 26, 2014 at 9:13 PM, Jay Kreps <ja...@gmail.com>
> wrote:
> > >
> > > > Thanks for the detailed thoughts. Let me elaborate on the config
> thing.
> > > >
> > > > I agree that at first glance key-value strings don't seem like a very
> > > good
> > > > configuration api for a client. Surely a well-typed config class
> would
> > be
> > > > better! I actually disagree and let me see if I can convince you.
> > > >
> > > > My reasoning has nothing to do with the api and everything to do with
> > > > operations.
> > > >
> > > > Clients are embedded in applications which are themselves configured.
> > In
> > > > any place that takes operations seriously the configuration for these
> > > > applications will be version controlled and maintained through some
> > kind
> > > of
> > > > config management system. If we give a config class with getters and
> > > > setters the application has to expose those properties to its
> > > > configuration. What invariably happens is that the application
> exposes
> > > only
> > > > a choice few properties that they thought they would change.
> > Furthermore
> > > > the application will make up a name for these configs that seems
> > > intuitive
> > > > at the time in the 2 seconds the engineer spends thinking about it.
> > > >
> > > > Now consider the result of this in the large. You end up with dozens
> or
> > > > hundreds of applications that have the client embedded. Each exposes
> a
> > > > different, inadequate subset of the possible configs, each with
> > different
> > > > names. It is a nightmare.
> > > >
> > > > If you use a string-string map the config system can directly get a
> > > bundle
> > > > of config key-value pairs and put them into the client. This means
> that
> > > all
> > > > configuration is automatically available with the name documented on
> > the
> > > > website in every application that does this. If you upgrade to a new
> > > kafka
> > > > version with more configs those will be exposed too. If you realize
> > that
> > > > you need to change a default you can just go through your configs and
> > > > change it everywhere as it will have the same name everywhere.
> > > >
> > > > -Jay
> > > >
> > > >
> > > >
> > > >
> > > > On Sun, Jan 26, 2014 at 4:47 PM, Clark Breyman <cl...@breyman.com>
> > > wrote:
> > > >
> > > > > Thanks Jay. I'll see if I can put together a more complete
> response,
> > > > > perhaps as separate threads so that topics don't get entangled. In
> > the
> > > > mean
> > > > > time, here's a couple responses:
> > > > >
> > > > > Serialization: you've broken out a sub-thread so i'll reply there.
> My
> > > > bias
> > > > > is that I like generics (except for type-erasure) and in particular
> > > they
> > > > > make it easy to compose serializers for compound payloads (e.g.
> when
> > a
> > > > > common header wraps a payload of parameterized type). I'll respond
> to
> > > > your
> > > > > 4-options message with an example.
> > > > >
> > > > > Build: I've seen a lot of "maven-compatible" build systems produce
> > > > > "artifacts" that aren't really artifacts - no embedded POM or,
> worst,
> > > > > malformed POM. I know the sbt-generated artifacts were this way -
> > onus
> > > is
> > > > > on me to see what gradle is spitting out and what a maven build
> might
> > > > look
> > > > > like. Maven may be old and boring, but it gets out of the way and
> > > > > integrates really seamlessly with a lot of IDEs. When some scala
> > > > projects I
> > > > > was working on in the fall of 2011 switched from sbt to maven,
> build
> > > > became
> > > > > a non-issue.
> > > > >
> > > > > Config: Not a big deal  and no, I don't think a dropwizard
> dependency
> > > is
> > > > > appropriate. I do like using simple entity beans (POJO's not j2EE)
> > for
> > > > > configuration, especially if they can be marshalled without
> > annotation
> > > by
> > > > > Jackson. I only mentioned the dropwizard-extras  because it has
> some
> > > > entity
> > > > > bean versions of the ZK and Kafka configs.
> > > > >
> > > > > Domain-packaging: Also not a big deal - it's what's expected and
> it's
> > > > > pretty free in most IDE's. The advantages I see is that it is clear
> > > > whether
> > > > > something is from the Apache Kafka project and whether something is
> > > from
> > > > > another org and related to Kafka. That said, nothing really
> enforces
> > > it.
> > > > >
> > > > > Futures: I'll see if I can create some examples to demonstrate
> Future
> > > > > making interop easier.
> > > > >
> > > > > Regards,
> > > > > C
> > > > >
> > > > >
> > > > >
> > > > >
> > > > > On Fri, Jan 24, 2014 at 4:36 PM, Jay Kreps <ja...@gmail.com>
> > > wrote:
> > > > >
> > > > > > Hey Clark,
> > > > > >
> > > > > > - Serialization: Yes I agree with these though I don't consider
> the
> > > > loss
> > > > > of
> > > > > > generics a big issue. I'll try to summarize what I would consider
> > the
> > > > > best
> > > > > > alternative api with raw byte[].
> > > > > >
> > > > > > - Maven: We had this debate a few months back and the consensus
> was
> > > > > gradle.
> > > > > > Is there a specific issue with the poms gradle makes? I am
> > extremely
> > > > > loath
> > > > > > to revisit the issue as build issues are a recurring thing and no
> > one
> > > > > ever
> > > > > > agrees and ultimately our build needs are very simple.
> > > > > >
> > > > > > - Config: I'm not sure if I follow the point. Are you saying we
> > > should
> > > > > use
> > > > > > something in dropwizard for config? One principle here is to try
> to
> > > > > remove
> > > > > > as many client dependencies as possible as we inevitably run into
> > > > > terrible
> > > > > > compatibility issues with users who use the same library or its
> > > > > > dependencies at different versions. Or are you talking about
> > > > maintaining
> > > > > > compatibility with existing config parameters? I think as much
> as a
> > > > > config
> > > > > > in the existing client makes sense it should have the same name
> (I
> > > was
> > > > a
> > > > > > bit sloppy about that so I'll fix any errors there). There are a
> > few
> > > > new
> > > > > > things and we should give those reasonable defaults. I think
> config
> > > is
> > > > > > important so I'll start a thread on the config package in there.
> > > > > >
> > > > > > - org.apache.kafka: We could do this. I always considered it kind
> > of
> > > an
> > > > > odd
> > > > > > thing Java programmers do that has no real motivation (but I
> could
> > be
> > > > > > re-educated!). I don't think it ends up reducing naming conflicts
> > in
> > > > > > practice and it adds a lot of noise and nested directories. Is
> > there
> > > a
> > > > > > reason you prefer this or just to be more standard?
> > > > > >
> > > > > > - Future: Basically I didn't see any particular advantage. The
> > > cancel()
> > > > > > method doesn't really make sense so probably wouldn't work.
> > Likewise
> > > I
> > > > > > dislike the checked exceptions it requires. Basically I just
> wrote
> > > out
> > > > > some
> > > > > > code examples and it seemed cleaner with a special purpose
> object.
> > I
> > > > > wasn't
> > > > > > actually aware of plans for improved futures in java 8 or the
> other
> > > > > > integrations. Maybe you could elaborate on this a bit and show
> how
> > it
> > > > > would
> > > > > > be used? Sounds promising, I just don't know a lot about it.
> > > > > >
> > > > > > -Jay
> > > > > >
> > > > > >
> > > > > > On Fri, Jan 24, 2014 at 3:30 PM, Clark Breyman <
> clark@breyman.com>
> > > > > wrote:
> > > > > >
> > > > > > > Jay - Thanks for the call for comments. Here's some initial
> > input:
> > > > > > >
> > > > > > > - Make message serialization a client responsibility (making
> all
> > > > > messages
> > > > > > > byte[]). Reflection-based loading makes it harder to use
> generic
> > > > codecs
> > > > > > > (e.g.  Envelope<PREFIX, DATA, SUFFIX>) or build up codec
> > > > > > programmatically.
> > > > > > > Non-default partitioning should require an explicit partition
> > key.
> > > > > > >
> > > > > > > - I really like the fact that it will be native Java. Please
> > > consider
> > > > > > using
> > > > > > > native maven and not sbt, gradle, ivy, etc as they don't
> reliably
> > > > play
> > > > > > nice
> > > > > > > in the maven ecosystem. A jar without a well-formed pom doesn't
> > > feel
> > > > > > like a
> > > > > > > real artifact. The pom's generated by sbt et al. are not well
> > > formed.
> > > > > > Using
> > > > > > > maven will make builds and IDE integration much smoother.
> > > > > > >
> > > > > > > - Look at Nick Telford's dropwizard-extras package in which he
> > > > defines
> > > > > > some
> > > > > > > Jackson-compatible POJO's for loading configuration. Seems like
> > > your
> > > > > > client
> > > > > > > migration is similar. The config objects should have
> constructors
> > > or
> > > > > > > factories that accept Map<String, String> and Properties for
> ease
> > > of
> > > > > > > migration.
> > > > > > >
> > > > > > > - Would you consider using the org.apache.kafka package for the
> > new
> > > > API
> > > > > > > (quibble)
> > > > > > >
> > > > > > > - Why create your own futures rather than use
> > > > > > > java.util.concurrent.Future<Long> or similar? Standard futures
> > will
> > > > > play
> > > > > > > nice with other reactive libs and things like J8's
> > > ComposableFuture.
> > > > > > >
> > > > > > > Thanks again,
> > > > > > > C
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > On Fri, Jan 24, 2014 at 2:46 PM, Roger Hoover <
> > > > roger.hoover@gmail.com
> > > > > > > >wrote:
> > > > > > >
> > > > > > > > A couple comments:
> > > > > > > >
> > > > > > > > 1) Why does the config use a broker list instead of
> discovering
> > > the
> > > > > > > brokers
> > > > > > > > in ZooKeeper?  It doesn't match the HighLevelConsumer API.
> > > > > > > >
> > > > > > > > 2) It looks like broker connections are created on demand.
>  I'm
> > > > > > wondering
> > > > > > > > if sometimes you might want to flush out config or network
> > > > > connectivity
> > > > > > > > issues before pushing the first message through.
> > > > > > > >
> > > > > > > > Should there also be a KafkaProducer.connect() or .open()
> > method
> > > or
> > > > > > > > connectAll()?  I guess it would try to connect to all brokers
> > in
> > > > the
> > > > > > > > BROKER_LIST_CONFIG
> > > > > > > >
> > > > > > > > HTH,
> > > > > > > >
> > > > > > > > Roger
> > > > > > > >
> > > > > > > >
> > > > > > > > On Fri, Jan 24, 2014 at 11:54 AM, Jay Kreps <
> > jay.kreps@gmail.com
> > > >
> > > > > > wrote:
> > > > > > > >
> > > > > > > > > As mentioned in a previous email we are working on a
> > > > > > re-implementation
> > > > > > > of
> > > > > > > > > the producer. I would like to use this email thread to
> > discuss
> > > > the
> > > > > > > > details
> > > > > > > > > of the public API and the configuration. I would love for
> us
> > to
> > > > be
> > > > > > > > > incredibly picky about this public api now so it is as good
> > as
> > > > > > possible
> > > > > > > > and
> > > > > > > > > we don't need to break it in the future.
> > > > > > > > >
> > > > > > > > > The best way to get a feel for the API is actually to take
> a
> > > look
> > > > > at
> > > > > > > the
> > > > > > > > > javadoc, my hope is to get the api docs good enough so that
> > it
> > > is
> > > > > > > > > self-explanatory:
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> http://empathybox.com/kafka-javadoc/index.html?kafka/clients/producer/KafkaProducer.html
> > > > > > > > >
> > > > > > > > > Please take a look at this API and give me any thoughts you
> > may
> > > > > have!
> > > > > > > > >
> > > > > > > > > It may also be reasonable to take a look at the configs:
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> http://empathybox.com/kafka-javadoc/kafka/clients/producer/ProducerConfig.html
> > > > > > > > >
> > > > > > > > > The actual code is posted here:
> > > > > > > > > https://issues.apache.org/jira/browse/KAFKA-1227
> > > > > > > > >
> > > > > > > > > A few questions or comments to kick things off:
> > > > > > > > > 1. We need to make a decision on whether serialization of
> the
> > > > > user's
> > > > > > > key
> > > > > > > > > and value should be done by the user (with our api just
> > taking
> > > > > > byte[])
> > > > > > > or
> > > > > > > > > if we should take an object and allow the user to
> configure a
> > > > > > > Serializer
> > > > > > > > > class which we instantiate via reflection. We take the
> later
> > > > > approach
> > > > > > > in
> > > > > > > > > the current producer, and I have carried this through to
> this
> > > > > > > prototype.
> > > > > > > > > The tradeoff I see is this: taking byte[] is actually
> > simpler,
> > > > the
> > > > > > user
> > > > > > > > can
> > > > > > > > > directly do whatever serialization they like. The
> > complication
> > > is
> > > > > > > > actually
> > > > > > > > > partitioning. Currently partitioning is done by a similar
> > > plug-in
> > > > > api
> > > > > > > > > (Partitioner) which the user can implement and configure to
> > > > > override
> > > > > > > how
> > > > > > > > > partitions are assigned. If we take byte[] as input then we
> > > have
> > > > no
> > > > > > > > access
> > > > > > > > > to the original object and partitioning MUST be done on the
> > > > byte[].
> > > > > > > This
> > > > > > > > is
> > > > > > > > > fine for hash partitioning. However for various types of
> > > semantic
> > > > > > > > > partitioning (range partitioning, or whatever) you would
> want
> > > > > access
> > > > > > to
> > > > > > > > the
> > > > > > > > > original object. In the current approach a producer who
> > wishes
> > > to
> > > > > > send
> > > > > > > > > byte[] they have serialized in their own code can configure
> > the
> > > > > > > > > BytesSerialization we supply which is just a "no op"
> > > > serialization.
> > > > > > > > > 2. We should obsess over naming and make sure each of the
> > class
> > > > > names
> > > > > > > are
> > > > > > > > > good.
> > > > > > > > > 3. Jun has already pointed out that we need to include the
> > > topic
> > > > > and
> > > > > > > > > partition in the response, which is absolutely right. I
> > haven't
> > > > > done
> > > > > > > that
> > > > > > > > > yet but that definitely needs to be there.
> > > > > > > > > 4. Currently RecordSend.await will throw an exception if
> the
> > > > > request
> > > > > > > > > failed. The intention here is that
> > > producer.send(message).await()
> > > > > > > exactly
> > > > > > > > > simulates a synchronous call. Guozhang has noted that this
> > is a
> > > > > > little
> > > > > > > > > annoying since the user must then catch exceptions. However
> > if
> > > we
> > > > > > > remove
> > > > > > > > > this then if the user doesn't check for errors they won't
> > know
> > > > one
> > > > > > has
> > > > > > > > > occurred, which I predict will be a common mistake.
> > > > > > > > > 5. Perhaps there is more we could do to make the async
> > > callbacks
> > > > > and
> > > > > > > > future
> > > > > > > > > we give back intuitive and easy to program against?
> > > > > > > > >
> > > > > > > > > Some background info on implementation:
> > > > > > > > >
> > > > > > > > > At a high level the primary difference in this producer is
> > that
> > > > it
> > > > > > > > removes
> > > > > > > > > the distinction between the "sync" and "async" producer.
> > > > > Effectively
> > > > > > > all
> > > > > > > > > requests are sent asynchronously but always return a future
> > > > > response
> > > > > > > > object
> > > > > > > > > that gives the offset as well as any error that may have
> > > occurred
> > > > > > when
> > > > > > > > the
> > > > > > > > > request is complete. The batching that is done in the async
> > > > > producer
> > > > > > > only
> > > > > > > > > today is done whenever possible now. This means that the
> sync
> > > > > > producer,
> > > > > > > > > under load, can get performance as good as the async
> producer
> > > > > > > > (preliminary
> > > > > > > > > results show the producer getting 1m messages/sec). This
> > works
> > > > > > similar
> > > > > > > to
> > > > > > > > > group commit in databases but with respect to the actual
> > > network
> > > > > > > > > transmission--any messages that arrive while a send is in
> > > > progress
> > > > > > are
> > > > > > > > > batched together. It is also possible to encourage batching
> > > even
> > > > > > under
> > > > > > > > low
> > > > > > > > > load to save server resources by introducing a delay on the
> > > send
> > > > to
> > > > > > > allow
> > > > > > > > > more messages to accumulate; this is done using the
> > > > linger.msconfig
> > > > > > > > (this
> > > > > > > > > is similar to Nagle's algorithm in TCP).
> > > > > > > > >
> > > > > > > > > This producer does all network communication asynchronously
> > and
> > > > in
> > > > > > > > parallel
> > > > > > > > > to all servers so the performance penalty for acks=-1 and
> > > waiting
> > > > > on
> > > > > > > > > replication should be much reduced. I haven't done much
> > > > > benchmarking
> > > > > > on
> > > > > > > > > this yet, though.
> > > > > > > > >
> > > > > > > > > The high level design is described a little here, though
> this
> > > is
> > > > > now
> > > > > > a
> > > > > > > > > little out of date:
> > > > > > > > >
> > > https://cwiki.apache.org/confluence/display/KAFKA/Client+Rewrite
> > > > > > > > >
> > > > > > > > > -Jay
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: New Producer Public API

Posted by Ross Black <ro...@gmail.com>.
Hi Jay,

- Just to add some more info/confusion about possibly using Future ...
  If Kafka uses a JDK future, it plays nicely with other frameworks as well.
  Google Guava has a ListenableFuture that allows callback handling to be
added via the returned future, and allows the callbacks to be passed off to
a specified executor.

http://docs.guava-libraries.googlecode.com/git-history/release/javadoc/com/google/common/util/concurrent/ListenableFuture.html
  The JDK future can easily be converted to a listenable future.

- On the question of byte[] vs Object, could this be solved by layering the
API?  eg. a raw producer (use byte[] and specify the partition number) and
a normal producer (use generic object and specify a Partitioner)?

- I am confused by the keys in ProducerRecord and Partitioner.  What is the
usage for both a key and a partition key? (I am not yet using 0.8)


Thanks,
Ross



On 28 January 2014 05:00, Xavier Stevens <xa...@gaikai.com> wrote:

> AutoCloseable would be nice for us as most of our code is using Java 7 at
> this point.
>
> I like Dropwizard's configuration mapping to POJOs via Jackson, but if you
> wanted to stick with property maps I don't care enough to object.
>
> If the producer only dealt with bytes, is there a way we could still due
> partition plugins without specifying the number explicitly? I would prefer
> to be able to pass in field(s) that would be used by the partitioner.
> Obviously if this wasn't possible you could always deserialize the object
> in the partitioner and grab the fields you want, but that seems really
> expensive to do on every message.
>
> It would also be nice to have a Java API Encoder constructor taking in
> VerifiableProperties. Scala understands how to handle "props:
> VerifiableProperties = null", but Java doesn't. So you don't run into this
> problem until runtime.
>
>
> -Xavier
>
>
> On Mon, Jan 27, 2014 at 9:37 AM, Clark Breyman <cl...@breyman.com> wrote:
>
> > Jay -
> >
> > Config - your explanation makes sense. I'm just so accustomed to having
> > Jackson automatically map my configuration objects to POJOs that I've
> > stopped using property files. They are lingua franca. The only thought
> > might be to separate the config interface from the implementation to
> allow
> > for alternatives, but that might undermine your point of "do it this way
> so
> > that everyone can find it where they expect it".
> >
> > Serialization: Of the options, I like 1A the best, though possibly with
> > either an option to specify a partition key rather than ID or a helper to
> > translate an arbitrary byte[] or long into a partition number.
> >
> > Thanks
> > Clark
> >
> >
> > On Sun, Jan 26, 2014 at 9:13 PM, Jay Kreps <ja...@gmail.com> wrote:
> >
> > > Thanks for the detailed thoughts. Let me elaborate on the config thing.
> > >
> > > I agree that at first glance key-value strings don't seem like a very
> > good
> > > configuration api for a client. Surely a well-typed config class would
> be
> > > better! I actually disagree and let me see if I can convince you.
> > >
> > > My reasoning has nothing to do with the api and everything to do with
> > > operations.
> > >
> > > Clients are embedded in applications which are themselves configured.
> In
> > > any place that takes operations seriously the configuration for these
> > > applications will be version controlled and maintained through some
> kind
> > of
> > > config management system. If we give a config class with getters and
> > > setters the application has to expose those properties to its
> > > configuration. What invariably happens is that the application exposes
> > only
> > > a choice few properties that they thought they would change.
> Furthermore
> > > the application will make up a name for these configs that seems
> > intuitive
> > > at the time in the 2 seconds the engineer spends thinking about it.
> > >
> > > Now consider the result of this in the large. You end up with dozens or
> > > hundreds of applications that have the client embedded. Each exposes a
> > > different, inadequate subset of the possible configs, each with
> different
> > > names. It is a nightmare.
> > >
> > > If you use a string-string map the config system can directly get a
> > bundle
> > > of config key-value pairs and put them into the client. This means that
> > all
> > > configuration is automatically available with the name documented on
> the
> > > website in every application that does this. If you upgrade to a new
> > kafka
> > > version with more configs those will be exposed too. If you realize
> that
> > > you need to change a default you can just go through your configs and
> > > change it everywhere as it will have the same name everywhere.
> > >
> > > -Jay
> > >
> > >
> > >
> > >
> > > On Sun, Jan 26, 2014 at 4:47 PM, Clark Breyman <cl...@breyman.com>
> > wrote:
> > >
> > > > Thanks Jay. I'll see if I can put together a more complete response,
> > > > perhaps as separate threads so that topics don't get entangled. In
> the
> > > mean
> > > > time, here's a couple responses:
> > > >
> > > > Serialization: you've broken out a sub-thread so i'll reply there. My
> > > bias
> > > > is that I like generics (except for type-erasure) and in particular
> > they
> > > > make it easy to compose serializers for compound payloads (e.g. when
> a
> > > > common header wraps a payload of parameterized type). I'll respond to
> > > your
> > > > 4-options message with an example.
> > > >
> > > > Build: I've seen a lot of "maven-compatible" build systems produce
> > > > "artifacts" that aren't really artifacts - no embedded POM or, worst,
> > > > malformed POM. I know the sbt-generated artifacts were this way -
> onus
> > is
> > > > on me to see what gradle is spitting out and what a maven build might
> > > look
> > > > like. Maven may be old and boring, but it gets out of the way and
> > > > integrates really seamlessly with a lot of IDEs. When some scala
> > > projects I
> > > > was working on in the fall of 2011 switched from sbt to maven, build
> > > became
> > > > a non-issue.
> > > >
> > > > Config: Not a big deal  and no, I don't think a dropwizard dependency
> > is
> > > > appropriate. I do like using simple entity beans (POJO's not j2EE)
> for
> > > > configuration, especially if they can be marshalled without
> annotation
> > by
> > > > Jackson. I only mentioned the dropwizard-extras  because it has some
> > > entity
> > > > bean versions of the ZK and Kafka configs.
> > > >
> > > > Domain-packaging: Also not a big deal - it's what's expected and it's
> > > > pretty free in most IDE's. The advantages I see is that it is clear
> > > whether
> > > > something is from the Apache Kafka project and whether something is
> > from
> > > > another org and related to Kafka. That said, nothing really enforces
> > it.
> > > >
> > > > Futures: I'll see if I can create some examples to demonstrate Future
> > > > making interop easier.
> > > >
> > > > Regards,
> > > > C
> > > >
> > > >
> > > >
> > > >
> > > > On Fri, Jan 24, 2014 at 4:36 PM, Jay Kreps <ja...@gmail.com>
> > wrote:
> > > >
> > > > > Hey Clark,
> > > > >
> > > > > - Serialization: Yes I agree with these though I don't consider the
> > > loss
> > > > of
> > > > > generics a big issue. I'll try to summarize what I would consider
> the
> > > > best
> > > > > alternative api with raw byte[].
> > > > >
> > > > > - Maven: We had this debate a few months back and the consensus was
> > > > gradle.
> > > > > Is there a specific issue with the poms gradle makes? I am
> extremely
> > > > loath
> > > > > to revisit the issue as build issues are a recurring thing and no
> one
> > > > ever
> > > > > agrees and ultimately our build needs are very simple.
> > > > >
> > > > > - Config: I'm not sure if I follow the point. Are you saying we
> > should
> > > > use
> > > > > something in dropwizard for config? One principle here is to try to
> > > > remove
> > > > > as many client dependencies as possible as we inevitably run into
> > > > terrible
> > > > > compatibility issues with users who use the same library or its
> > > > > dependencies at different versions. Or are you talking about
> > > maintaining
> > > > > compatibility with existing config parameters? I think as much as a
> > > > config
> > > > > in the existing client makes sense it should have the same name (I
> > was
> > > a
> > > > > bit sloppy about that so I'll fix any errors there). There are a
> few
> > > new
> > > > > things and we should give those reasonable defaults. I think config
> > is
> > > > > important so I'll start a thread on the config package in there.
> > > > >
> > > > > - org.apache.kafka: We could do this. I always considered it kind
> of
> > an
> > > > odd
> > > > > thing Java programmers do that has no real motivation (but I could
> be
> > > > > re-educated!). I don't think it ends up reducing naming conflicts
> in
> > > > > practice and it adds a lot of noise and nested directories. Is
> there
> > a
> > > > > reason you prefer this or just to be more standard?
> > > > >
> > > > > - Future: Basically I didn't see any particular advantage. The
> > cancel()
> > > > > method doesn't really make sense so probably wouldn't work.
> Likewise
> > I
> > > > > dislike the checked exceptions it requires. Basically I just wrote
> > out
> > > > some
> > > > > code examples and it seemed cleaner with a special purpose object.
> I
> > > > wasn't
> > > > > actually aware of plans for improved futures in java 8 or the other
> > > > > integrations. Maybe you could elaborate on this a bit and show how
> it
> > > > would
> > > > > be used? Sounds promising, I just don't know a lot about it.
> > > > >
> > > > > -Jay
> > > > >
> > > > >
> > > > > On Fri, Jan 24, 2014 at 3:30 PM, Clark Breyman <cl...@breyman.com>
> > > > wrote:
> > > > >
> > > > > > Jay - Thanks for the call for comments. Here's some initial
> input:
> > > > > >
> > > > > > - Make message serialization a client responsibility (making all
> > > > messages
> > > > > > byte[]). Reflection-based loading makes it harder to use generic
> > > codecs
> > > > > > (e.g.  Envelope<PREFIX, DATA, SUFFIX>) or build up codec
> > > > > programmatically.
> > > > > > Non-default partitioning should require an explicit partition
> key.
> > > > > >
> > > > > > - I really like the fact that it will be native Java. Please
> > consider
> > > > > using
> > > > > > native maven and not sbt, gradle, ivy, etc as they don't reliably
> > > play
> > > > > nice
> > > > > > in the maven ecosystem. A jar without a well-formed pom doesn't
> > feel
> > > > > like a
> > > > > > real artifact. The pom's generated by sbt et al. are not well
> > formed.
> > > > > Using
> > > > > > maven will make builds and IDE integration much smoother.
> > > > > >
> > > > > > - Look at Nick Telford's dropwizard-extras package in which he
> > > defines
> > > > > some
> > > > > > Jackson-compatible POJO's for loading configuration. Seems like
> > your
> > > > > client
> > > > > > migration is similar. The config objects should have constructors
> > or
> > > > > > factories that accept Map<String, String> and Properties for ease
> > of
> > > > > > migration.
> > > > > >
> > > > > > - Would you consider using the org.apache.kafka package for the
> new
> > > API
> > > > > > (quibble)
> > > > > >
> > > > > > - Why create your own futures rather than use
> > > > > > java.util.concurrent.Future<Long> or similar? Standard futures
> will
> > > > play
> > > > > > nice with other reactive libs and things like J8's
> > ComposableFuture.
> > > > > >
> > > > > > Thanks again,
> > > > > > C
> > > > > >
> > > > > >
> > > > > >
> > > > > > On Fri, Jan 24, 2014 at 2:46 PM, Roger Hoover <
> > > roger.hoover@gmail.com
> > > > > > >wrote:
> > > > > >
> > > > > > > A couple comments:
> > > > > > >
> > > > > > > 1) Why does the config use a broker list instead of discovering
> > the
> > > > > > brokers
> > > > > > > in ZooKeeper?  It doesn't match the HighLevelConsumer API.
> > > > > > >
> > > > > > > 2) It looks like broker connections are created on demand.  I'm
> > > > > wondering
> > > > > > > if sometimes you might want to flush out config or network
> > > > connectivity
> > > > > > > issues before pushing the first message through.
> > > > > > >
> > > > > > > Should there also be a KafkaProducer.connect() or .open()
> method
> > or
> > > > > > > connectAll()?  I guess it would try to connect to all brokers
> in
> > > the
> > > > > > > BROKER_LIST_CONFIG
> > > > > > >
> > > > > > > HTH,
> > > > > > >
> > > > > > > Roger
> > > > > > >
> > > > > > >
> > > > > > > On Fri, Jan 24, 2014 at 11:54 AM, Jay Kreps <
> jay.kreps@gmail.com
> > >
> > > > > wrote:
> > > > > > >
> > > > > > > > As mentioned in a previous email we are working on a
> > > > > re-implementation
> > > > > > of
> > > > > > > > the producer. I would like to use this email thread to
> discuss
> > > the
> > > > > > > details
> > > > > > > > of the public API and the configuration. I would love for us
> to
> > > be
> > > > > > > > incredibly picky about this public api now so it is as good
> as
> > > > > possible
> > > > > > > and
> > > > > > > > we don't need to break it in the future.
> > > > > > > >
> > > > > > > > The best way to get a feel for the API is actually to take a
> > look
> > > > at
> > > > > > the
> > > > > > > > javadoc, my hope is to get the api docs good enough so that
> it
> > is
> > > > > > > > self-explanatory:
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> http://empathybox.com/kafka-javadoc/index.html?kafka/clients/producer/KafkaProducer.html
> > > > > > > >
> > > > > > > > Please take a look at this API and give me any thoughts you
> may
> > > > have!
> > > > > > > >
> > > > > > > > It may also be reasonable to take a look at the configs:
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> http://empathybox.com/kafka-javadoc/kafka/clients/producer/ProducerConfig.html
> > > > > > > >
> > > > > > > > The actual code is posted here:
> > > > > > > > https://issues.apache.org/jira/browse/KAFKA-1227
> > > > > > > >
> > > > > > > > A few questions or comments to kick things off:
> > > > > > > > 1. We need to make a decision on whether serialization of the
> > > > user's
> > > > > > key
> > > > > > > > and value should be done by the user (with our api just
> taking
> > > > > byte[])
> > > > > > or
> > > > > > > > if we should take an object and allow the user to configure a
> > > > > > Serializer
> > > > > > > > class which we instantiate via reflection. We take the later
> > > > approach
> > > > > > in
> > > > > > > > the current producer, and I have carried this through to this
> > > > > > prototype.
> > > > > > > > The tradeoff I see is this: taking byte[] is actually
> simpler,
> > > the
> > > > > user
> > > > > > > can
> > > > > > > > directly do whatever serialization they like. The
> complication
> > is
> > > > > > > actually
> > > > > > > > partitioning. Currently partitioning is done by a similar
> > plug-in
> > > > api
> > > > > > > > (Partitioner) which the user can implement and configure to
> > > > override
> > > > > > how
> > > > > > > > partitions are assigned. If we take byte[] as input then we
> > have
> > > no
> > > > > > > access
> > > > > > > > to the original object and partitioning MUST be done on the
> > > byte[].
> > > > > > This
> > > > > > > is
> > > > > > > > fine for hash partitioning. However for various types of
> > semantic
> > > > > > > > partitioning (range partitioning, or whatever) you would want
> > > > access
> > > > > to
> > > > > > > the
> > > > > > > > original object. In the current approach a producer who
> wishes
> > to
> > > > > send
> > > > > > > > byte[] they have serialized in their own code can configure
> the
> > > > > > > > BytesSerialization we supply which is just a "no op"
> > > serialization.
> > > > > > > > 2. We should obsess over naming and make sure each of the
> class
> > > > names
> > > > > > are
> > > > > > > > good.
> > > > > > > > 3. Jun has already pointed out that we need to include the
> > topic
> > > > and
> > > > > > > > partition in the response, which is absolutely right. I
> haven't
> > > > done
> > > > > > that
> > > > > > > > yet but that definitely needs to be there.
> > > > > > > > 4. Currently RecordSend.await will throw an exception if the
> > > > request
> > > > > > > > failed. The intention here is that
> > producer.send(message).await()
> > > > > > exactly
> > > > > > > > simulates a synchronous call. Guozhang has noted that this
> is a
> > > > > little
> > > > > > > > annoying since the user must then catch exceptions. However
> if
> > we
> > > > > > remove
> > > > > > > > this then if the user doesn't check for errors they won't
> know
> > > one
> > > > > has
> > > > > > > > occurred, which I predict will be a common mistake.
> > > > > > > > 5. Perhaps there is more we could do to make the async
> > callbacks
> > > > and
> > > > > > > future
> > > > > > > > we give back intuitive and easy to program against?
> > > > > > > >
> > > > > > > > Some background info on implementation:
> > > > > > > >
> > > > > > > > At a high level the primary difference in this producer is
> that
> > > it
> > > > > > > removes
> > > > > > > > the distinction between the "sync" and "async" producer.
> > > > Effectively
> > > > > > all
> > > > > > > > requests are sent asynchronously but always return a future
> > > > response
> > > > > > > object
> > > > > > > > that gives the offset as well as any error that may have
> > occurred
> > > > > when
> > > > > > > the
> > > > > > > > request is complete. The batching that is done in the async
> > > > producer
> > > > > > only
> > > > > > > > today is done whenever possible now. This means that the sync
> > > > > producer,
> > > > > > > > under load, can get performance as good as the async producer
> > > > > > > (preliminary
> > > > > > > > results show the producer getting 1m messages/sec). This
> works
> > > > > similar
> > > > > > to
> > > > > > > > group commit in databases but with respect to the actual
> > network
> > > > > > > > transmission--any messages that arrive while a send is in
> > > progress
> > > > > are
> > > > > > > > batched together. It is also possible to encourage batching
> > even
> > > > > under
> > > > > > > low
> > > > > > > > load to save server resources by introducing a delay on the
> > send
> > > to
> > > > > > allow
> > > > > > > > more messages to accumulate; this is done using the
> > > linger.msconfig
> > > > > > > (this
> > > > > > > > is similar to Nagle's algorithm in TCP).
> > > > > > > >
> > > > > > > > This producer does all network communication asynchronously
> and
> > > in
> > > > > > > parallel
> > > > > > > > to all servers so the performance penalty for acks=-1 and
> > waiting
> > > > on
> > > > > > > > replication should be much reduced. I haven't done much
> > > > benchmarking
> > > > > on
> > > > > > > > this yet, though.
> > > > > > > >
> > > > > > > > The high level design is described a little here, though this
> > is
> > > > now
> > > > > a
> > > > > > > > little out of date:
> > > > > > > >
> > https://cwiki.apache.org/confluence/display/KAFKA/Client+Rewrite
> > > > > > > >
> > > > > > > > -Jay
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: New Producer Public API

Posted by Xavier Stevens <xa...@gaikai.com>.
AutoCloseable would be nice for us as most of our code is using Java 7 at
this point.

I like Dropwizard's configuration mapping to POJOs via Jackson, but if you
wanted to stick with property maps I don't care enough to object.

If the producer only dealt with bytes, is there a way we could still due
partition plugins without specifying the number explicitly? I would prefer
to be able to pass in field(s) that would be used by the partitioner.
Obviously if this wasn't possible you could always deserialize the object
in the partitioner and grab the fields you want, but that seems really
expensive to do on every message.

It would also be nice to have a Java API Encoder constructor taking in
VerifiableProperties. Scala understands how to handle "props:
VerifiableProperties = null", but Java doesn't. So you don't run into this
problem until runtime.


-Xavier


On Mon, Jan 27, 2014 at 9:37 AM, Clark Breyman <cl...@breyman.com> wrote:

> Jay -
>
> Config - your explanation makes sense. I'm just so accustomed to having
> Jackson automatically map my configuration objects to POJOs that I've
> stopped using property files. They are lingua franca. The only thought
> might be to separate the config interface from the implementation to allow
> for alternatives, but that might undermine your point of "do it this way so
> that everyone can find it where they expect it".
>
> Serialization: Of the options, I like 1A the best, though possibly with
> either an option to specify a partition key rather than ID or a helper to
> translate an arbitrary byte[] or long into a partition number.
>
> Thanks
> Clark
>
>
> On Sun, Jan 26, 2014 at 9:13 PM, Jay Kreps <ja...@gmail.com> wrote:
>
> > Thanks for the detailed thoughts. Let me elaborate on the config thing.
> >
> > I agree that at first glance key-value strings don't seem like a very
> good
> > configuration api for a client. Surely a well-typed config class would be
> > better! I actually disagree and let me see if I can convince you.
> >
> > My reasoning has nothing to do with the api and everything to do with
> > operations.
> >
> > Clients are embedded in applications which are themselves configured. In
> > any place that takes operations seriously the configuration for these
> > applications will be version controlled and maintained through some kind
> of
> > config management system. If we give a config class with getters and
> > setters the application has to expose those properties to its
> > configuration. What invariably happens is that the application exposes
> only
> > a choice few properties that they thought they would change. Furthermore
> > the application will make up a name for these configs that seems
> intuitive
> > at the time in the 2 seconds the engineer spends thinking about it.
> >
> > Now consider the result of this in the large. You end up with dozens or
> > hundreds of applications that have the client embedded. Each exposes a
> > different, inadequate subset of the possible configs, each with different
> > names. It is a nightmare.
> >
> > If you use a string-string map the config system can directly get a
> bundle
> > of config key-value pairs and put them into the client. This means that
> all
> > configuration is automatically available with the name documented on the
> > website in every application that does this. If you upgrade to a new
> kafka
> > version with more configs those will be exposed too. If you realize that
> > you need to change a default you can just go through your configs and
> > change it everywhere as it will have the same name everywhere.
> >
> > -Jay
> >
> >
> >
> >
> > On Sun, Jan 26, 2014 at 4:47 PM, Clark Breyman <cl...@breyman.com>
> wrote:
> >
> > > Thanks Jay. I'll see if I can put together a more complete response,
> > > perhaps as separate threads so that topics don't get entangled. In the
> > mean
> > > time, here's a couple responses:
> > >
> > > Serialization: you've broken out a sub-thread so i'll reply there. My
> > bias
> > > is that I like generics (except for type-erasure) and in particular
> they
> > > make it easy to compose serializers for compound payloads (e.g. when a
> > > common header wraps a payload of parameterized type). I'll respond to
> > your
> > > 4-options message with an example.
> > >
> > > Build: I've seen a lot of "maven-compatible" build systems produce
> > > "artifacts" that aren't really artifacts - no embedded POM or, worst,
> > > malformed POM. I know the sbt-generated artifacts were this way - onus
> is
> > > on me to see what gradle is spitting out and what a maven build might
> > look
> > > like. Maven may be old and boring, but it gets out of the way and
> > > integrates really seamlessly with a lot of IDEs. When some scala
> > projects I
> > > was working on in the fall of 2011 switched from sbt to maven, build
> > became
> > > a non-issue.
> > >
> > > Config: Not a big deal  and no, I don't think a dropwizard dependency
> is
> > > appropriate. I do like using simple entity beans (POJO's not j2EE) for
> > > configuration, especially if they can be marshalled without annotation
> by
> > > Jackson. I only mentioned the dropwizard-extras  because it has some
> > entity
> > > bean versions of the ZK and Kafka configs.
> > >
> > > Domain-packaging: Also not a big deal - it's what's expected and it's
> > > pretty free in most IDE's. The advantages I see is that it is clear
> > whether
> > > something is from the Apache Kafka project and whether something is
> from
> > > another org and related to Kafka. That said, nothing really enforces
> it.
> > >
> > > Futures: I'll see if I can create some examples to demonstrate Future
> > > making interop easier.
> > >
> > > Regards,
> > > C
> > >
> > >
> > >
> > >
> > > On Fri, Jan 24, 2014 at 4:36 PM, Jay Kreps <ja...@gmail.com>
> wrote:
> > >
> > > > Hey Clark,
> > > >
> > > > - Serialization: Yes I agree with these though I don't consider the
> > loss
> > > of
> > > > generics a big issue. I'll try to summarize what I would consider the
> > > best
> > > > alternative api with raw byte[].
> > > >
> > > > - Maven: We had this debate a few months back and the consensus was
> > > gradle.
> > > > Is there a specific issue with the poms gradle makes? I am extremely
> > > loath
> > > > to revisit the issue as build issues are a recurring thing and no one
> > > ever
> > > > agrees and ultimately our build needs are very simple.
> > > >
> > > > - Config: I'm not sure if I follow the point. Are you saying we
> should
> > > use
> > > > something in dropwizard for config? One principle here is to try to
> > > remove
> > > > as many client dependencies as possible as we inevitably run into
> > > terrible
> > > > compatibility issues with users who use the same library or its
> > > > dependencies at different versions. Or are you talking about
> > maintaining
> > > > compatibility with existing config parameters? I think as much as a
> > > config
> > > > in the existing client makes sense it should have the same name (I
> was
> > a
> > > > bit sloppy about that so I'll fix any errors there). There are a few
> > new
> > > > things and we should give those reasonable defaults. I think config
> is
> > > > important so I'll start a thread on the config package in there.
> > > >
> > > > - org.apache.kafka: We could do this. I always considered it kind of
> an
> > > odd
> > > > thing Java programmers do that has no real motivation (but I could be
> > > > re-educated!). I don't think it ends up reducing naming conflicts in
> > > > practice and it adds a lot of noise and nested directories. Is there
> a
> > > > reason you prefer this or just to be more standard?
> > > >
> > > > - Future: Basically I didn't see any particular advantage. The
> cancel()
> > > > method doesn't really make sense so probably wouldn't work. Likewise
> I
> > > > dislike the checked exceptions it requires. Basically I just wrote
> out
> > > some
> > > > code examples and it seemed cleaner with a special purpose object. I
> > > wasn't
> > > > actually aware of plans for improved futures in java 8 or the other
> > > > integrations. Maybe you could elaborate on this a bit and show how it
> > > would
> > > > be used? Sounds promising, I just don't know a lot about it.
> > > >
> > > > -Jay
> > > >
> > > >
> > > > On Fri, Jan 24, 2014 at 3:30 PM, Clark Breyman <cl...@breyman.com>
> > > wrote:
> > > >
> > > > > Jay - Thanks for the call for comments. Here's some initial input:
> > > > >
> > > > > - Make message serialization a client responsibility (making all
> > > messages
> > > > > byte[]). Reflection-based loading makes it harder to use generic
> > codecs
> > > > > (e.g.  Envelope<PREFIX, DATA, SUFFIX>) or build up codec
> > > > programmatically.
> > > > > Non-default partitioning should require an explicit partition key.
> > > > >
> > > > > - I really like the fact that it will be native Java. Please
> consider
> > > > using
> > > > > native maven and not sbt, gradle, ivy, etc as they don't reliably
> > play
> > > > nice
> > > > > in the maven ecosystem. A jar without a well-formed pom doesn't
> feel
> > > > like a
> > > > > real artifact. The pom's generated by sbt et al. are not well
> formed.
> > > > Using
> > > > > maven will make builds and IDE integration much smoother.
> > > > >
> > > > > - Look at Nick Telford's dropwizard-extras package in which he
> > defines
> > > > some
> > > > > Jackson-compatible POJO's for loading configuration. Seems like
> your
> > > > client
> > > > > migration is similar. The config objects should have constructors
> or
> > > > > factories that accept Map<String, String> and Properties for ease
> of
> > > > > migration.
> > > > >
> > > > > - Would you consider using the org.apache.kafka package for the new
> > API
> > > > > (quibble)
> > > > >
> > > > > - Why create your own futures rather than use
> > > > > java.util.concurrent.Future<Long> or similar? Standard futures will
> > > play
> > > > > nice with other reactive libs and things like J8's
> ComposableFuture.
> > > > >
> > > > > Thanks again,
> > > > > C
> > > > >
> > > > >
> > > > >
> > > > > On Fri, Jan 24, 2014 at 2:46 PM, Roger Hoover <
> > roger.hoover@gmail.com
> > > > > >wrote:
> > > > >
> > > > > > A couple comments:
> > > > > >
> > > > > > 1) Why does the config use a broker list instead of discovering
> the
> > > > > brokers
> > > > > > in ZooKeeper?  It doesn't match the HighLevelConsumer API.
> > > > > >
> > > > > > 2) It looks like broker connections are created on demand.  I'm
> > > > wondering
> > > > > > if sometimes you might want to flush out config or network
> > > connectivity
> > > > > > issues before pushing the first message through.
> > > > > >
> > > > > > Should there also be a KafkaProducer.connect() or .open() method
> or
> > > > > > connectAll()?  I guess it would try to connect to all brokers in
> > the
> > > > > > BROKER_LIST_CONFIG
> > > > > >
> > > > > > HTH,
> > > > > >
> > > > > > Roger
> > > > > >
> > > > > >
> > > > > > On Fri, Jan 24, 2014 at 11:54 AM, Jay Kreps <jay.kreps@gmail.com
> >
> > > > wrote:
> > > > > >
> > > > > > > As mentioned in a previous email we are working on a
> > > > re-implementation
> > > > > of
> > > > > > > the producer. I would like to use this email thread to discuss
> > the
> > > > > > details
> > > > > > > of the public API and the configuration. I would love for us to
> > be
> > > > > > > incredibly picky about this public api now so it is as good as
> > > > possible
> > > > > > and
> > > > > > > we don't need to break it in the future.
> > > > > > >
> > > > > > > The best way to get a feel for the API is actually to take a
> look
> > > at
> > > > > the
> > > > > > > javadoc, my hope is to get the api docs good enough so that it
> is
> > > > > > > self-explanatory:
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> http://empathybox.com/kafka-javadoc/index.html?kafka/clients/producer/KafkaProducer.html
> > > > > > >
> > > > > > > Please take a look at this API and give me any thoughts you may
> > > have!
> > > > > > >
> > > > > > > It may also be reasonable to take a look at the configs:
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> http://empathybox.com/kafka-javadoc/kafka/clients/producer/ProducerConfig.html
> > > > > > >
> > > > > > > The actual code is posted here:
> > > > > > > https://issues.apache.org/jira/browse/KAFKA-1227
> > > > > > >
> > > > > > > A few questions or comments to kick things off:
> > > > > > > 1. We need to make a decision on whether serialization of the
> > > user's
> > > > > key
> > > > > > > and value should be done by the user (with our api just taking
> > > > byte[])
> > > > > or
> > > > > > > if we should take an object and allow the user to configure a
> > > > > Serializer
> > > > > > > class which we instantiate via reflection. We take the later
> > > approach
> > > > > in
> > > > > > > the current producer, and I have carried this through to this
> > > > > prototype.
> > > > > > > The tradeoff I see is this: taking byte[] is actually simpler,
> > the
> > > > user
> > > > > > can
> > > > > > > directly do whatever serialization they like. The complication
> is
> > > > > > actually
> > > > > > > partitioning. Currently partitioning is done by a similar
> plug-in
> > > api
> > > > > > > (Partitioner) which the user can implement and configure to
> > > override
> > > > > how
> > > > > > > partitions are assigned. If we take byte[] as input then we
> have
> > no
> > > > > > access
> > > > > > > to the original object and partitioning MUST be done on the
> > byte[].
> > > > > This
> > > > > > is
> > > > > > > fine for hash partitioning. However for various types of
> semantic
> > > > > > > partitioning (range partitioning, or whatever) you would want
> > > access
> > > > to
> > > > > > the
> > > > > > > original object. In the current approach a producer who wishes
> to
> > > > send
> > > > > > > byte[] they have serialized in their own code can configure the
> > > > > > > BytesSerialization we supply which is just a "no op"
> > serialization.
> > > > > > > 2. We should obsess over naming and make sure each of the class
> > > names
> > > > > are
> > > > > > > good.
> > > > > > > 3. Jun has already pointed out that we need to include the
> topic
> > > and
> > > > > > > partition in the response, which is absolutely right. I haven't
> > > done
> > > > > that
> > > > > > > yet but that definitely needs to be there.
> > > > > > > 4. Currently RecordSend.await will throw an exception if the
> > > request
> > > > > > > failed. The intention here is that
> producer.send(message).await()
> > > > > exactly
> > > > > > > simulates a synchronous call. Guozhang has noted that this is a
> > > > little
> > > > > > > annoying since the user must then catch exceptions. However if
> we
> > > > > remove
> > > > > > > this then if the user doesn't check for errors they won't know
> > one
> > > > has
> > > > > > > occurred, which I predict will be a common mistake.
> > > > > > > 5. Perhaps there is more we could do to make the async
> callbacks
> > > and
> > > > > > future
> > > > > > > we give back intuitive and easy to program against?
> > > > > > >
> > > > > > > Some background info on implementation:
> > > > > > >
> > > > > > > At a high level the primary difference in this producer is that
> > it
> > > > > > removes
> > > > > > > the distinction between the "sync" and "async" producer.
> > > Effectively
> > > > > all
> > > > > > > requests are sent asynchronously but always return a future
> > > response
> > > > > > object
> > > > > > > that gives the offset as well as any error that may have
> occurred
> > > > when
> > > > > > the
> > > > > > > request is complete. The batching that is done in the async
> > > producer
> > > > > only
> > > > > > > today is done whenever possible now. This means that the sync
> > > > producer,
> > > > > > > under load, can get performance as good as the async producer
> > > > > > (preliminary
> > > > > > > results show the producer getting 1m messages/sec). This works
> > > > similar
> > > > > to
> > > > > > > group commit in databases but with respect to the actual
> network
> > > > > > > transmission--any messages that arrive while a send is in
> > progress
> > > > are
> > > > > > > batched together. It is also possible to encourage batching
> even
> > > > under
> > > > > > low
> > > > > > > load to save server resources by introducing a delay on the
> send
> > to
> > > > > allow
> > > > > > > more messages to accumulate; this is done using the
> > linger.msconfig
> > > > > > (this
> > > > > > > is similar to Nagle's algorithm in TCP).
> > > > > > >
> > > > > > > This producer does all network communication asynchronously and
> > in
> > > > > > parallel
> > > > > > > to all servers so the performance penalty for acks=-1 and
> waiting
> > > on
> > > > > > > replication should be much reduced. I haven't done much
> > > benchmarking
> > > > on
> > > > > > > this yet, though.
> > > > > > >
> > > > > > > The high level design is described a little here, though this
> is
> > > now
> > > > a
> > > > > > > little out of date:
> > > > > > >
> https://cwiki.apache.org/confluence/display/KAFKA/Client+Rewrite
> > > > > > >
> > > > > > > -Jay
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: New Producer Public API

Posted by Clark Breyman <cl...@breyman.com>.
Jay -

Config - your explanation makes sense. I'm just so accustomed to having
Jackson automatically map my configuration objects to POJOs that I've
stopped using property files. They are lingua franca. The only thought
might be to separate the config interface from the implementation to allow
for alternatives, but that might undermine your point of "do it this way so
that everyone can find it where they expect it".

Serialization: Of the options, I like 1A the best, though possibly with
either an option to specify a partition key rather than ID or a helper to
translate an arbitrary byte[] or long into a partition number.

Thanks
Clark


On Sun, Jan 26, 2014 at 9:13 PM, Jay Kreps <ja...@gmail.com> wrote:

> Thanks for the detailed thoughts. Let me elaborate on the config thing.
>
> I agree that at first glance key-value strings don't seem like a very good
> configuration api for a client. Surely a well-typed config class would be
> better! I actually disagree and let me see if I can convince you.
>
> My reasoning has nothing to do with the api and everything to do with
> operations.
>
> Clients are embedded in applications which are themselves configured. In
> any place that takes operations seriously the configuration for these
> applications will be version controlled and maintained through some kind of
> config management system. If we give a config class with getters and
> setters the application has to expose those properties to its
> configuration. What invariably happens is that the application exposes only
> a choice few properties that they thought they would change. Furthermore
> the application will make up a name for these configs that seems intuitive
> at the time in the 2 seconds the engineer spends thinking about it.
>
> Now consider the result of this in the large. You end up with dozens or
> hundreds of applications that have the client embedded. Each exposes a
> different, inadequate subset of the possible configs, each with different
> names. It is a nightmare.
>
> If you use a string-string map the config system can directly get a bundle
> of config key-value pairs and put them into the client. This means that all
> configuration is automatically available with the name documented on the
> website in every application that does this. If you upgrade to a new kafka
> version with more configs those will be exposed too. If you realize that
> you need to change a default you can just go through your configs and
> change it everywhere as it will have the same name everywhere.
>
> -Jay
>
>
>
>
> On Sun, Jan 26, 2014 at 4:47 PM, Clark Breyman <cl...@breyman.com> wrote:
>
> > Thanks Jay. I'll see if I can put together a more complete response,
> > perhaps as separate threads so that topics don't get entangled. In the
> mean
> > time, here's a couple responses:
> >
> > Serialization: you've broken out a sub-thread so i'll reply there. My
> bias
> > is that I like generics (except for type-erasure) and in particular they
> > make it easy to compose serializers for compound payloads (e.g. when a
> > common header wraps a payload of parameterized type). I'll respond to
> your
> > 4-options message with an example.
> >
> > Build: I've seen a lot of "maven-compatible" build systems produce
> > "artifacts" that aren't really artifacts - no embedded POM or, worst,
> > malformed POM. I know the sbt-generated artifacts were this way - onus is
> > on me to see what gradle is spitting out and what a maven build might
> look
> > like. Maven may be old and boring, but it gets out of the way and
> > integrates really seamlessly with a lot of IDEs. When some scala
> projects I
> > was working on in the fall of 2011 switched from sbt to maven, build
> became
> > a non-issue.
> >
> > Config: Not a big deal  and no, I don't think a dropwizard dependency is
> > appropriate. I do like using simple entity beans (POJO's not j2EE) for
> > configuration, especially if they can be marshalled without annotation by
> > Jackson. I only mentioned the dropwizard-extras  because it has some
> entity
> > bean versions of the ZK and Kafka configs.
> >
> > Domain-packaging: Also not a big deal - it's what's expected and it's
> > pretty free in most IDE's. The advantages I see is that it is clear
> whether
> > something is from the Apache Kafka project and whether something is from
> > another org and related to Kafka. That said, nothing really enforces it.
> >
> > Futures: I'll see if I can create some examples to demonstrate Future
> > making interop easier.
> >
> > Regards,
> > C
> >
> >
> >
> >
> > On Fri, Jan 24, 2014 at 4:36 PM, Jay Kreps <ja...@gmail.com> wrote:
> >
> > > Hey Clark,
> > >
> > > - Serialization: Yes I agree with these though I don't consider the
> loss
> > of
> > > generics a big issue. I'll try to summarize what I would consider the
> > best
> > > alternative api with raw byte[].
> > >
> > > - Maven: We had this debate a few months back and the consensus was
> > gradle.
> > > Is there a specific issue with the poms gradle makes? I am extremely
> > loath
> > > to revisit the issue as build issues are a recurring thing and no one
> > ever
> > > agrees and ultimately our build needs are very simple.
> > >
> > > - Config: I'm not sure if I follow the point. Are you saying we should
> > use
> > > something in dropwizard for config? One principle here is to try to
> > remove
> > > as many client dependencies as possible as we inevitably run into
> > terrible
> > > compatibility issues with users who use the same library or its
> > > dependencies at different versions. Or are you talking about
> maintaining
> > > compatibility with existing config parameters? I think as much as a
> > config
> > > in the existing client makes sense it should have the same name (I was
> a
> > > bit sloppy about that so I'll fix any errors there). There are a few
> new
> > > things and we should give those reasonable defaults. I think config is
> > > important so I'll start a thread on the config package in there.
> > >
> > > - org.apache.kafka: We could do this. I always considered it kind of an
> > odd
> > > thing Java programmers do that has no real motivation (but I could be
> > > re-educated!). I don't think it ends up reducing naming conflicts in
> > > practice and it adds a lot of noise and nested directories. Is there a
> > > reason you prefer this or just to be more standard?
> > >
> > > - Future: Basically I didn't see any particular advantage. The cancel()
> > > method doesn't really make sense so probably wouldn't work. Likewise I
> > > dislike the checked exceptions it requires. Basically I just wrote out
> > some
> > > code examples and it seemed cleaner with a special purpose object. I
> > wasn't
> > > actually aware of plans for improved futures in java 8 or the other
> > > integrations. Maybe you could elaborate on this a bit and show how it
> > would
> > > be used? Sounds promising, I just don't know a lot about it.
> > >
> > > -Jay
> > >
> > >
> > > On Fri, Jan 24, 2014 at 3:30 PM, Clark Breyman <cl...@breyman.com>
> > wrote:
> > >
> > > > Jay - Thanks for the call for comments. Here's some initial input:
> > > >
> > > > - Make message serialization a client responsibility (making all
> > messages
> > > > byte[]). Reflection-based loading makes it harder to use generic
> codecs
> > > > (e.g.  Envelope<PREFIX, DATA, SUFFIX>) or build up codec
> > > programmatically.
> > > > Non-default partitioning should require an explicit partition key.
> > > >
> > > > - I really like the fact that it will be native Java. Please consider
> > > using
> > > > native maven and not sbt, gradle, ivy, etc as they don't reliably
> play
> > > nice
> > > > in the maven ecosystem. A jar without a well-formed pom doesn't feel
> > > like a
> > > > real artifact. The pom's generated by sbt et al. are not well formed.
> > > Using
> > > > maven will make builds and IDE integration much smoother.
> > > >
> > > > - Look at Nick Telford's dropwizard-extras package in which he
> defines
> > > some
> > > > Jackson-compatible POJO's for loading configuration. Seems like your
> > > client
> > > > migration is similar. The config objects should have constructors or
> > > > factories that accept Map<String, String> and Properties for ease of
> > > > migration.
> > > >
> > > > - Would you consider using the org.apache.kafka package for the new
> API
> > > > (quibble)
> > > >
> > > > - Why create your own futures rather than use
> > > > java.util.concurrent.Future<Long> or similar? Standard futures will
> > play
> > > > nice with other reactive libs and things like J8's ComposableFuture.
> > > >
> > > > Thanks again,
> > > > C
> > > >
> > > >
> > > >
> > > > On Fri, Jan 24, 2014 at 2:46 PM, Roger Hoover <
> roger.hoover@gmail.com
> > > > >wrote:
> > > >
> > > > > A couple comments:
> > > > >
> > > > > 1) Why does the config use a broker list instead of discovering the
> > > > brokers
> > > > > in ZooKeeper?  It doesn't match the HighLevelConsumer API.
> > > > >
> > > > > 2) It looks like broker connections are created on demand.  I'm
> > > wondering
> > > > > if sometimes you might want to flush out config or network
> > connectivity
> > > > > issues before pushing the first message through.
> > > > >
> > > > > Should there also be a KafkaProducer.connect() or .open() method or
> > > > > connectAll()?  I guess it would try to connect to all brokers in
> the
> > > > > BROKER_LIST_CONFIG
> > > > >
> > > > > HTH,
> > > > >
> > > > > Roger
> > > > >
> > > > >
> > > > > On Fri, Jan 24, 2014 at 11:54 AM, Jay Kreps <ja...@gmail.com>
> > > wrote:
> > > > >
> > > > > > As mentioned in a previous email we are working on a
> > > re-implementation
> > > > of
> > > > > > the producer. I would like to use this email thread to discuss
> the
> > > > > details
> > > > > > of the public API and the configuration. I would love for us to
> be
> > > > > > incredibly picky about this public api now so it is as good as
> > > possible
> > > > > and
> > > > > > we don't need to break it in the future.
> > > > > >
> > > > > > The best way to get a feel for the API is actually to take a look
> > at
> > > > the
> > > > > > javadoc, my hope is to get the api docs good enough so that it is
> > > > > > self-explanatory:
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> http://empathybox.com/kafka-javadoc/index.html?kafka/clients/producer/KafkaProducer.html
> > > > > >
> > > > > > Please take a look at this API and give me any thoughts you may
> > have!
> > > > > >
> > > > > > It may also be reasonable to take a look at the configs:
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> http://empathybox.com/kafka-javadoc/kafka/clients/producer/ProducerConfig.html
> > > > > >
> > > > > > The actual code is posted here:
> > > > > > https://issues.apache.org/jira/browse/KAFKA-1227
> > > > > >
> > > > > > A few questions or comments to kick things off:
> > > > > > 1. We need to make a decision on whether serialization of the
> > user's
> > > > key
> > > > > > and value should be done by the user (with our api just taking
> > > byte[])
> > > > or
> > > > > > if we should take an object and allow the user to configure a
> > > > Serializer
> > > > > > class which we instantiate via reflection. We take the later
> > approach
> > > > in
> > > > > > the current producer, and I have carried this through to this
> > > > prototype.
> > > > > > The tradeoff I see is this: taking byte[] is actually simpler,
> the
> > > user
> > > > > can
> > > > > > directly do whatever serialization they like. The complication is
> > > > > actually
> > > > > > partitioning. Currently partitioning is done by a similar plug-in
> > api
> > > > > > (Partitioner) which the user can implement and configure to
> > override
> > > > how
> > > > > > partitions are assigned. If we take byte[] as input then we have
> no
> > > > > access
> > > > > > to the original object and partitioning MUST be done on the
> byte[].
> > > > This
> > > > > is
> > > > > > fine for hash partitioning. However for various types of semantic
> > > > > > partitioning (range partitioning, or whatever) you would want
> > access
> > > to
> > > > > the
> > > > > > original object. In the current approach a producer who wishes to
> > > send
> > > > > > byte[] they have serialized in their own code can configure the
> > > > > > BytesSerialization we supply which is just a "no op"
> serialization.
> > > > > > 2. We should obsess over naming and make sure each of the class
> > names
> > > > are
> > > > > > good.
> > > > > > 3. Jun has already pointed out that we need to include the topic
> > and
> > > > > > partition in the response, which is absolutely right. I haven't
> > done
> > > > that
> > > > > > yet but that definitely needs to be there.
> > > > > > 4. Currently RecordSend.await will throw an exception if the
> > request
> > > > > > failed. The intention here is that producer.send(message).await()
> > > > exactly
> > > > > > simulates a synchronous call. Guozhang has noted that this is a
> > > little
> > > > > > annoying since the user must then catch exceptions. However if we
> > > > remove
> > > > > > this then if the user doesn't check for errors they won't know
> one
> > > has
> > > > > > occurred, which I predict will be a common mistake.
> > > > > > 5. Perhaps there is more we could do to make the async callbacks
> > and
> > > > > future
> > > > > > we give back intuitive and easy to program against?
> > > > > >
> > > > > > Some background info on implementation:
> > > > > >
> > > > > > At a high level the primary difference in this producer is that
> it
> > > > > removes
> > > > > > the distinction between the "sync" and "async" producer.
> > Effectively
> > > > all
> > > > > > requests are sent asynchronously but always return a future
> > response
> > > > > object
> > > > > > that gives the offset as well as any error that may have occurred
> > > when
> > > > > the
> > > > > > request is complete. The batching that is done in the async
> > producer
> > > > only
> > > > > > today is done whenever possible now. This means that the sync
> > > producer,
> > > > > > under load, can get performance as good as the async producer
> > > > > (preliminary
> > > > > > results show the producer getting 1m messages/sec). This works
> > > similar
> > > > to
> > > > > > group commit in databases but with respect to the actual network
> > > > > > transmission--any messages that arrive while a send is in
> progress
> > > are
> > > > > > batched together. It is also possible to encourage batching even
> > > under
> > > > > low
> > > > > > load to save server resources by introducing a delay on the send
> to
> > > > allow
> > > > > > more messages to accumulate; this is done using the
> linger.msconfig
> > > > > (this
> > > > > > is similar to Nagle's algorithm in TCP).
> > > > > >
> > > > > > This producer does all network communication asynchronously and
> in
> > > > > parallel
> > > > > > to all servers so the performance penalty for acks=-1 and waiting
> > on
> > > > > > replication should be much reduced. I haven't done much
> > benchmarking
> > > on
> > > > > > this yet, though.
> > > > > >
> > > > > > The high level design is described a little here, though this is
> > now
> > > a
> > > > > > little out of date:
> > > > > > https://cwiki.apache.org/confluence/display/KAFKA/Client+Rewrite
> > > > > >
> > > > > > -Jay
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: New Producer Public API

Posted by Jay Kreps <ja...@gmail.com>.
Thanks for the detailed thoughts. Let me elaborate on the config thing.

I agree that at first glance key-value strings don't seem like a very good
configuration api for a client. Surely a well-typed config class would be
better! I actually disagree and let me see if I can convince you.

My reasoning has nothing to do with the api and everything to do with
operations.

Clients are embedded in applications which are themselves configured. In
any place that takes operations seriously the configuration for these
applications will be version controlled and maintained through some kind of
config management system. If we give a config class with getters and
setters the application has to expose those properties to its
configuration. What invariably happens is that the application exposes only
a choice few properties that they thought they would change. Furthermore
the application will make up a name for these configs that seems intuitive
at the time in the 2 seconds the engineer spends thinking about it.

Now consider the result of this in the large. You end up with dozens or
hundreds of applications that have the client embedded. Each exposes a
different, inadequate subset of the possible configs, each with different
names. It is a nightmare.

If you use a string-string map the config system can directly get a bundle
of config key-value pairs and put them into the client. This means that all
configuration is automatically available with the name documented on the
website in every application that does this. If you upgrade to a new kafka
version with more configs those will be exposed too. If you realize that
you need to change a default you can just go through your configs and
change it everywhere as it will have the same name everywhere.

-Jay




On Sun, Jan 26, 2014 at 4:47 PM, Clark Breyman <cl...@breyman.com> wrote:

> Thanks Jay. I'll see if I can put together a more complete response,
> perhaps as separate threads so that topics don't get entangled. In the mean
> time, here's a couple responses:
>
> Serialization: you've broken out a sub-thread so i'll reply there. My bias
> is that I like generics (except for type-erasure) and in particular they
> make it easy to compose serializers for compound payloads (e.g. when a
> common header wraps a payload of parameterized type). I'll respond to your
> 4-options message with an example.
>
> Build: I've seen a lot of "maven-compatible" build systems produce
> "artifacts" that aren't really artifacts - no embedded POM or, worst,
> malformed POM. I know the sbt-generated artifacts were this way - onus is
> on me to see what gradle is spitting out and what a maven build might look
> like. Maven may be old and boring, but it gets out of the way and
> integrates really seamlessly with a lot of IDEs. When some scala projects I
> was working on in the fall of 2011 switched from sbt to maven, build became
> a non-issue.
>
> Config: Not a big deal  and no, I don't think a dropwizard dependency is
> appropriate. I do like using simple entity beans (POJO's not j2EE) for
> configuration, especially if they can be marshalled without annotation by
> Jackson. I only mentioned the dropwizard-extras  because it has some entity
> bean versions of the ZK and Kafka configs.
>
> Domain-packaging: Also not a big deal - it's what's expected and it's
> pretty free in most IDE's. The advantages I see is that it is clear whether
> something is from the Apache Kafka project and whether something is from
> another org and related to Kafka. That said, nothing really enforces it.
>
> Futures: I'll see if I can create some examples to demonstrate Future
> making interop easier.
>
> Regards,
> C
>
>
>
>
> On Fri, Jan 24, 2014 at 4:36 PM, Jay Kreps <ja...@gmail.com> wrote:
>
> > Hey Clark,
> >
> > - Serialization: Yes I agree with these though I don't consider the loss
> of
> > generics a big issue. I'll try to summarize what I would consider the
> best
> > alternative api with raw byte[].
> >
> > - Maven: We had this debate a few months back and the consensus was
> gradle.
> > Is there a specific issue with the poms gradle makes? I am extremely
> loath
> > to revisit the issue as build issues are a recurring thing and no one
> ever
> > agrees and ultimately our build needs are very simple.
> >
> > - Config: I'm not sure if I follow the point. Are you saying we should
> use
> > something in dropwizard for config? One principle here is to try to
> remove
> > as many client dependencies as possible as we inevitably run into
> terrible
> > compatibility issues with users who use the same library or its
> > dependencies at different versions. Or are you talking about maintaining
> > compatibility with existing config parameters? I think as much as a
> config
> > in the existing client makes sense it should have the same name (I was a
> > bit sloppy about that so I'll fix any errors there). There are a few new
> > things and we should give those reasonable defaults. I think config is
> > important so I'll start a thread on the config package in there.
> >
> > - org.apache.kafka: We could do this. I always considered it kind of an
> odd
> > thing Java programmers do that has no real motivation (but I could be
> > re-educated!). I don't think it ends up reducing naming conflicts in
> > practice and it adds a lot of noise and nested directories. Is there a
> > reason you prefer this or just to be more standard?
> >
> > - Future: Basically I didn't see any particular advantage. The cancel()
> > method doesn't really make sense so probably wouldn't work. Likewise I
> > dislike the checked exceptions it requires. Basically I just wrote out
> some
> > code examples and it seemed cleaner with a special purpose object. I
> wasn't
> > actually aware of plans for improved futures in java 8 or the other
> > integrations. Maybe you could elaborate on this a bit and show how it
> would
> > be used? Sounds promising, I just don't know a lot about it.
> >
> > -Jay
> >
> >
> > On Fri, Jan 24, 2014 at 3:30 PM, Clark Breyman <cl...@breyman.com>
> wrote:
> >
> > > Jay - Thanks for the call for comments. Here's some initial input:
> > >
> > > - Make message serialization a client responsibility (making all
> messages
> > > byte[]). Reflection-based loading makes it harder to use generic codecs
> > > (e.g.  Envelope<PREFIX, DATA, SUFFIX>) or build up codec
> > programmatically.
> > > Non-default partitioning should require an explicit partition key.
> > >
> > > - I really like the fact that it will be native Java. Please consider
> > using
> > > native maven and not sbt, gradle, ivy, etc as they don't reliably play
> > nice
> > > in the maven ecosystem. A jar without a well-formed pom doesn't feel
> > like a
> > > real artifact. The pom's generated by sbt et al. are not well formed.
> > Using
> > > maven will make builds and IDE integration much smoother.
> > >
> > > - Look at Nick Telford's dropwizard-extras package in which he defines
> > some
> > > Jackson-compatible POJO's for loading configuration. Seems like your
> > client
> > > migration is similar. The config objects should have constructors or
> > > factories that accept Map<String, String> and Properties for ease of
> > > migration.
> > >
> > > - Would you consider using the org.apache.kafka package for the new API
> > > (quibble)
> > >
> > > - Why create your own futures rather than use
> > > java.util.concurrent.Future<Long> or similar? Standard futures will
> play
> > > nice with other reactive libs and things like J8's ComposableFuture.
> > >
> > > Thanks again,
> > > C
> > >
> > >
> > >
> > > On Fri, Jan 24, 2014 at 2:46 PM, Roger Hoover <roger.hoover@gmail.com
> > > >wrote:
> > >
> > > > A couple comments:
> > > >
> > > > 1) Why does the config use a broker list instead of discovering the
> > > brokers
> > > > in ZooKeeper?  It doesn't match the HighLevelConsumer API.
> > > >
> > > > 2) It looks like broker connections are created on demand.  I'm
> > wondering
> > > > if sometimes you might want to flush out config or network
> connectivity
> > > > issues before pushing the first message through.
> > > >
> > > > Should there also be a KafkaProducer.connect() or .open() method or
> > > > connectAll()?  I guess it would try to connect to all brokers in the
> > > > BROKER_LIST_CONFIG
> > > >
> > > > HTH,
> > > >
> > > > Roger
> > > >
> > > >
> > > > On Fri, Jan 24, 2014 at 11:54 AM, Jay Kreps <ja...@gmail.com>
> > wrote:
> > > >
> > > > > As mentioned in a previous email we are working on a
> > re-implementation
> > > of
> > > > > the producer. I would like to use this email thread to discuss the
> > > > details
> > > > > of the public API and the configuration. I would love for us to be
> > > > > incredibly picky about this public api now so it is as good as
> > possible
> > > > and
> > > > > we don't need to break it in the future.
> > > > >
> > > > > The best way to get a feel for the API is actually to take a look
> at
> > > the
> > > > > javadoc, my hope is to get the api docs good enough so that it is
> > > > > self-explanatory:
> > > > >
> > > > >
> > > >
> > >
> >
> http://empathybox.com/kafka-javadoc/index.html?kafka/clients/producer/KafkaProducer.html
> > > > >
> > > > > Please take a look at this API and give me any thoughts you may
> have!
> > > > >
> > > > > It may also be reasonable to take a look at the configs:
> > > > >
> > > > >
> > > >
> > >
> >
> http://empathybox.com/kafka-javadoc/kafka/clients/producer/ProducerConfig.html
> > > > >
> > > > > The actual code is posted here:
> > > > > https://issues.apache.org/jira/browse/KAFKA-1227
> > > > >
> > > > > A few questions or comments to kick things off:
> > > > > 1. We need to make a decision on whether serialization of the
> user's
> > > key
> > > > > and value should be done by the user (with our api just taking
> > byte[])
> > > or
> > > > > if we should take an object and allow the user to configure a
> > > Serializer
> > > > > class which we instantiate via reflection. We take the later
> approach
> > > in
> > > > > the current producer, and I have carried this through to this
> > > prototype.
> > > > > The tradeoff I see is this: taking byte[] is actually simpler, the
> > user
> > > > can
> > > > > directly do whatever serialization they like. The complication is
> > > > actually
> > > > > partitioning. Currently partitioning is done by a similar plug-in
> api
> > > > > (Partitioner) which the user can implement and configure to
> override
> > > how
> > > > > partitions are assigned. If we take byte[] as input then we have no
> > > > access
> > > > > to the original object and partitioning MUST be done on the byte[].
> > > This
> > > > is
> > > > > fine for hash partitioning. However for various types of semantic
> > > > > partitioning (range partitioning, or whatever) you would want
> access
> > to
> > > > the
> > > > > original object. In the current approach a producer who wishes to
> > send
> > > > > byte[] they have serialized in their own code can configure the
> > > > > BytesSerialization we supply which is just a "no op" serialization.
> > > > > 2. We should obsess over naming and make sure each of the class
> names
> > > are
> > > > > good.
> > > > > 3. Jun has already pointed out that we need to include the topic
> and
> > > > > partition in the response, which is absolutely right. I haven't
> done
> > > that
> > > > > yet but that definitely needs to be there.
> > > > > 4. Currently RecordSend.await will throw an exception if the
> request
> > > > > failed. The intention here is that producer.send(message).await()
> > > exactly
> > > > > simulates a synchronous call. Guozhang has noted that this is a
> > little
> > > > > annoying since the user must then catch exceptions. However if we
> > > remove
> > > > > this then if the user doesn't check for errors they won't know one
> > has
> > > > > occurred, which I predict will be a common mistake.
> > > > > 5. Perhaps there is more we could do to make the async callbacks
> and
> > > > future
> > > > > we give back intuitive and easy to program against?
> > > > >
> > > > > Some background info on implementation:
> > > > >
> > > > > At a high level the primary difference in this producer is that it
> > > > removes
> > > > > the distinction between the "sync" and "async" producer.
> Effectively
> > > all
> > > > > requests are sent asynchronously but always return a future
> response
> > > > object
> > > > > that gives the offset as well as any error that may have occurred
> > when
> > > > the
> > > > > request is complete. The batching that is done in the async
> producer
> > > only
> > > > > today is done whenever possible now. This means that the sync
> > producer,
> > > > > under load, can get performance as good as the async producer
> > > > (preliminary
> > > > > results show the producer getting 1m messages/sec). This works
> > similar
> > > to
> > > > > group commit in databases but with respect to the actual network
> > > > > transmission--any messages that arrive while a send is in progress
> > are
> > > > > batched together. It is also possible to encourage batching even
> > under
> > > > low
> > > > > load to save server resources by introducing a delay on the send to
> > > allow
> > > > > more messages to accumulate; this is done using the linger.msconfig
> > > > (this
> > > > > is similar to Nagle's algorithm in TCP).
> > > > >
> > > > > This producer does all network communication asynchronously and in
> > > > parallel
> > > > > to all servers so the performance penalty for acks=-1 and waiting
> on
> > > > > replication should be much reduced. I haven't done much
> benchmarking
> > on
> > > > > this yet, though.
> > > > >
> > > > > The high level design is described a little here, though this is
> now
> > a
> > > > > little out of date:
> > > > > https://cwiki.apache.org/confluence/display/KAFKA/Client+Rewrite
> > > > >
> > > > > -Jay
> > > > >
> > > >
> > >
> >
>

Re: New Producer Public API

Posted by Clark Breyman <cl...@breyman.com>.
Thanks Jay. I'll see if I can put together a more complete response,
perhaps as separate threads so that topics don't get entangled. In the mean
time, here's a couple responses:

Serialization: you've broken out a sub-thread so i'll reply there. My bias
is that I like generics (except for type-erasure) and in particular they
make it easy to compose serializers for compound payloads (e.g. when a
common header wraps a payload of parameterized type). I'll respond to your
4-options message with an example.

Build: I've seen a lot of "maven-compatible" build systems produce
"artifacts" that aren't really artifacts - no embedded POM or, worst,
malformed POM. I know the sbt-generated artifacts were this way - onus is
on me to see what gradle is spitting out and what a maven build might look
like. Maven may be old and boring, but it gets out of the way and
integrates really seamlessly with a lot of IDEs. When some scala projects I
was working on in the fall of 2011 switched from sbt to maven, build became
a non-issue.

Config: Not a big deal  and no, I don't think a dropwizard dependency is
appropriate. I do like using simple entity beans (POJO's not j2EE) for
configuration, especially if they can be marshalled without annotation by
Jackson. I only mentioned the dropwizard-extras  because it has some entity
bean versions of the ZK and Kafka configs.

Domain-packaging: Also not a big deal - it's what's expected and it's
pretty free in most IDE's. The advantages I see is that it is clear whether
something is from the Apache Kafka project and whether something is from
another org and related to Kafka. That said, nothing really enforces it.

Futures: I'll see if I can create some examples to demonstrate Future
making interop easier.

Regards,
C




On Fri, Jan 24, 2014 at 4:36 PM, Jay Kreps <ja...@gmail.com> wrote:

> Hey Clark,
>
> - Serialization: Yes I agree with these though I don't consider the loss of
> generics a big issue. I'll try to summarize what I would consider the best
> alternative api with raw byte[].
>
> - Maven: We had this debate a few months back and the consensus was gradle.
> Is there a specific issue with the poms gradle makes? I am extremely loath
> to revisit the issue as build issues are a recurring thing and no one ever
> agrees and ultimately our build needs are very simple.
>
> - Config: I'm not sure if I follow the point. Are you saying we should use
> something in dropwizard for config? One principle here is to try to remove
> as many client dependencies as possible as we inevitably run into terrible
> compatibility issues with users who use the same library or its
> dependencies at different versions. Or are you talking about maintaining
> compatibility with existing config parameters? I think as much as a config
> in the existing client makes sense it should have the same name (I was a
> bit sloppy about that so I'll fix any errors there). There are a few new
> things and we should give those reasonable defaults. I think config is
> important so I'll start a thread on the config package in there.
>
> - org.apache.kafka: We could do this. I always considered it kind of an odd
> thing Java programmers do that has no real motivation (but I could be
> re-educated!). I don't think it ends up reducing naming conflicts in
> practice and it adds a lot of noise and nested directories. Is there a
> reason you prefer this or just to be more standard?
>
> - Future: Basically I didn't see any particular advantage. The cancel()
> method doesn't really make sense so probably wouldn't work. Likewise I
> dislike the checked exceptions it requires. Basically I just wrote out some
> code examples and it seemed cleaner with a special purpose object. I wasn't
> actually aware of plans for improved futures in java 8 or the other
> integrations. Maybe you could elaborate on this a bit and show how it would
> be used? Sounds promising, I just don't know a lot about it.
>
> -Jay
>
>
> On Fri, Jan 24, 2014 at 3:30 PM, Clark Breyman <cl...@breyman.com> wrote:
>
> > Jay - Thanks for the call for comments. Here's some initial input:
> >
> > - Make message serialization a client responsibility (making all messages
> > byte[]). Reflection-based loading makes it harder to use generic codecs
> > (e.g.  Envelope<PREFIX, DATA, SUFFIX>) or build up codec
> programmatically.
> > Non-default partitioning should require an explicit partition key.
> >
> > - I really like the fact that it will be native Java. Please consider
> using
> > native maven and not sbt, gradle, ivy, etc as they don't reliably play
> nice
> > in the maven ecosystem. A jar without a well-formed pom doesn't feel
> like a
> > real artifact. The pom's generated by sbt et al. are not well formed.
> Using
> > maven will make builds and IDE integration much smoother.
> >
> > - Look at Nick Telford's dropwizard-extras package in which he defines
> some
> > Jackson-compatible POJO's for loading configuration. Seems like your
> client
> > migration is similar. The config objects should have constructors or
> > factories that accept Map<String, String> and Properties for ease of
> > migration.
> >
> > - Would you consider using the org.apache.kafka package for the new API
> > (quibble)
> >
> > - Why create your own futures rather than use
> > java.util.concurrent.Future<Long> or similar? Standard futures will play
> > nice with other reactive libs and things like J8's ComposableFuture.
> >
> > Thanks again,
> > C
> >
> >
> >
> > On Fri, Jan 24, 2014 at 2:46 PM, Roger Hoover <roger.hoover@gmail.com
> > >wrote:
> >
> > > A couple comments:
> > >
> > > 1) Why does the config use a broker list instead of discovering the
> > brokers
> > > in ZooKeeper?  It doesn't match the HighLevelConsumer API.
> > >
> > > 2) It looks like broker connections are created on demand.  I'm
> wondering
> > > if sometimes you might want to flush out config or network connectivity
> > > issues before pushing the first message through.
> > >
> > > Should there also be a KafkaProducer.connect() or .open() method or
> > > connectAll()?  I guess it would try to connect to all brokers in the
> > > BROKER_LIST_CONFIG
> > >
> > > HTH,
> > >
> > > Roger
> > >
> > >
> > > On Fri, Jan 24, 2014 at 11:54 AM, Jay Kreps <ja...@gmail.com>
> wrote:
> > >
> > > > As mentioned in a previous email we are working on a
> re-implementation
> > of
> > > > the producer. I would like to use this email thread to discuss the
> > > details
> > > > of the public API and the configuration. I would love for us to be
> > > > incredibly picky about this public api now so it is as good as
> possible
> > > and
> > > > we don't need to break it in the future.
> > > >
> > > > The best way to get a feel for the API is actually to take a look at
> > the
> > > > javadoc, my hope is to get the api docs good enough so that it is
> > > > self-explanatory:
> > > >
> > > >
> > >
> >
> http://empathybox.com/kafka-javadoc/index.html?kafka/clients/producer/KafkaProducer.html
> > > >
> > > > Please take a look at this API and give me any thoughts you may have!
> > > >
> > > > It may also be reasonable to take a look at the configs:
> > > >
> > > >
> > >
> >
> http://empathybox.com/kafka-javadoc/kafka/clients/producer/ProducerConfig.html
> > > >
> > > > The actual code is posted here:
> > > > https://issues.apache.org/jira/browse/KAFKA-1227
> > > >
> > > > A few questions or comments to kick things off:
> > > > 1. We need to make a decision on whether serialization of the user's
> > key
> > > > and value should be done by the user (with our api just taking
> byte[])
> > or
> > > > if we should take an object and allow the user to configure a
> > Serializer
> > > > class which we instantiate via reflection. We take the later approach
> > in
> > > > the current producer, and I have carried this through to this
> > prototype.
> > > > The tradeoff I see is this: taking byte[] is actually simpler, the
> user
> > > can
> > > > directly do whatever serialization they like. The complication is
> > > actually
> > > > partitioning. Currently partitioning is done by a similar plug-in api
> > > > (Partitioner) which the user can implement and configure to override
> > how
> > > > partitions are assigned. If we take byte[] as input then we have no
> > > access
> > > > to the original object and partitioning MUST be done on the byte[].
> > This
> > > is
> > > > fine for hash partitioning. However for various types of semantic
> > > > partitioning (range partitioning, or whatever) you would want access
> to
> > > the
> > > > original object. In the current approach a producer who wishes to
> send
> > > > byte[] they have serialized in their own code can configure the
> > > > BytesSerialization we supply which is just a "no op" serialization.
> > > > 2. We should obsess over naming and make sure each of the class names
> > are
> > > > good.
> > > > 3. Jun has already pointed out that we need to include the topic and
> > > > partition in the response, which is absolutely right. I haven't done
> > that
> > > > yet but that definitely needs to be there.
> > > > 4. Currently RecordSend.await will throw an exception if the request
> > > > failed. The intention here is that producer.send(message).await()
> > exactly
> > > > simulates a synchronous call. Guozhang has noted that this is a
> little
> > > > annoying since the user must then catch exceptions. However if we
> > remove
> > > > this then if the user doesn't check for errors they won't know one
> has
> > > > occurred, which I predict will be a common mistake.
> > > > 5. Perhaps there is more we could do to make the async callbacks and
> > > future
> > > > we give back intuitive and easy to program against?
> > > >
> > > > Some background info on implementation:
> > > >
> > > > At a high level the primary difference in this producer is that it
> > > removes
> > > > the distinction between the "sync" and "async" producer. Effectively
> > all
> > > > requests are sent asynchronously but always return a future response
> > > object
> > > > that gives the offset as well as any error that may have occurred
> when
> > > the
> > > > request is complete. The batching that is done in the async producer
> > only
> > > > today is done whenever possible now. This means that the sync
> producer,
> > > > under load, can get performance as good as the async producer
> > > (preliminary
> > > > results show the producer getting 1m messages/sec). This works
> similar
> > to
> > > > group commit in databases but with respect to the actual network
> > > > transmission--any messages that arrive while a send is in progress
> are
> > > > batched together. It is also possible to encourage batching even
> under
> > > low
> > > > load to save server resources by introducing a delay on the send to
> > allow
> > > > more messages to accumulate; this is done using the linger.ms config
> > > (this
> > > > is similar to Nagle's algorithm in TCP).
> > > >
> > > > This producer does all network communication asynchronously and in
> > > parallel
> > > > to all servers so the performance penalty for acks=-1 and waiting on
> > > > replication should be much reduced. I haven't done much benchmarking
> on
> > > > this yet, though.
> > > >
> > > > The high level design is described a little here, though this is now
> a
> > > > little out of date:
> > > > https://cwiki.apache.org/confluence/display/KAFKA/Client+Rewrite
> > > >
> > > > -Jay
> > > >
> > >
> >
>

Re: New Producer Public API

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

- Serialization: Yes I agree with these though I don't consider the loss of
generics a big issue. I'll try to summarize what I would consider the best
alternative api with raw byte[].

- Maven: We had this debate a few months back and the consensus was gradle.
Is there a specific issue with the poms gradle makes? I am extremely loath
to revisit the issue as build issues are a recurring thing and no one ever
agrees and ultimately our build needs are very simple.

- Config: I'm not sure if I follow the point. Are you saying we should use
something in dropwizard for config? One principle here is to try to remove
as many client dependencies as possible as we inevitably run into terrible
compatibility issues with users who use the same library or its
dependencies at different versions. Or are you talking about maintaining
compatibility with existing config parameters? I think as much as a config
in the existing client makes sense it should have the same name (I was a
bit sloppy about that so I'll fix any errors there). There are a few new
things and we should give those reasonable defaults. I think config is
important so I'll start a thread on the config package in there.

- org.apache.kafka: We could do this. I always considered it kind of an odd
thing Java programmers do that has no real motivation (but I could be
re-educated!). I don't think it ends up reducing naming conflicts in
practice and it adds a lot of noise and nested directories. Is there a
reason you prefer this or just to be more standard?

- Future: Basically I didn't see any particular advantage. The cancel()
method doesn't really make sense so probably wouldn't work. Likewise I
dislike the checked exceptions it requires. Basically I just wrote out some
code examples and it seemed cleaner with a special purpose object. I wasn't
actually aware of plans for improved futures in java 8 or the other
integrations. Maybe you could elaborate on this a bit and show how it would
be used? Sounds promising, I just don't know a lot about it.

-Jay


On Fri, Jan 24, 2014 at 3:30 PM, Clark Breyman <cl...@breyman.com> wrote:

> Jay - Thanks for the call for comments. Here's some initial input:
>
> - Make message serialization a client responsibility (making all messages
> byte[]). Reflection-based loading makes it harder to use generic codecs
> (e.g.  Envelope<PREFIX, DATA, SUFFIX>) or build up codec programmatically.
> Non-default partitioning should require an explicit partition key.
>
> - I really like the fact that it will be native Java. Please consider using
> native maven and not sbt, gradle, ivy, etc as they don't reliably play nice
> in the maven ecosystem. A jar without a well-formed pom doesn't feel like a
> real artifact. The pom's generated by sbt et al. are not well formed. Using
> maven will make builds and IDE integration much smoother.
>
> - Look at Nick Telford's dropwizard-extras package in which he defines some
> Jackson-compatible POJO's for loading configuration. Seems like your client
> migration is similar. The config objects should have constructors or
> factories that accept Map<String, String> and Properties for ease of
> migration.
>
> - Would you consider using the org.apache.kafka package for the new API
> (quibble)
>
> - Why create your own futures rather than use
> java.util.concurrent.Future<Long> or similar? Standard futures will play
> nice with other reactive libs and things like J8's ComposableFuture.
>
> Thanks again,
> C
>
>
>
> On Fri, Jan 24, 2014 at 2:46 PM, Roger Hoover <roger.hoover@gmail.com
> >wrote:
>
> > A couple comments:
> >
> > 1) Why does the config use a broker list instead of discovering the
> brokers
> > in ZooKeeper?  It doesn't match the HighLevelConsumer API.
> >
> > 2) It looks like broker connections are created on demand.  I'm wondering
> > if sometimes you might want to flush out config or network connectivity
> > issues before pushing the first message through.
> >
> > Should there also be a KafkaProducer.connect() or .open() method or
> > connectAll()?  I guess it would try to connect to all brokers in the
> > BROKER_LIST_CONFIG
> >
> > HTH,
> >
> > Roger
> >
> >
> > On Fri, Jan 24, 2014 at 11:54 AM, Jay Kreps <ja...@gmail.com> wrote:
> >
> > > As mentioned in a previous email we are working on a re-implementation
> of
> > > the producer. I would like to use this email thread to discuss the
> > details
> > > of the public API and the configuration. I would love for us to be
> > > incredibly picky about this public api now so it is as good as possible
> > and
> > > we don't need to break it in the future.
> > >
> > > The best way to get a feel for the API is actually to take a look at
> the
> > > javadoc, my hope is to get the api docs good enough so that it is
> > > self-explanatory:
> > >
> > >
> >
> http://empathybox.com/kafka-javadoc/index.html?kafka/clients/producer/KafkaProducer.html
> > >
> > > Please take a look at this API and give me any thoughts you may have!
> > >
> > > It may also be reasonable to take a look at the configs:
> > >
> > >
> >
> http://empathybox.com/kafka-javadoc/kafka/clients/producer/ProducerConfig.html
> > >
> > > The actual code is posted here:
> > > https://issues.apache.org/jira/browse/KAFKA-1227
> > >
> > > A few questions or comments to kick things off:
> > > 1. We need to make a decision on whether serialization of the user's
> key
> > > and value should be done by the user (with our api just taking byte[])
> or
> > > if we should take an object and allow the user to configure a
> Serializer
> > > class which we instantiate via reflection. We take the later approach
> in
> > > the current producer, and I have carried this through to this
> prototype.
> > > The tradeoff I see is this: taking byte[] is actually simpler, the user
> > can
> > > directly do whatever serialization they like. The complication is
> > actually
> > > partitioning. Currently partitioning is done by a similar plug-in api
> > > (Partitioner) which the user can implement and configure to override
> how
> > > partitions are assigned. If we take byte[] as input then we have no
> > access
> > > to the original object and partitioning MUST be done on the byte[].
> This
> > is
> > > fine for hash partitioning. However for various types of semantic
> > > partitioning (range partitioning, or whatever) you would want access to
> > the
> > > original object. In the current approach a producer who wishes to send
> > > byte[] they have serialized in their own code can configure the
> > > BytesSerialization we supply which is just a "no op" serialization.
> > > 2. We should obsess over naming and make sure each of the class names
> are
> > > good.
> > > 3. Jun has already pointed out that we need to include the topic and
> > > partition in the response, which is absolutely right. I haven't done
> that
> > > yet but that definitely needs to be there.
> > > 4. Currently RecordSend.await will throw an exception if the request
> > > failed. The intention here is that producer.send(message).await()
> exactly
> > > simulates a synchronous call. Guozhang has noted that this is a little
> > > annoying since the user must then catch exceptions. However if we
> remove
> > > this then if the user doesn't check for errors they won't know one has
> > > occurred, which I predict will be a common mistake.
> > > 5. Perhaps there is more we could do to make the async callbacks and
> > future
> > > we give back intuitive and easy to program against?
> > >
> > > Some background info on implementation:
> > >
> > > At a high level the primary difference in this producer is that it
> > removes
> > > the distinction between the "sync" and "async" producer. Effectively
> all
> > > requests are sent asynchronously but always return a future response
> > object
> > > that gives the offset as well as any error that may have occurred when
> > the
> > > request is complete. The batching that is done in the async producer
> only
> > > today is done whenever possible now. This means that the sync producer,
> > > under load, can get performance as good as the async producer
> > (preliminary
> > > results show the producer getting 1m messages/sec). This works similar
> to
> > > group commit in databases but with respect to the actual network
> > > transmission--any messages that arrive while a send is in progress are
> > > batched together. It is also possible to encourage batching even under
> > low
> > > load to save server resources by introducing a delay on the send to
> allow
> > > more messages to accumulate; this is done using the linger.ms config
> > (this
> > > is similar to Nagle's algorithm in TCP).
> > >
> > > This producer does all network communication asynchronously and in
> > parallel
> > > to all servers so the performance penalty for acks=-1 and waiting on
> > > replication should be much reduced. I haven't done much benchmarking on
> > > this yet, though.
> > >
> > > The high level design is described a little here, though this is now a
> > > little out of date:
> > > https://cwiki.apache.org/confluence/display/KAFKA/Client+Rewrite
> > >
> > > -Jay
> > >
> >
>

Re: New Producer Public API

Posted by Clark Breyman <cl...@breyman.com>.
Jay - Thanks for the call for comments. Here's some initial input:

- Make message serialization a client responsibility (making all messages
byte[]). Reflection-based loading makes it harder to use generic codecs
(e.g.  Envelope<PREFIX, DATA, SUFFIX>) or build up codec programmatically.
Non-default partitioning should require an explicit partition key.

- I really like the fact that it will be native Java. Please consider using
native maven and not sbt, gradle, ivy, etc as they don't reliably play nice
in the maven ecosystem. A jar without a well-formed pom doesn't feel like a
real artifact. The pom's generated by sbt et al. are not well formed. Using
maven will make builds and IDE integration much smoother.

- Look at Nick Telford's dropwizard-extras package in which he defines some
Jackson-compatible POJO's for loading configuration. Seems like your client
migration is similar. The config objects should have constructors or
factories that accept Map<String, String> and Properties for ease of
migration.

- Would you consider using the org.apache.kafka package for the new API
(quibble)

- Why create your own futures rather than use
java.util.concurrent.Future<Long> or similar? Standard futures will play
nice with other reactive libs and things like J8's ComposableFuture.

Thanks again,
C



On Fri, Jan 24, 2014 at 2:46 PM, Roger Hoover <ro...@gmail.com>wrote:

> A couple comments:
>
> 1) Why does the config use a broker list instead of discovering the brokers
> in ZooKeeper?  It doesn't match the HighLevelConsumer API.
>
> 2) It looks like broker connections are created on demand.  I'm wondering
> if sometimes you might want to flush out config or network connectivity
> issues before pushing the first message through.
>
> Should there also be a KafkaProducer.connect() or .open() method or
> connectAll()?  I guess it would try to connect to all brokers in the
> BROKER_LIST_CONFIG
>
> HTH,
>
> Roger
>
>
> On Fri, Jan 24, 2014 at 11:54 AM, Jay Kreps <ja...@gmail.com> wrote:
>
> > As mentioned in a previous email we are working on a re-implementation of
> > the producer. I would like to use this email thread to discuss the
> details
> > of the public API and the configuration. I would love for us to be
> > incredibly picky about this public api now so it is as good as possible
> and
> > we don't need to break it in the future.
> >
> > The best way to get a feel for the API is actually to take a look at the
> > javadoc, my hope is to get the api docs good enough so that it is
> > self-explanatory:
> >
> >
> http://empathybox.com/kafka-javadoc/index.html?kafka/clients/producer/KafkaProducer.html
> >
> > Please take a look at this API and give me any thoughts you may have!
> >
> > It may also be reasonable to take a look at the configs:
> >
> >
> http://empathybox.com/kafka-javadoc/kafka/clients/producer/ProducerConfig.html
> >
> > The actual code is posted here:
> > https://issues.apache.org/jira/browse/KAFKA-1227
> >
> > A few questions or comments to kick things off:
> > 1. We need to make a decision on whether serialization of the user's key
> > and value should be done by the user (with our api just taking byte[]) or
> > if we should take an object and allow the user to configure a Serializer
> > class which we instantiate via reflection. We take the later approach in
> > the current producer, and I have carried this through to this prototype.
> > The tradeoff I see is this: taking byte[] is actually simpler, the user
> can
> > directly do whatever serialization they like. The complication is
> actually
> > partitioning. Currently partitioning is done by a similar plug-in api
> > (Partitioner) which the user can implement and configure to override how
> > partitions are assigned. If we take byte[] as input then we have no
> access
> > to the original object and partitioning MUST be done on the byte[]. This
> is
> > fine for hash partitioning. However for various types of semantic
> > partitioning (range partitioning, or whatever) you would want access to
> the
> > original object. In the current approach a producer who wishes to send
> > byte[] they have serialized in their own code can configure the
> > BytesSerialization we supply which is just a "no op" serialization.
> > 2. We should obsess over naming and make sure each of the class names are
> > good.
> > 3. Jun has already pointed out that we need to include the topic and
> > partition in the response, which is absolutely right. I haven't done that
> > yet but that definitely needs to be there.
> > 4. Currently RecordSend.await will throw an exception if the request
> > failed. The intention here is that producer.send(message).await() exactly
> > simulates a synchronous call. Guozhang has noted that this is a little
> > annoying since the user must then catch exceptions. However if we remove
> > this then if the user doesn't check for errors they won't know one has
> > occurred, which I predict will be a common mistake.
> > 5. Perhaps there is more we could do to make the async callbacks and
> future
> > we give back intuitive and easy to program against?
> >
> > Some background info on implementation:
> >
> > At a high level the primary difference in this producer is that it
> removes
> > the distinction between the "sync" and "async" producer. Effectively all
> > requests are sent asynchronously but always return a future response
> object
> > that gives the offset as well as any error that may have occurred when
> the
> > request is complete. The batching that is done in the async producer only
> > today is done whenever possible now. This means that the sync producer,
> > under load, can get performance as good as the async producer
> (preliminary
> > results show the producer getting 1m messages/sec). This works similar to
> > group commit in databases but with respect to the actual network
> > transmission--any messages that arrive while a send is in progress are
> > batched together. It is also possible to encourage batching even under
> low
> > load to save server resources by introducing a delay on the send to allow
> > more messages to accumulate; this is done using the linger.ms config
> (this
> > is similar to Nagle's algorithm in TCP).
> >
> > This producer does all network communication asynchronously and in
> parallel
> > to all servers so the performance penalty for acks=-1 and waiting on
> > replication should be much reduced. I haven't done much benchmarking on
> > this yet, though.
> >
> > The high level design is described a little here, though this is now a
> > little out of date:
> > https://cwiki.apache.org/confluence/display/KAFKA/Client+Rewrite
> >
> > -Jay
> >
>

Re: New Producer Public API

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

These are good questions.

1. The producer since 0.8 is actually zookeeper free, so this is not new to
this client it is true for the current client as well. Our experience was
that direct zookeeper connections from zillions of producers wasn't a good
idea for a number of reasons. Our intention is to remove this dependency
from the consumer as well. The configuration in the producer doesn't need
the full set of brokers, though, just one or two machines to bootstrap the
state of the cluster from--in other words it isn't like you need to
reconfigure your clients every time you add some servers. This is exactly
how zookeeper works too--if we used zookeeper you would need to give a list
of zk urls in case a particular zk server was down. Basically either way
you need a few statically configured nodes to go to discover the full state
of the cluster. For people who don't like hard coding hosts you can use a
VIP or dns or something instead.

2. Yes this is a good point and was a concern I had too--the current
behavior is that with bad urls the client would start normally and then
hang trying to fetch metadata when the first message is sent and finally
give up and throw an exception. This is not ideal.

The challenge is this: we use the bootstrap urls to fetch metadata for
particular topics but we don't know which until we start getting messages
for them. We have the option of fetching metadata for all topics but the
problem is that for a cluster hosting tens of thousands of topics that is
actually a ton of data.

An alternative that this just made me think of is that we could proactively
connect to bootstrap urls sequentially until one succeeds when the producer
is first created and fail fast if we can't establish a connection. This
would not be wasted work as we could use the connection for the metadata
request when the first message is sent. I like this solution and will
implement it. So thanks for asking!

-Jay



On Fri, Jan 24, 2014 at 2:46 PM, Roger Hoover <ro...@gmail.com>wrote:

> A couple comments:
>
> 1) Why does the config use a broker list instead of discovering the brokers
> in ZooKeeper?  It doesn't match the HighLevelConsumer API.
>
> 2) It looks like broker connections are created on demand.  I'm wondering
> if sometimes you might want to flush out config or network connectivity
> issues before pushing the first message through.
>
> Should there also be a KafkaProducer.connect() or .open() method or
> connectAll()?  I guess it would try to connect to all brokers in the
> BROKER_LIST_CONFIG
>
> HTH,
>
> Roger
>
>
> On Fri, Jan 24, 2014 at 11:54 AM, Jay Kreps <ja...@gmail.com> wrote:
>
> > As mentioned in a previous email we are working on a re-implementation of
> > the producer. I would like to use this email thread to discuss the
> details
> > of the public API and the configuration. I would love for us to be
> > incredibly picky about this public api now so it is as good as possible
> and
> > we don't need to break it in the future.
> >
> > The best way to get a feel for the API is actually to take a look at the
> > javadoc, my hope is to get the api docs good enough so that it is
> > self-explanatory:
> >
> >
> http://empathybox.com/kafka-javadoc/index.html?kafka/clients/producer/KafkaProducer.html
> >
> > Please take a look at this API and give me any thoughts you may have!
> >
> > It may also be reasonable to take a look at the configs:
> >
> >
> http://empathybox.com/kafka-javadoc/kafka/clients/producer/ProducerConfig.html
> >
> > The actual code is posted here:
> > https://issues.apache.org/jira/browse/KAFKA-1227
> >
> > A few questions or comments to kick things off:
> > 1. We need to make a decision on whether serialization of the user's key
> > and value should be done by the user (with our api just taking byte[]) or
> > if we should take an object and allow the user to configure a Serializer
> > class which we instantiate via reflection. We take the later approach in
> > the current producer, and I have carried this through to this prototype.
> > The tradeoff I see is this: taking byte[] is actually simpler, the user
> can
> > directly do whatever serialization they like. The complication is
> actually
> > partitioning. Currently partitioning is done by a similar plug-in api
> > (Partitioner) which the user can implement and configure to override how
> > partitions are assigned. If we take byte[] as input then we have no
> access
> > to the original object and partitioning MUST be done on the byte[]. This
> is
> > fine for hash partitioning. However for various types of semantic
> > partitioning (range partitioning, or whatever) you would want access to
> the
> > original object. In the current approach a producer who wishes to send
> > byte[] they have serialized in their own code can configure the
> > BytesSerialization we supply which is just a "no op" serialization.
> > 2. We should obsess over naming and make sure each of the class names are
> > good.
> > 3. Jun has already pointed out that we need to include the topic and
> > partition in the response, which is absolutely right. I haven't done that
> > yet but that definitely needs to be there.
> > 4. Currently RecordSend.await will throw an exception if the request
> > failed. The intention here is that producer.send(message).await() exactly
> > simulates a synchronous call. Guozhang has noted that this is a little
> > annoying since the user must then catch exceptions. However if we remove
> > this then if the user doesn't check for errors they won't know one has
> > occurred, which I predict will be a common mistake.
> > 5. Perhaps there is more we could do to make the async callbacks and
> future
> > we give back intuitive and easy to program against?
> >
> > Some background info on implementation:
> >
> > At a high level the primary difference in this producer is that it
> removes
> > the distinction between the "sync" and "async" producer. Effectively all
> > requests are sent asynchronously but always return a future response
> object
> > that gives the offset as well as any error that may have occurred when
> the
> > request is complete. The batching that is done in the async producer only
> > today is done whenever possible now. This means that the sync producer,
> > under load, can get performance as good as the async producer
> (preliminary
> > results show the producer getting 1m messages/sec). This works similar to
> > group commit in databases but with respect to the actual network
> > transmission--any messages that arrive while a send is in progress are
> > batched together. It is also possible to encourage batching even under
> low
> > load to save server resources by introducing a delay on the send to allow
> > more messages to accumulate; this is done using the linger.ms config
> (this
> > is similar to Nagle's algorithm in TCP).
> >
> > This producer does all network communication asynchronously and in
> parallel
> > to all servers so the performance penalty for acks=-1 and waiting on
> > replication should be much reduced. I haven't done much benchmarking on
> > this yet, though.
> >
> > The high level design is described a little here, though this is now a
> > little out of date:
> > https://cwiki.apache.org/confluence/display/KAFKA/Client+Rewrite
> >
> > -Jay
> >
>

Re: New Producer Public API

Posted by Roger Hoover <ro...@gmail.com>.
A couple comments:

1) Why does the config use a broker list instead of discovering the brokers
in ZooKeeper?  It doesn't match the HighLevelConsumer API.

2) It looks like broker connections are created on demand.  I'm wondering
if sometimes you might want to flush out config or network connectivity
issues before pushing the first message through.

Should there also be a KafkaProducer.connect() or .open() method or
connectAll()?  I guess it would try to connect to all brokers in the
BROKER_LIST_CONFIG

HTH,

Roger


On Fri, Jan 24, 2014 at 11:54 AM, Jay Kreps <ja...@gmail.com> wrote:

> As mentioned in a previous email we are working on a re-implementation of
> the producer. I would like to use this email thread to discuss the details
> of the public API and the configuration. I would love for us to be
> incredibly picky about this public api now so it is as good as possible and
> we don't need to break it in the future.
>
> The best way to get a feel for the API is actually to take a look at the
> javadoc, my hope is to get the api docs good enough so that it is
> self-explanatory:
>
> http://empathybox.com/kafka-javadoc/index.html?kafka/clients/producer/KafkaProducer.html
>
> Please take a look at this API and give me any thoughts you may have!
>
> It may also be reasonable to take a look at the configs:
>
> http://empathybox.com/kafka-javadoc/kafka/clients/producer/ProducerConfig.html
>
> The actual code is posted here:
> https://issues.apache.org/jira/browse/KAFKA-1227
>
> A few questions or comments to kick things off:
> 1. We need to make a decision on whether serialization of the user's key
> and value should be done by the user (with our api just taking byte[]) or
> if we should take an object and allow the user to configure a Serializer
> class which we instantiate via reflection. We take the later approach in
> the current producer, and I have carried this through to this prototype.
> The tradeoff I see is this: taking byte[] is actually simpler, the user can
> directly do whatever serialization they like. The complication is actually
> partitioning. Currently partitioning is done by a similar plug-in api
> (Partitioner) which the user can implement and configure to override how
> partitions are assigned. If we take byte[] as input then we have no access
> to the original object and partitioning MUST be done on the byte[]. This is
> fine for hash partitioning. However for various types of semantic
> partitioning (range partitioning, or whatever) you would want access to the
> original object. In the current approach a producer who wishes to send
> byte[] they have serialized in their own code can configure the
> BytesSerialization we supply which is just a "no op" serialization.
> 2. We should obsess over naming and make sure each of the class names are
> good.
> 3. Jun has already pointed out that we need to include the topic and
> partition in the response, which is absolutely right. I haven't done that
> yet but that definitely needs to be there.
> 4. Currently RecordSend.await will throw an exception if the request
> failed. The intention here is that producer.send(message).await() exactly
> simulates a synchronous call. Guozhang has noted that this is a little
> annoying since the user must then catch exceptions. However if we remove
> this then if the user doesn't check for errors they won't know one has
> occurred, which I predict will be a common mistake.
> 5. Perhaps there is more we could do to make the async callbacks and future
> we give back intuitive and easy to program against?
>
> Some background info on implementation:
>
> At a high level the primary difference in this producer is that it removes
> the distinction between the "sync" and "async" producer. Effectively all
> requests are sent asynchronously but always return a future response object
> that gives the offset as well as any error that may have occurred when the
> request is complete. The batching that is done in the async producer only
> today is done whenever possible now. This means that the sync producer,
> under load, can get performance as good as the async producer (preliminary
> results show the producer getting 1m messages/sec). This works similar to
> group commit in databases but with respect to the actual network
> transmission--any messages that arrive while a send is in progress are
> batched together. It is also possible to encourage batching even under low
> load to save server resources by introducing a delay on the send to allow
> more messages to accumulate; this is done using the linger.ms config (this
> is similar to Nagle's algorithm in TCP).
>
> This producer does all network communication asynchronously and in parallel
> to all servers so the performance penalty for acks=-1 and waiting on
> replication should be much reduced. I haven't done much benchmarking on
> this yet, though.
>
> The high level design is described a little here, though this is now a
> little out of date:
> https://cwiki.apache.org/confluence/display/KAFKA/Client+Rewrite
>
> -Jay
>

Re: New Producer Public API

Posted by Roger Hoover <ro...@gmail.com>.
A couple comments:

1) Why does the config use a broker list instead of discovering the brokers
in ZooKeeper?  It doesn't match the HighLevelConsumer API.

2) It looks like broker connections are created on demand.  I'm wondering
if sometimes you might want to flush out config or network connectivity
issues before pushing the first message through.

Should there also be a KafkaProducer.connect() or .open() method or
connectAll()?  I guess it would try to connect to all brokers in the
BROKER_LIST_CONFIG

HTH,

Roger


On Fri, Jan 24, 2014 at 11:54 AM, Jay Kreps <ja...@gmail.com> wrote:

> As mentioned in a previous email we are working on a re-implementation of
> the producer. I would like to use this email thread to discuss the details
> of the public API and the configuration. I would love for us to be
> incredibly picky about this public api now so it is as good as possible and
> we don't need to break it in the future.
>
> The best way to get a feel for the API is actually to take a look at the
> javadoc, my hope is to get the api docs good enough so that it is
> self-explanatory:
>
> http://empathybox.com/kafka-javadoc/index.html?kafka/clients/producer/KafkaProducer.html
>
> Please take a look at this API and give me any thoughts you may have!
>
> It may also be reasonable to take a look at the configs:
>
> http://empathybox.com/kafka-javadoc/kafka/clients/producer/ProducerConfig.html
>
> The actual code is posted here:
> https://issues.apache.org/jira/browse/KAFKA-1227
>
> A few questions or comments to kick things off:
> 1. We need to make a decision on whether serialization of the user's key
> and value should be done by the user (with our api just taking byte[]) or
> if we should take an object and allow the user to configure a Serializer
> class which we instantiate via reflection. We take the later approach in
> the current producer, and I have carried this through to this prototype.
> The tradeoff I see is this: taking byte[] is actually simpler, the user can
> directly do whatever serialization they like. The complication is actually
> partitioning. Currently partitioning is done by a similar plug-in api
> (Partitioner) which the user can implement and configure to override how
> partitions are assigned. If we take byte[] as input then we have no access
> to the original object and partitioning MUST be done on the byte[]. This is
> fine for hash partitioning. However for various types of semantic
> partitioning (range partitioning, or whatever) you would want access to the
> original object. In the current approach a producer who wishes to send
> byte[] they have serialized in their own code can configure the
> BytesSerialization we supply which is just a "no op" serialization.
> 2. We should obsess over naming and make sure each of the class names are
> good.
> 3. Jun has already pointed out that we need to include the topic and
> partition in the response, which is absolutely right. I haven't done that
> yet but that definitely needs to be there.
> 4. Currently RecordSend.await will throw an exception if the request
> failed. The intention here is that producer.send(message).await() exactly
> simulates a synchronous call. Guozhang has noted that this is a little
> annoying since the user must then catch exceptions. However if we remove
> this then if the user doesn't check for errors they won't know one has
> occurred, which I predict will be a common mistake.
> 5. Perhaps there is more we could do to make the async callbacks and future
> we give back intuitive and easy to program against?
>
> Some background info on implementation:
>
> At a high level the primary difference in this producer is that it removes
> the distinction between the "sync" and "async" producer. Effectively all
> requests are sent asynchronously but always return a future response object
> that gives the offset as well as any error that may have occurred when the
> request is complete. The batching that is done in the async producer only
> today is done whenever possible now. This means that the sync producer,
> under load, can get performance as good as the async producer (preliminary
> results show the producer getting 1m messages/sec). This works similar to
> group commit in databases but with respect to the actual network
> transmission--any messages that arrive while a send is in progress are
> batched together. It is also possible to encourage batching even under low
> load to save server resources by introducing a delay on the send to allow
> more messages to accumulate; this is done using the linger.ms config (this
> is similar to Nagle's algorithm in TCP).
>
> This producer does all network communication asynchronously and in parallel
> to all servers so the performance penalty for acks=-1 and waiting on
> replication should be much reduced. I haven't done much benchmarking on
> this yet, though.
>
> The high level design is described a little here, though this is now a
> little out of date:
> https://cwiki.apache.org/confluence/display/KAFKA/Client+Rewrite
>
> -Jay
>

Re: New Producer Public API

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

Metadata: Yes, this is how it works. You give a URL or a few URLs to
bootstrap from. From then on any metadata change will percolate up to all
producers so you should be able to dynamically change the cluster in any
way without needing to restart or reconfigure the producers. So I think you
are happy there...

For serializers and partitioners. Currently there is a default partitioner
that partitions by whatever key you supply. There is no default serializer
you MUST specify this in the configuration for the producer. The reason for
requiring the user to specify it is that it is really confusing if we
default to some default serializer because then when you forget to
configure the serializer you get this weird class cast exception from using
the wrong serializer with the wrong data. The other alternatives are what
Clark mentioned do you prefer the current approach to that?

-Jay


On Fri, Jan 24, 2014 at 4:59 PM, Joseph Lawson <jl...@roomkey.com> wrote:

> My 2 cents:
>
> Getting the broker metadata via active brokers is the way to go. It allows
> one to dynamically rebalance or introduce a whole new set of servers into a
> cluster just by adding them to the cluster and migrating partitions. We use
> this to periodically introduce newer Kafka cluster cloudformations into our
> running kafka cluster and retire an old Kafka broker cloudformation without
> disrupting the consumers or producers.
>
> I'd rather have the partition action done as is with a default serializer
> that takes a key and does the balancing. I can feed data to partitions such
> as user sessions using a cookie id unique to their session and know any
> partition will have the whole session if I want to practice on the stream.
> Also there is the option to include a custom serializer via passing it into
> the producer constructor so I feel like having to include my own every time
> is a step backwards. I might be misunderstanding this.
>
> Joe Lawson
>
> On Jan 24, 2014 2:54 PM, Jay Kreps <ja...@gmail.com> wrote:
> As mentioned in a previous email we are working on a re-implementation of
> the producer. I would like to use this email thread to discuss the details
> of the public API and the configuration. I would love for us to be
> incredibly picky about this public api now so it is as good as possible and
> we don't need to break it in the future.
>
> The best way to get a feel for the API is actually to take a look at the
> javadoc, my hope is to get the api docs good enough so that it is
> self-explanatory:
>
> http://empathybox.com/kafka-javadoc/index.html?kafka/clients/producer/KafkaProducer.html
>
> Please take a look at this API and give me any thoughts you may have!
>
> It may also be reasonable to take a look at the configs:
>
> http://empathybox.com/kafka-javadoc/kafka/clients/producer/ProducerConfig.html
>
> The actual code is posted here:
> https://issues.apache.org/jira/browse/KAFKA-1227
>
> A few questions or comments to kick things off:
> 1. We need to make a decision on whether serialization of the user's key
> and value should be done by the user (with our api just taking byte[]) or
> if we should take an object and allow the user to configure a Serializer
> class which we instantiate via reflection. We take the later approach in
> the current producer, and I have carried this through to this prototype.
> The tradeoff I see is this: taking byte[] is actually simpler, the user can
> directly do whatever serialization they like. The complication is actually
> partitioning. Currently partitioning is done by a similar plug-in api
> (Partitioner) which the user can implement and configure to override how
> partitions are assigned. If we take byte[] as input then we have no access
> to the original object and partitioning MUST be done on the byte[]. This is
> fine for hash partitioning. However for various types of semantic
> partitioning (range partitioning, or whatever) you would want access to the
> original object. In the current approach a producer who wishes to send
> byte[] they have serialized in their own code can configure the
> BytesSerialization we supply which is just a "no op" serialization.
> 2. We should obsess over naming and make sure each of the class names are
> good.
> 3. Jun has already pointed out that we need to include the topic and
> partition in the response, which is absolutely right. I haven't done that
> yet but that definitely needs to be there.
> 4. Currently RecordSend.await will throw an exception if the request
> failed. The intention here is that producer.send(message).await() exactly
> simulates a synchronous call. Guozhang has noted that this is a little
> annoying since the user must then catch exceptions. However if we remove
> this then if the user doesn't check for errors they won't know one has
> occurred, which I predict will be a common mistake.
> 5. Perhaps there is more we could do to make the async callbacks and future
> we give back intuitive and easy to program against?
>
> Some background info on implementation:
>
> At a high level the primary difference in this producer is that it removes
> the distinction between the "sync" and "async" producer. Effectively all
> requests are sent asynchronously but always return a future response object
> that gives the offset as well as any error that may have occurred when the
> request is complete. The batching that is done in the async producer only
> today is done whenever possible now. This means that the sync producer,
> under load, can get performance as good as the async producer (preliminary
> results show the producer getting 1m messages/sec). This works similar to
> group commit in databases but with respect to the actual network
> transmission--any messages that arrive while a send is in progress are
> batched together. It is also possible to encourage batching even under low
> load to save server resources by introducing a delay on the send to allow
> more messages to accumulate; this is done using the linger.ms config (this
> is similar to Nagle's algorithm in TCP).
>
> This producer does all network communication asynchronously and in parallel
> to all servers so the performance penalty for acks=-1 and waiting on
> replication should be much reduced. I haven't done much benchmarking on
> this yet, though.
>
> The high level design is described a little here, though this is now a
> little out of date:
> https://cwiki.apache.org/confluence/display/KAFKA/Client+Rewrite
>
> -Jay
>

Re: New Producer Public API

Posted by Joseph Lawson <jl...@roomkey.com>.
My 2 cents:

Getting the broker metadata via active brokers is the way to go. It allows one to dynamically rebalance or introduce a whole new set of servers into a cluster just by adding them to the cluster and migrating partitions. We use this to periodically introduce newer Kafka cluster cloudformations into our running kafka cluster and retire an old Kafka broker cloudformation without disrupting the consumers or producers.

I'd rather have the partition action done as is with a default serializer that takes a key and does the balancing. I can feed data to partitions such as user sessions using a cookie id unique to their session and know any partition will have the whole session if I want to practice on the stream. Also there is the option to include a custom serializer via passing it into the producer constructor so I feel like having to include my own every time is a step backwards. I might be misunderstanding this.

Joe Lawson

On Jan 24, 2014 2:54 PM, Jay Kreps <ja...@gmail.com> wrote:
As mentioned in a previous email we are working on a re-implementation of
the producer. I would like to use this email thread to discuss the details
of the public API and the configuration. I would love for us to be
incredibly picky about this public api now so it is as good as possible and
we don't need to break it in the future.

The best way to get a feel for the API is actually to take a look at the
javadoc, my hope is to get the api docs good enough so that it is
self-explanatory:
http://empathybox.com/kafka-javadoc/index.html?kafka/clients/producer/KafkaProducer.html

Please take a look at this API and give me any thoughts you may have!

It may also be reasonable to take a look at the configs:
http://empathybox.com/kafka-javadoc/kafka/clients/producer/ProducerConfig.html

The actual code is posted here:
https://issues.apache.org/jira/browse/KAFKA-1227

A few questions or comments to kick things off:
1. We need to make a decision on whether serialization of the user's key
and value should be done by the user (with our api just taking byte[]) or
if we should take an object and allow the user to configure a Serializer
class which we instantiate via reflection. We take the later approach in
the current producer, and I have carried this through to this prototype.
The tradeoff I see is this: taking byte[] is actually simpler, the user can
directly do whatever serialization they like. The complication is actually
partitioning. Currently partitioning is done by a similar plug-in api
(Partitioner) which the user can implement and configure to override how
partitions are assigned. If we take byte[] as input then we have no access
to the original object and partitioning MUST be done on the byte[]. This is
fine for hash partitioning. However for various types of semantic
partitioning (range partitioning, or whatever) you would want access to the
original object. In the current approach a producer who wishes to send
byte[] they have serialized in their own code can configure the
BytesSerialization we supply which is just a "no op" serialization.
2. We should obsess over naming and make sure each of the class names are
good.
3. Jun has already pointed out that we need to include the topic and
partition in the response, which is absolutely right. I haven't done that
yet but that definitely needs to be there.
4. Currently RecordSend.await will throw an exception if the request
failed. The intention here is that producer.send(message).await() exactly
simulates a synchronous call. Guozhang has noted that this is a little
annoying since the user must then catch exceptions. However if we remove
this then if the user doesn't check for errors they won't know one has
occurred, which I predict will be a common mistake.
5. Perhaps there is more we could do to make the async callbacks and future
we give back intuitive and easy to program against?

Some background info on implementation:

At a high level the primary difference in this producer is that it removes
the distinction between the "sync" and "async" producer. Effectively all
requests are sent asynchronously but always return a future response object
that gives the offset as well as any error that may have occurred when the
request is complete. The batching that is done in the async producer only
today is done whenever possible now. This means that the sync producer,
under load, can get performance as good as the async producer (preliminary
results show the producer getting 1m messages/sec). This works similar to
group commit in databases but with respect to the actual network
transmission--any messages that arrive while a send is in progress are
batched together. It is also possible to encourage batching even under low
load to save server resources by introducing a delay on the send to allow
more messages to accumulate; this is done using the linger.ms config (this
is similar to Nagle's algorithm in TCP).

This producer does all network communication asynchronously and in parallel
to all servers so the performance penalty for acks=-1 and waiting on
replication should be much reduced. I haven't done much benchmarking on
this yet, though.

The high level design is described a little here, though this is now a
little out of date:
https://cwiki.apache.org/confluence/display/KAFKA/Client+Rewrite

-Jay

Fwd: New Producer Public API

Posted by Jay Kreps <ja...@gmail.com>.
A few of us have been working on improving the Kafka clients. If anyone has
input on the api design we are discussing the producer now. Thread is below.

-Jay
---------- Forwarded message ----------
From: Jay Kreps <ja...@gmail.com>
Date: Fri, Jan 24, 2014 at 11:54 AM
Subject: New Producer Public API
To: "dev@kafka.apache.org" <de...@kafka.apache.org>, "users@kafka.apache.org"
<us...@kafka.apache.org>


As mentioned in a previous email we are working on a re-implementation of
the producer. I would like to use this email thread to discuss the details
of the public API and the configuration. I would love for us to be
incredibly picky about this public api now so it is as good as possible and
we don't need to break it in the future.

The best way to get a feel for the API is actually to take a look at the
javadoc, my hope is to get the api docs good enough so that it is
self-explanatory:
http://empathybox.com/kafka-javadoc/index.html?kafka/clients/producer/KafkaProducer.html

Please take a look at this API and give me any thoughts you may have!

It may also be reasonable to take a look at the configs:
http://empathybox.com/kafka-javadoc/kafka/clients/producer/ProducerConfig.html

The actual code is posted here:
https://issues.apache.org/jira/browse/KAFKA-1227

A few questions or comments to kick things off:
1. We need to make a decision on whether serialization of the user's key
and value should be done by the user (with our api just taking byte[]) or
if we should take an object and allow the user to configure a Serializer
class which we instantiate via reflection. We take the later approach in
the current producer, and I have carried this through to this prototype.
The tradeoff I see is this: taking byte[] is actually simpler, the user can
directly do whatever serialization they like. The complication is actually
partitioning. Currently partitioning is done by a similar plug-in api
(Partitioner) which the user can implement and configure to override how
partitions are assigned. If we take byte[] as input then we have no access
to the original object and partitioning MUST be done on the byte[]. This is
fine for hash partitioning. However for various types of semantic
partitioning (range partitioning, or whatever) you would want access to the
original object. In the current approach a producer who wishes to send
byte[] they have serialized in their own code can configure the
BytesSerialization we supply which is just a "no op" serialization.
2. We should obsess over naming and make sure each of the class names are
good.
3. Jun has already pointed out that we need to include the topic and
partition in the response, which is absolutely right. I haven't done that
yet but that definitely needs to be there.
4. Currently RecordSend.await will throw an exception if the request
failed. The intention here is that producer.send(message).await() exactly
simulates a synchronous call. Guozhang has noted that this is a little
annoying since the user must then catch exceptions. However if we remove
this then if the user doesn't check for errors they won't know one has
occurred, which I predict will be a common mistake.
5. Perhaps there is more we could do to make the async callbacks and future
we give back intuitive and easy to program against?

Some background info on implementation:

At a high level the primary difference in this producer is that it removes
the distinction between the "sync" and "async" producer. Effectively all
requests are sent asynchronously but always return a future response object
that gives the offset as well as any error that may have occurred when the
request is complete. The batching that is done in the async producer only
today is done whenever possible now. This means that the sync producer,
under load, can get performance as good as the async producer (preliminary
results show the producer getting 1m messages/sec). This works similar to
group commit in databases but with respect to the actual network
transmission--any messages that arrive while a send is in progress are
batched together. It is also possible to encourage batching even under low
load to save server resources by introducing a delay on the send to allow
more messages to accumulate; this is done using the linger.ms config (this
is similar to Nagle's algorithm in TCP).

This producer does all network communication asynchronously and in parallel
to all servers so the performance penalty for acks=-1 and waiting on
replication should be much reduced. I haven't done much benchmarking on
this yet, though.

The high level design is described a little here, though this is now a
little out of date:
https://cwiki.apache.org/confluence/display/KAFKA/Client+Rewrite

-Jay