Hi all,

 Here is a suggest (PIP) on consumers can set custom retry delay when 
consumer fail.




------------




PIP : Support Consumers  Set Custom Retry Delay 
## Motivation


For many online business systems, various exceptions usually occur in business 
logic processing, so the message needs to be re-consumed, but users hope that 
this delay time can be controlled flexibly. The current user's processing 
method is usually to send this message to a special retry topic, because 
production can specify any delay, so consumers subscribe the business topic and 
retry topic at the same time. I think this logic can be supported by pulsar 
itself, making it easier for users to use, and it looks like this is a very 
common requirement.


## Proposed changes


This change can be supported on the client side,  need to add a set of 
interfaces to org.apache.pulsar.client.api.Consumer
```java
void reconsumeLater(Message<?&gt; message, long delayTime, TimeUnit unit) 
throws PulsarClientException;
CompletableFuture<Void&gt; reconsumeLaterAsync(Message<?&gt; message, long 
delayTime, TimeUnit unit);
CompletableFuture<Void&gt; reconsumeLaterAsync(Messages<?&gt; messages, int 
delayLevel);
&nbsp;CompletableFuture<Void&gt; reconsumeLaterCumulativeAsync(Message<?&gt; 
message, long delayTime, TimeUnit unit);
```
DeadLetterPolicy add retry topic&nbsp;
```java
public class DeadLetterPolicy {


&nbsp; &nbsp; /**
&nbsp; &nbsp; &nbsp;* Maximum number of times that a message will be 
redelivered before being sent to the dead letter queue.
&nbsp; &nbsp; &nbsp;*/
&nbsp; &nbsp; private int maxRedeliverCount;


&nbsp; &nbsp; /**
&nbsp; &nbsp; &nbsp;* Name of the retry topic where the failing messages will 
be sent.
&nbsp; &nbsp; &nbsp;*/
&nbsp; &nbsp; private String retryLetterTopic;


&nbsp; &nbsp; /**
&nbsp; &nbsp; &nbsp;* Name of the dead topic where the failing messages will be 
sent.
&nbsp; &nbsp; &nbsp;*/
&nbsp; &nbsp; private String deadLetterTopic;


}


```
org.apache.pulsar.client.impl.ConsumerImpl add a retry producer
```java
&nbsp; private volatile Producer<T&gt; deadLetterProducer;


&nbsp; private volatile Producer<T&gt; retryLetterProducer;
```
Can specify whether to enable retry when creating a consumer??default 
unenable&nbsp;
```java
&nbsp; &nbsp; @Override
&nbsp; &nbsp; public ConsumerBuilder<T&gt; enableRetry(boolean retryEnable) {
&nbsp; &nbsp; &nbsp; &nbsp; conf.setRetryEnable(retryEnable);
&nbsp; &nbsp; &nbsp; &nbsp; return this;
&nbsp; &nbsp; }
```


------------




Thanks,


Dezhi

Reply via email to