Re: [DISCUSS] Consumer reconnection causes repeated consumption messages

2022-09-20 Thread 丛搏
Hi Haiting, In this case, the user wants to reset to the first message reconsume all the messages, but the user can't receive a message anymore, the `admin.topics().resetCursor(topic, subName, MessageId.earliest)` may use HTTP to call, so it is wrong behavior. Thanks, Bo Haiting Jiang 于2022年9月1

Re: [DISCUSS] Consumer reconnection causes repeated consumption messages

2022-09-14 Thread Haiting Jiang
Hi Bo, > In this case, the consumer can not receive any message again in the > end. We have to fix it because it causes a loss of messages. > I think we need to redefine the use of cumulative acks and not just > solve the problem of receiving messages repeatedly. > There seems to be nothing wrong

Re: [DISCUSS] Consumer reconnection causes repeated consumption messages

2022-09-14 Thread 丛搏
Hi Haiting, Michael , I tested an example, and there may be some problems when using the sub reset cursor and cumulative ack at the same time. e.g ``` Message message = null; for (int i = 0; i < 3; i++) { message = consumer.receive(); } admin.topics().resetCursor(topic, subName, MessageId.ea

Re: [DISCUSS] Consumer reconnection causes repeated consumption messages

2022-09-09 Thread Xiangying Meng
Hi, Bo I totally agree with this approach. Suppose we now implement deduplication for common messages on the client side. In that case, there is no need to add other logic that may cause API break-changes to guarantee transaction exactly-once semantics. Yours, Xiangying On Fri, Sep 9, 2022 at 10:1

Re: [DISCUSS] Consumer reconnection causes repeated consumption messages

2022-09-09 Thread 丛搏
Hi all, you can see the problem in a google doc and comments. google doc link: https://docs.google.com/document/d/1J1xGcj8YORrdlCa_XDt28uV0TMp03gSmX_z43ZRhwZo/edit?usp=sharing Thanks! Bo 丛搏 于2022年9月8日周四 10:55写道: > > Hello, Pulsar community: > > > Now the consumer does not filter messages that

Re: [DISCUSS] Consumer reconnection causes repeated consumption messages

2022-09-09 Thread 丛搏
Hi Michael you are right, now the current handover is very blunt. We can implement a flexible and efficient solution by changing the protocol in a 3.0 enhancement. But we also need this optimization in 2.0, but this optimization should not be too complicated at best. Thanks, Bo Michael Marshall

Re: [DISCUSS] Consumer reconnection causes repeated consumption messages

2022-09-09 Thread 丛搏
Hi Haiting, good point! we need to pay attention to all reset cursor operations. I think this is a configurable optimization because some users don't call `void redeliverUnacknowledgedMessages().` This optimization is a break-change operation. I will find all the operations to reset the cursor, an

Re: [DISCUSS] Consumer reconnection causes repeated consumption messages

2022-09-08 Thread Michael Marshall
I agree that reducing unnecessary duplicates is a good goal. For the topic unloading case, it might help to think about how we can improve the protocol. The current handover is very blunt. A new solution could focus on decreasing duplicate message delivery while also focusing on decreasing time wh

Re: [DISCUSS] Consumer reconnection causes repeated consumption messages

2022-09-08 Thread Haiting Jiang
Hi Bo, Overall it makes sense to me. It is basically the same as broker side deduplication mechanism when producing messages, which uses `sequenceId`. In your case, messageId is used for deduplication. It should work as long as the received messageId increases monotonically. So we should be caref

Re: [DISCUSS] Consumer reconnection causes repeated consumption messages

2022-09-08 Thread 丛搏
Hi Haiting When using cumulative ack, we can save the maximum received MessageId on the consumer client side to filter the message duplication caused by reconnection, if the consumer client process restarts the maximum received MessageId will not exist in the consumer client. This requires the use

Re: [DISCUSS] Consumer reconnection causes repeated consumption messages

2022-09-07 Thread Haiting Jiang
>From the user's perspective, I think we should always avoid delivering repeated messages. But can the broker tell if the reconnection is caused by topic unloading or consumer client process restarting? For the latter case, the message should be redelivered, it's the whole point of user explicit ac

[DISCUSS] Consumer reconnection causes repeated consumption messages

2022-09-07 Thread 丛搏
Hello, Pulsar community: Now the consumer does not filter messages that have already been consumed. After consumer reconnection, the broker will dispatch messages to the consumer from the markDeletePosition. In Failover and Exclusive subscription type, all messages in a topic will be dispatched t