You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@pulsar.apache.org by Enrico Olivelli <eo...@gmail.com> on 2021/12/24 09:53:37 UTC

[DISCUSS] PIP-124: Pulsar Client Shared State API

Hello everyone,
I want to start a discussion about PIP-124 Pulsar Client Shared  State API

This is the PIP document
https://github.com/apache/pulsar/issues/13490

This is a demo implementation (a proof-of-concept):
https://github.com/eolivelli/pulsar-shared-state-manager

Please take a look and share your thoughts

I believe that this will unlock the potential of the Exclusive
Producer and it will also make easier the life of many developers who
are using Pulsar and need some API to share configuration, metadata,
or any simple key-value data structure without adding a Database or
other components to their library, Pulsar IO connector or Pulsar
Protocol Handler.

Thanks
Enrico

Re: [DISCUSS] PIP-124: Pulsar Client Shared State API

Posted by Enrico Olivelli <eo...@gmail.com>.
@matteo ping

Enrico

Il Mer 29 Dic 2021, 08:35 Enrico Olivelli <eo...@gmail.com> ha scritto:

> Matteo,
>
> Il Mer 29 Dic 2021, 02:57 Matteo Merli <ma...@gmail.com> ha
> scritto:
>
>> > * Add an API to the Java client that makes it easier to maintain a
>> consistent Share State between instances of an application.
>> > * Provide some ready to use recipes, like a simple key-value store
>> >
>> > It is not a goal to implement a Pulsar backed Database system
>>
>> While the first use case for Pulsar was indeed to be the
>> messaging/replication platform for a distributed database, and it has
>> been working in production for many years, I'm not convinced to add
>> this level of API as part of the Pulsar client API.
>>
>> Pulsar API has been designed to be high-level and easy to use (and
>> reason about), with in mind the use cases of application developers. I
>> don't think that a "storage" level API fits well with the rest of
>> abstractions.
>>
>> > public interface PulsarMap<K,V> extends AutoCloseable {
>> > ..
>> >  CompletableFuture<V> put(K key, V value)
>>
>> If all the logic is implemented in the client side, when there are
>> multiple clients sharing the same, how can any of them mutate the
>> state, since we actually enforce that there is a single exclusive
>> producer? Would a user get an error if there's already a different
>> client writing?
>>
>> My impression is that, while looking convenient, a shared Map
>> interface is not the best abstraction for either case:
>>  * If you're actually building a DB, you will definitely need access
>> to the log itself rather than a Map interface
>>  * If you want to share some state across multiple processes without
>> using a DB, there are many tricky API, consistency and semantic
>> problems to solve, many of which are just pushed down to the
>> application which will need to be aware and understand them. At that
>> point, I would seriously recommend using a DB, or if the question is:
>> "I don't want to use an additional external system", then to use the
>> BK TableService component.
>>
>
> This is usually not a option because the BK TableService does not support
> well multi tenancy and also the application would need to connect to the
> Bookies (think about configuration, security...)
>
>
>>
>> I think this feature should be best viewed as a recipe, as it doesn't
>> depend on or benefits from any internal broker support. If there are
>> enough interest and concrete use cases it can be then included later.
>>
>
> My initial proposal was to push this to Pulsar Adapters.
> I changed the proposal before sending the PIP because I think it very
> useful for Protocol Handlers  and in Pulsar IO connectors.
>
> I am totally fine to add this to pulsar-adapters, but I want to see this
> in the Pulsar repo and released as part of an official Pulsar recipe.
>
> @Matteo does this sound like a good option to you?
>
> Otherwise we miss the possibility to make it easier for Pulsar users to
> leverage this power of Pulsar.
>
> In Pravega you have State Synchronizers and they are a great foundational
> API and we are missing something like that in Pulsar.
>
> Enrico
>
>
>
>> --
>> Matteo Merli
>> <ma...@gmail.com>
>>
>> On Fri, Dec 24, 2021 at 1:53 AM Enrico Olivelli <eo...@gmail.com>
>> wrote:
>> >
>> > Hello everyone,
>> > I want to start a discussion about PIP-124 Pulsar Client Shared  State
>> API
>> >
>> > This is the PIP document
>> > https://github.com/apache/pulsar/issues/13490
>> >
>> > This is a demo implementation (a proof-of-concept):
>> > https://github.com/eolivelli/pulsar-shared-state-manager
>> >
>> > Please take a look and share your thoughts
>> >
>> > I believe that this will unlock the potential of the Exclusive
>> > Producer and it will also make easier the life of many developers who
>> > are using Pulsar and need some API to share configuration, metadata,
>> > or any simple key-value data structure without adding a Database or
>> > other components to their library, Pulsar IO connector or Pulsar
>> > Protocol Handler.
>> >
>> > Thanks
>> > Enrico
>>
>

Re: [DISCUSS] PIP-124: Pulsar Client Shared State API

Posted by Enrico Olivelli <eo...@gmail.com>.
Matteo,

Il Mer 29 Dic 2021, 02:57 Matteo Merli <ma...@gmail.com> ha scritto:

> > * Add an API to the Java client that makes it easier to maintain a
> consistent Share State between instances of an application.
> > * Provide some ready to use recipes, like a simple key-value store
> >
> > It is not a goal to implement a Pulsar backed Database system
>
> While the first use case for Pulsar was indeed to be the
> messaging/replication platform for a distributed database, and it has
> been working in production for many years, I'm not convinced to add
> this level of API as part of the Pulsar client API.
>
> Pulsar API has been designed to be high-level and easy to use (and
> reason about), with in mind the use cases of application developers. I
> don't think that a "storage" level API fits well with the rest of
> abstractions.
>
> > public interface PulsarMap<K,V> extends AutoCloseable {
> > ..
> >  CompletableFuture<V> put(K key, V value)
>
> If all the logic is implemented in the client side, when there are
> multiple clients sharing the same, how can any of them mutate the
> state, since we actually enforce that there is a single exclusive
> producer? Would a user get an error if there's already a different
> client writing?
>
> My impression is that, while looking convenient, a shared Map
> interface is not the best abstraction for either case:
>  * If you're actually building a DB, you will definitely need access
> to the log itself rather than a Map interface
>  * If you want to share some state across multiple processes without
> using a DB, there are many tricky API, consistency and semantic
> problems to solve, many of which are just pushed down to the
> application which will need to be aware and understand them. At that
> point, I would seriously recommend using a DB, or if the question is:
> "I don't want to use an additional external system", then to use the
> BK TableService component.
>

This is usually not a option because the BK TableService does not support
well multi tenancy and also the application would need to connect to the
Bookies (think about configuration, security...)


>
> I think this feature should be best viewed as a recipe, as it doesn't
> depend on or benefits from any internal broker support. If there are
> enough interest and concrete use cases it can be then included later.
>

My initial proposal was to push this to Pulsar Adapters.
I changed the proposal before sending the PIP because I think it very
useful for Protocol Handlers  and in Pulsar IO connectors.

I am totally fine to add this to pulsar-adapters, but I want to see this in
the Pulsar repo and released as part of an official Pulsar recipe.

@Matteo does this sound like a good option to you?

Otherwise we miss the possibility to make it easier for Pulsar users to
leverage this power of Pulsar.

In Pravega you have State Synchronizers and they are a great foundational
API and we are missing something like that in Pulsar.

Enrico



> --
> Matteo Merli
> <ma...@gmail.com>
>
> On Fri, Dec 24, 2021 at 1:53 AM Enrico Olivelli <eo...@gmail.com>
> wrote:
> >
> > Hello everyone,
> > I want to start a discussion about PIP-124 Pulsar Client Shared  State
> API
> >
> > This is the PIP document
> > https://github.com/apache/pulsar/issues/13490
> >
> > This is a demo implementation (a proof-of-concept):
> > https://github.com/eolivelli/pulsar-shared-state-manager
> >
> > Please take a look and share your thoughts
> >
> > I believe that this will unlock the potential of the Exclusive
> > Producer and it will also make easier the life of many developers who
> > are using Pulsar and need some API to share configuration, metadata,
> > or any simple key-value data structure without adding a Database or
> > other components to their library, Pulsar IO connector or Pulsar
> > Protocol Handler.
> >
> > Thanks
> > Enrico
>

Re: [DISCUSS] PIP-124: Pulsar Client Shared State API

Posted by Matteo Merli <ma...@gmail.com>.
> * Add an API to the Java client that makes it easier to maintain a consistent Share State between instances of an application.
> * Provide some ready to use recipes, like a simple key-value store
>
> It is not a goal to implement a Pulsar backed Database system

While the first use case for Pulsar was indeed to be the
messaging/replication platform for a distributed database, and it has
been working in production for many years, I'm not convinced to add
this level of API as part of the Pulsar client API.

Pulsar API has been designed to be high-level and easy to use (and
reason about), with in mind the use cases of application developers. I
don't think that a "storage" level API fits well with the rest of
abstractions.

> public interface PulsarMap<K,V> extends AutoCloseable {
> ..
>  CompletableFuture<V> put(K key, V value)

If all the logic is implemented in the client side, when there are
multiple clients sharing the same, how can any of them mutate the
state, since we actually enforce that there is a single exclusive
producer? Would a user get an error if there's already a different
client writing?

My impression is that, while looking convenient, a shared Map
interface is not the best abstraction for either case:
 * If you're actually building a DB, you will definitely need access
to the log itself rather than a Map interface
 * If you want to share some state across multiple processes without
using a DB, there are many tricky API, consistency and semantic
problems to solve, many of which are just pushed down to the
application which will need to be aware and understand them. At that
point, I would seriously recommend using a DB, or if the question is:
"I don't want to use an additional external system", then to use the
BK TableService component.


I think this feature should be best viewed as a recipe, as it doesn't
depend on or benefits from any internal broker support. If there are
enough interest and concrete use cases it can be then included later.

--
Matteo Merli
<ma...@gmail.com>

On Fri, Dec 24, 2021 at 1:53 AM Enrico Olivelli <eo...@gmail.com> wrote:
>
> Hello everyone,
> I want to start a discussion about PIP-124 Pulsar Client Shared  State API
>
> This is the PIP document
> https://github.com/apache/pulsar/issues/13490
>
> This is a demo implementation (a proof-of-concept):
> https://github.com/eolivelli/pulsar-shared-state-manager
>
> Please take a look and share your thoughts
>
> I believe that this will unlock the potential of the Exclusive
> Producer and it will also make easier the life of many developers who
> are using Pulsar and need some API to share configuration, metadata,
> or any simple key-value data structure without adding a Database or
> other components to their library, Pulsar IO connector or Pulsar
> Protocol Handler.
>
> Thanks
> Enrico

Re: [DISCUSS] PIP-124: Pulsar Client Shared State API

Posted by Haiting Jiang <ji...@apache.org>.
+1, Great feature.

> It is not a goal to implement a Pulsar backed Database system

But this does look like a database system. I believe there would be some limitations here, either in availability, consistency, performance, storage cost, etc.
So I think we can provide more information about disadvantages to guide user's choice.


On 2021/12/24 09:53:37 Enrico Olivelli wrote:
> Hello everyone,
> I want to start a discussion about PIP-124 Pulsar Client Shared  State API
> 
> This is the PIP document
> https://github.com/apache/pulsar/issues/13490
> 
> This is a demo implementation (a proof-of-concept):
> https://github.com/eolivelli/pulsar-shared-state-manager
> 
> Please take a look and share your thoughts
> 
> I believe that this will unlock the potential of the Exclusive
> Producer and it will also make easier the life of many developers who
> are using Pulsar and need some API to share configuration, metadata,
> or any simple key-value data structure without adding a Database or
> other components to their library, Pulsar IO connector or Pulsar
> Protocol Handler.
> 
> Thanks
> Enrico
>