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