merlimat commented on issue #2400: PIP-22: Dead Letter Topic
URL: https://github.com/apache/incubator-pulsar/pull/2400#issuecomment-415917621
 
 
   @codelipenghui @sijie 
   
   I'm still convinced it's much easier to approach this problem in the
   client side 😄 
   
   Other than the scalability concerns and the complexity in changing the
   dispatcher code, I think we should also consider:
   
    * If the broker is re-publishing, we would need to have some complex
      way to do authorization.  Consumer has its own credentials, but
      broker will have to create a producer using the broker credentials,
      after having verified that consumer also has permission to publish
      on the DLQ. Worse, if we revoke the permission on the DLQ topic, we
      would also have to figure that out in broker. Moving the publishing
      to consumer, will make that problem go away: consumer will use the
      same credentials when re-publishing and it will work same as a
      regular producer.
   
    * If the consumer has a schema, it might be desirable to keep the
      schema when republishing. Right now, the producer app needs to have
      the Java pojo to create a schema enabled producer, but broker won't
      have the Pojo definition.
   
    * In the current implementation, it's not handling the fact that
      different consumers on the same subscriptions could have different
      DLQ settings. Eg: `consumer-a`: 10 retries and publish on `dlq-a`
      and `consumer-a` 5 retries with republish on `dlq-b`. If done on
      broker side, broker would have to somehow deal with these
      inconsistencies.  If the implementation is done on client library,
      these inconsistencies will be fine: the broker just needs to keep
      (and communicate) the current count of re-deliveries. Each consumer
      will then take action based on its local configuration.
   
   > a) doing DLQ in client actually make things a bit more
   > complicated. because the redelivery count is tracked at
   > broker, but the publishing happens at the client. Things like
   > configurations/policies are separated in different places,
   > which can result in inconsistent behaviors, which make
   > debugging much harder when issue occurs.
   
   The current PR is anyway configuring the DLQ in client API. If we
   were to configure the DLQ as a server side policy we would probably
   need to have fine-grained policies per-topic (while currently these
   are only at the namespace level, except for authorization which is
   both at namespace and topic level). For DLQ it would even be at
   the subscription level.
   
   > b) doing DLQ in client means the same logic has to be implemented in
   >  different language client. The implementation can quickly become
   >  inconsistent between different language client.
   
   This I agree. It would require to do implementation in both Java and C++
   client (plus the wrappers). My hope is that the client implementation
   would be not be very complicated.
   
   > c) regarding the resource concerns, I don't think the resource usage
   >  is different if it is still broker tracking the redelivery
   >  count. unless there is something I am missing.
   
   Well, the difference here would be that broker just need to keep only
   a counter, inside a map there already exist.
   
   If the broker is doing the publishes, then it needs to create and cache
   producers for all the DLQ topics. If each consumer has its own producers, 
   this could greatly increase the memory consumption in broker and we would
   have to carefully tune the producer queues to make sure we don't get out 
   of direct memory in the worst case scenarios.
   
   > d) doing DLQ in broker allows us doing optimization in future, like
   >  persisting delivery count to cursors or a retry queue, without
   >  changing client behaviors. especially if we want to support
   >  consistent behavior across different language clients.
   
   Optmizations like storing the counter would possible in either case.
   
   > "negative acks" is a good thing to have. However I don't think DLQ
   >  has to depend on "negative acks". because the current "negative
   >  acks" pulsar has is ack timeout. for DLQ, it doesn't really matter
   >  it is a ack timeout or explicit ack timeout. We don't really need to
   >  tie things together. If DLQ goes first, we still can introduce
   >  "negative acks" later, and it will not affect DLQ logic.
   
   Sure, we can defer "negative acks" changes and just focus on current
   re-delivery functionality.
   
   > I think the simplest thing is just let user configure message
   >  retention in DLQ topic. That is what typically cloud providers
   >  offer. Not sure we need to get into the business of managing
   >  subscriptions for DLQ.
   
   Again, the problem lies with policies which are currently namespace
   wide.
   
   I believe the best compromise solution we can adopt initially is to
   leave the freedom to the user. The documenation will clearly specify
   that you need to make sure the data is retained in the DLQ and present
   few alternatives, eg:
   
   ```
   pulsar-admin topics create-subscribtion my-topic-dlq --subscription dlq
   ```
   
   With this command, one can make sure that every message pushed to the
   `my-topic-dql` will be retained (indefinitely) and it will be
   accessible through the subscription named `dlq`.
   
   
   
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

Reply via email to