Sijie,

> If the lazy-loading approach sounds attractive to you and you like it,
> maybe the next step is to update the PIP, what do you think?

I think so too. I will update the PIP after discussing the authn/authz issue.

Regards,
-- 
Yuri Mizushima
yumiz...@yahoo-corp.jp
 

"Sijie Guo" <guosi...@gmail.com> wrote:

    Hi Yuri,

    Regarding the authn/authz issue, @Matteo Merli <mme...@apache.org> can
    probably chime in more about that part.

    If the lazy-loading approach sounds attractive to you and you like it,
    maybe the next step is to update the PIP, what do you think?

    - Sijie

    On Mon, Feb 8, 2021 at 6:57 PM Yuri Mizushima <yumiz...@yahoo-corp.jp>
    wrote:

    > Michael,
    >
    > Thank you for your comment!
    >
    > > Which Pulsar Clients will benefit from this proposal?
    > I think that this proposal will be useful to any clients.
    > In my schedule, if this proposal is accepted then I will implement this
    > feature to Java client.
    > If needed, then implement same feature to other clients such as C++, Go,
    > etc.
    >
    > Regards,
    > --
    > Yuri Mizushima
    > yumiz...@yahoo-corp.jp
    >
    >
    > "Michael Marshall" <mikemars...@gmail.com> wrote:
    >
    >     Hi Yuri and Sijie,
    >
    >     I definitely like the idea of lazily creating producers as well as
    > introducing a way to provide custom routing logic.
    >
    >     Which Pulsar Clients will benefit from this proposal? I’d love to see
    > this feature in the go client.
    >
    >     Thanks,
    >     Michael Marshall
    >
    >     > On Feb 7, 2021, at 9:53 PM, Yuri Mizushima <yumiz...@yahoo-corp.jp>
    > wrote:
    >     >
    >     > Sijie,
    >     >
    >     > Thank you for sharing!
    >     >
    >     > First, I considered your suggestion.
    >     > I think these implementations sound good.
    >     >
    >     > I think we should consider the State of partitioned producer: Ready,
    > Connecting, etc.
    >     > Currently, partitioned producer gets "Ready" only when all producers
    > connect to Broker correctly.
    >     >
    > 
https://github.com/apache/pulsar/blob/fa41d02bebfd841767846240f3ae574047f118f0/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PartitionedProducerImpl.java#L146
    >     > It seems that we should change meaning of state (or change handling)
    > if we introduce the lazy-load feature.
    >     > To guarantee the message ordering (e.g. using partitionKey),
    > partitioned producer should stop (or don't send messages to be routed to
    > unavailable partition) when producer can't connect to one of partition.
    >     >
    >     > Secondly, I considered Matteo's comments.
    >     > I couldn't understand well about issue of authn/authz. Please tell
    > me more detail.
    >     >
    >     > I wrote "connection" as number of producers which connect to broker.
    > Also, TCP connections between partitioned producer and broker will be less
    > than or equal to current in some cases. I'll show a case below.
    >     >
    >     > Suppose
    >     > * cluster has Broker0, 1, 2
    >     > * partitioned topic has 5 partitions
    >     > * limit conf is 3 partitions
    >     > * loadbalance partitions as below
    >     > - Broker0: partition-0, partition-1
    >     > - Broker1: partition-2
    >     > - Broker2: partition-3, partition-4
    >     >
    >     > Currently, client will create 3 connections (Broker0, 1, 2). If
    > client uses limit conf and elects partitions such as [0, 1, 2], then 
client
    > will create 2 connections (Broker0, 1). Of course, if client elects
    > partitions such as [0, 2, 3], then client will still create 3 connections.
    >     >
    >     > I'd like to decrease number of producers. I think that resources of
    > broker will be improved slightly by this feature because broker has list 
of
    > producers by some classes such as ServerCnx, AbstractTopic.
    >     >
    > 
https://github.com/apache/pulsar/blob/fa41d02bebfd841767846240f3ae574047f118f0/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java#L1096-L1097
    >     >
    > 
https://github.com/apache/pulsar/blob/fa41d02bebfd841767846240f3ae574047f118f0/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java#L577
    >     >
    >     > In my case, unspecified number of producers will connect to the same
    > partitioned topic with different rate. We need to set the number of
    > partitions according to the high-rate producer.
    >     > However, on the other hand, this number is excessively large for
    > low-rate producers.
    >     > I want to reduce such redundant producers for resource efficiency.
    >     >
    >     > Regards,
    >     > --
    >     > Yuri Mizushima
    >     > yumiz...@yahoo-corp.jp
    >     >
    >     >
    >     > "Sijie Guo" <guosi...@gmail.com> wrote:
    >     >
    >     >  Hi Yuri,
    >     >
    >     >  In today's community meeting, Matteo shared some of his thoughts
    > about this
    >     >  PIP.
    >     >
    >     >  You can find some meeting notes here:
    >     >
    > 
