Thank you, Jason. Nice to see inputs from you about messaging retention. I
have the same feeling that a centralized place for managing offsets (what
Pulsar calls cursors) is actually much better than current kafka model.
Would love to hear what is the plan for kafka community heading towards on
this part.

What is your thought wrt queuing vs streaming and selective acking?

I believe a lot of users are using the kafka high level consumers, it is
effectively an **unordered** messaging/streaming pattern. People using high
level consumers don't actually need any ordering guarantees. In this sense,
a *shared* subscription in Apache Pulsar seems to be better than current
Kafka's consumer group model, as it allows the consumption rate not limited
by the number of partitions, can actually grow beyond the number of
partitions. We do see a lot of operational pain points on production coming
from consumer lags, which I think it is very commonly seen during partition
rebalancing in a consumer group. Selective acking seems to provide a finer
granularity on acknowledgment, which can be actually good for avoiding
consumer lags and avoid reprocessing messages during partition rebalance.

Frankly speaking, **ordered** messaging is only true when using kafka low
level consumers and it seems only useful if users want to achieve
"exactly-once" processing. Reading more and more on the pulsar
documentation, it seems pulsar's messaging model is actually suitable and
better for most of the kafka use cases (e.g. high level consumer users,
at-least-once delivery). I am wondering is there any plan in Kafka for
improving its messaging model, regarding consumer lags happening during
partition rebalance (e.g. support shared consumption on one topic
partition, and selective acking). If so, will the messaging model like
Pulsar's?

The last question, from users perspective, since both kafka and pulsar are
distributed pub/sub messaging systems and both of them at the ASF, is there
any possibility for these two projects to collaborate, e.g. kafka adopts
pulsar's messaging model, pulsar can use kafka streams and kafka connect. I
believe a lot of people in the mailing list might have same or similar
question. From end-user perspective, if such collaboration can happen, that
is going to great for users and also the ASF. I would like to hear any
thoughts from kafka committers and pmc members.

- KN

On Mon, Dec 4, 2017 at 11:56 AM, Jason Gustafson <ja...@confluent.io> wrote:

> Hi Khurrum,
>
> Thanks for sharing the article. I think one interesting aspect of Pulsar
> that stands out to me is its notion of a subscription and how it impacts
> message retention. In Kafka, consumers are more loosely coupled and
> retention is enforced independently of consumption. There are some
> scenarios I can imagine where the tighter coupling might be beneficial. For
> example, in Kafka Streams, we often use intermediate topics to store the
> data in one stage of the topology's computation. These topics are
> exclusively owned by the application and once the messages have been
> successfully received by the next stage, we do not need to retain them
> further. But since consumption is independent of retention, we either have
> to choose a large retention time and deal with some temporary storage waste
> or we use a low retention time and possibly lose some messages during an
> outage.
>
> We have solved this problem to some extent in Kafka by introducing an API
> to delete the records in a partition up to a certain offset, but this
> effectively puts the burden of this use case on clients. It would be
> interesting to consider whether we could do something like Pulsar in the
> Kafka broker. For example, we have a consumer group coordinator which is
> able to track the progress of the group through its committed offsets. It
> might be possible to extend it to automatically delete records in a topic
> after offsets are committed if the topic is known to be exclusively owned
> by the consumer group. We already have the DeleteRecords API that need, so
> maybe this is "just" a matter of some additional topic metadata. I'd be
> interested to hear whether this kind of use case is common among our users.
>
> -Jason
>
> On Sun, Dec 3, 2017 at 10:29 PM, Khurrum Nasim <khurrumnas...@gmail.com>
> wrote:
>
> > Dear Kafka Community,
> >
> > I happened to read this blog post comparing the messaging model between
> > Apache Pulsar and Apache Kafka. It sounds interesting. Apache Pulsar
> claims
> > to unify streaming (kafka) and queuing (rabbitmq) in one unified API.
> > Pulsar also seems to support Kafka API. Have anyone taken a look at
> Pulsar?
> > How does the community think about this? Pulsar is also an Apache
> project.
> > Is there any collaboration can happen between these two projects?
> >
> > https://streaml.io/blog/pulsar-streaming-queuing/
> >
> > BTW, I am a Kafka user, loving Kafka a lot. Just try to see what other
> > people think about this.
> >
> > - KN
> >
>

Reply via email to