https://docs.google.com/document/d/19dXkVXeU2q_nHmkG8zURjKnYlvD96TbKf5KjYyASsOE/edit#bookmark=id.rezbt4xmjxpz
    >     >
    >     >  Matteo can also chime in as well.
    >     >
    >     >  - Sijie
    >     >
    >     >>  On Sun, Jan 31, 2021 at 7:21 PM Yuri Mizushima <
    > yumiz...@yahoo-corp.jp>
    >     >>  wrote:
    >     >> Sijie,
    >     >> Thank you for your reply!
    >     >> I'll check it.
    >     >> Regards,
    >     >> --
    >     >> Yuri Mizushima
    >     >> yumiz...@yahoo-corp.jp
    >     >> "Sijie Guo" <guosi...@gmail.com> wrote:
    >     >>  Yuri,
    >     >>  Thank you for bringing this up! This is a super helpful proposal!
    >     >>  The problem is very similar to what an RPC framework (like 
Finagle)
    >     >> with
    >     >>  client-side load balancing has.
    >     >>  An RPC framework with a client-side load-balancing mechanism needs
    > to
    >     >> send
    >     >>  requests across multiple nodes. If you have an RPC service that 
has
    >     >>  thousands of nodes, there are thousands of clients connecting to
    > that
    >     >> RPC
    >     >>  service. How to reduce the connections and how to effectively load
    >     >> balance
    >     >>  requests across thousands of nodes are the problems that a
    > client-side
    >     >>  loading technology needs to solve. If you think about "partition"
    > as
    >     >> "node"
    >     >>  and "partitioned producer" as "RPC client", the problem is exactly
    > the
    >     >>  same. Finagle (the Twitter RPC framework) has implemented a lot of
    >     >> client-side
    >     >>  load-balancing algorithms
    >     >>  <
    > https://twitter.github.io/finagle/guide/Clients.html#load-balancing>
    >     >> and
    >     >>  there are some great articles that you can reference
    >     >>  <
    >     >>
    > 
https://blog.twitter.com/engineering/en_us/topics/infrastructure/2019/daperture-load-balancer.html
    >     >>  .
    >     >>  I agree with the direction of introducing a mechanism to reduce 
the
    >     >> number
    >     >>  of producers in a partitioned topic producer. However, I have a
    > concern
    >     >>  about introducing `.numPartitionsLimit(10)` directly to the
    > producer
    >     >>  builder. It limits the possibility to implement different
    > algorithms on
    >     >>  selecting partitions.
    >     >>  So instead of directly implementing the logic within the
    > partitioned
    >     >> topic
    >     >>  producer, I think the proposal can be broken into two parts:
    >     >>  1) Introduce some kind of lazy-loading mechanism in the 
partitioned
    >     >>  producer to initialize the producers for partitions lazily. I.e.,
    > only
    >     >>  initialize a producer when the message router selects a partition.
    >     >>  2) Implement a message router that only selects one or N
    > partitions.
    >     >>  In this way, the partitioned producer is only responsible for
    > managing
    >     >> a
    >     >>  collection of producers, and the message router is responsible for
    >     >>  selecting the partitions. This allows people to be able to
    > implement
    >     >>  different message routers. We can even adopt the client-side load
    >     >> balancing
    >     >>  algorithms from Finagle.
    >     >>  Thanks,
    >     >>  Sijie
    >     >>  On Wed, Jan 27, 2021 at 7:18 PM Yuri Mizushima <
    > yumiz...@yahoo-corp.jp
    >     >>  wrote:
    >     >>> I notice that PIP-78 has already assigned to another issue.
    >     >>
    > 
https://mail-archives.apache.org/mod_mbox/pulsar-dev/202101.mbox/%3CCAG%3DTQOrPH49v9ToDE_aeQzEiDC%2BEgSR61ERoqanpWfQGvEB_Vw%40mail.gmail.com%3E
    >     >>> So, I'll change the PIP number to 79.
    >     >>
    > 
https://github.com/apache/pulsar/wiki/PIP-79%3A-Reduce-redundant-producers-from-partitioned-producer
    >     >>> Regards,
    >     >>> --
    >     >>> Yuri Mizushima
    >     >>> yumiz...@yahoo-corp.jp
    >     >>> "Yuri Mizushima" <yumiz...@yahoo-corp.jp> wrote:
    >     >>>  Dear Pulsar community,
    >     >>>  When partitioned producer connects to partitioned topic,
    >     >>>  sometimes doesn't need to connect to all of partitions depending
    >     >> on
    >     >>> rate, routing mode, etc.
    >     >>>  So, I drafted a PIP about reducing redundant producers from
    >     >>> partitioned producer.
    >     >>>  I'd like to use system resources (e.g. connections between
    >     >> Client and
    >     >>> Broker, memory usage of both Client and Broker)
    >     >>>  more efficiently by this feature.
    >     >>
    > 
https://github.com/apache/pulsar/wiki/PIP-78%3A-Reduce-redundant-producers-from-partitioned-producer
    >     >>>  Feel free to ask me any questions or suggestions, etc.
    >     >>>  Best regards,
    >     >>>  --
    >     >>>  Yuri Mizushima
    >     >>>  yumiz...@yahoo-corp.jp
    >
    >

Reply via email to