Hi,
Can we clarify how the customer can deup the following duplicate chunked
msgs?
Lets say we have dedup topic here,
// producer sends a chunked msg, m1
s1, c0
s1, c1
s1, c2 // complete
// producer oom and restarted without updating the sequence id to 2
// producer resend the msg(same content)
Hi community,
After internal discussions and agreement with Penghui and Zike, we've
come to a consensus that a small amount of dirty data within the topic
can be tolerated. As a result, this proposal has lost its
significance. We are now closing this discussion. Thanks to all
participants for thei
Hi Penghui,
>From my understanding.
>The message deduplication should only check the last chunk of the message.
Yes, I agree. If we only check the first chunk, the third chunk will be dropped.
Thanks,
Xiangying
On Mon, Aug 28, 2023 at 10:08 AM PengHui Li wrote:
>
> Hi Xiangying,
>
> Thanks for
Hi Xiangying,
Thanks for driving the proposal.
>From my understanding.
The message deduplication should only check the last chunk of the message.
It doesn't need to care about whether each chunk is duplicated.
The client side should handle issues like duplicated chunks.
For the example that you h
Share more information: Even for versions before 3.0.0, the approach
doesn't assemble chunks 3, 4, and 5 together.
Please note this line of code:
```java
chunkedMsgCtx = chunkedMessagesMap.computeIfAbsent(msgMetadata.getUuid(),
(key) -> ChunkedMessageCtx.get(totalChunks,
chunk
Hi Zike,
PR [0] has already fixed this bug and won't introduce compatibility issues.
PR [1] is unnecessary and can be closed. However, I still greatly
appreciate the information you provided.
[0] https://github.com/apache/pulsar/pull/20948
[1] https://github.com/apache/pulsar/pull/21070
On Sat,
> Hi Zike
You can see the processMessageChunk method of the ConsumerImpl.
Ah. That seems like a regression bug introduced by
https://github.com/apache/pulsar/pull/18511. I have pushed a PR to fix
it: https://github.com/apache/pulsar/pull/21070
For the behavior before Pulsar 3.0.0. The consumer sh
>> Consumer receive:
>1. SequenceID: 0, ChunkID: 0
>2. SequenceID: 0, ChunkID: 1
>3. SequenceID: 0, ChunkID: 0 // chunk ID out of order. Release this
>chunk and recycle its `chunkedMsgCtx`.
>4. SequenceID: 0, ChunkID: 1 // chunkedMsgCtx == null Release it.
>5. SequenceID: 0, ChunkID: 2 // chunked
>Regarding this comment, can you explain how the consumer only receives m1?
>Here, m1's and m2's uuid and msgId will be different(if we suffix with a
>chunkingSessionId), although the sequence ID is the same.
They have the same sequence ID, so the first chunk will be dropped
because the sequence I
> Consumer receive:
1. SequenceID: 0, ChunkID: 0
2. SequenceID: 0, ChunkID: 1
3. SequenceID: 0, ChunkID: 0 // chunk ID out of order. Release this
chunk and recycle its `chunkedMsgCtx`.
4. SequenceID: 0, ChunkID: 1 // chunkedMsgCtx == null Release it.
5. SequenceID: 0, ChunkID: 2 // chunkedMsgCtx
> In this case, the consumer only can receive m1.
Regarding this comment, can you explain how the consumer only receives m1?
Here, m1's and m2's uuid and msgId will be different(if we suffix with a
chunkingSessionId), although the sequence id is the same.
> If we throw an exception when users use
Hi Heesung,
Maybe we only need to maintain the last chunk ID in a map.
Map map1.
And we already have a map maintaining the last sequence ID.
Map map2.
If we do not throw an exception when users use the same sequence to
send the message.
For any incoming msg, m :
chunk ID = -1;
If m is a chunk me
However, what If the producer jvm gets restarted after the broker persists
the m1 (but before updating their sequence id in their persistence
storage), and the producer is trying to resend the same msg(so m2) with the
same sequence id after restarting?
On Fri, Aug 25, 2023 at 8:22 PM Xiangying
Don't we still need the broker dedup logic for the above case?
Then, probably brokers need to track the following.
Map
// additionally track
Map
ChunkingContext{
uuid,
numChunks,
lastChunkId
}
The chunked msg dedup logic might be like:
For any incoming chunked msg, m :
If m.currentSeqid < Last
Hi Heesung,
In this case, the consumer only can receive m1.
But it has the same content as the previous case: What should we do if
the user sends messages with the sequence ID that was used previously?
I am afraid to introduce the incompatibility in this case, so I only
added a warning log in th
Actually, can we think about this case too?
What happens if the cx sends the same chunked msg with the same seq id when
dedup is enabled?
// user send a chunked msg, m1
s1, c0
s1, c1
s1, c2 // complete
// user resend the duplicate msg, m2
s1, c0
s1, c1
s1, c2 //complete
Do consumers receive m1
Hi Heesung,
>I think this means, for the PIP, the broker side's chunk deduplication.
>I think brokers probably need to track map to dedup
What is the significance of doing this?
My understanding is that if the client crashes and restarts after
sending half of a chunk message and then it resends t
I think this means, for the PIP, the broker side's chunk deduplication.
I think brokers probably need to track map to dedup
chunks on the broker side.
On Fri, Aug 25, 2023 at 6:16 PM Xiangying Meng wrote:
> Hi Heesung
>
> It is a good point.
> Assume the producer application jvm restarts in t
Hi Heesung
It is a good point.
Assume the producer application jvm restarts in the middle of chunking and
resends the message chunks from the beginning with the previous sequence id.
For the previous version, it should be:
Producer send:
1. SequenceID: 0, ChunkID: 0
2. SequenceID: 0, ChunkID: 1
Hi, I meant
What if the producer application jvm restarts in the middle of chunking and
resends the message chunks from the beginning with the previous sequence id?
On Fri, Aug 25, 2023 at 5:15 PM Xiangying Meng wrote:
> Hi Heesung
>
> It is a good idea to cover this incompatibility case if t
Hi Heesung
It is a good idea to cover this incompatibility case if the producer
splits the chunk message again when retrying.
But in fact, the producer only resents the chunks that are assembled
to `OpSendMsg` instead of splitting the chunk message again.
So, there is no incompatibility issue of
>> I think brokers can track the last chunkMaxMessageSize for each producer.
> Using different chunkMaxMessageSize is just one of the aspects. In
PIP-132 [0], we have included the message metadata size when checking
maxMessageSize.The message metadata can be changed after splitting the
chunks. We
Hi Zike,
>How would this happen to get two duplicated and consecutive ChunkID-1
>messages? The producer should guarantee to retry the whole chunked
>messages instead of some parts of the chunks.
If the producer guarantees to retry the whole chunked messages instead
of some parts of the chunks,
Wh
HI xiangying
> The rewind operation is seen in the test log.
That seems weird. Not sure if this rewind is related to the chunk consuming.
> 1. SequenceID: 0, ChunkID: 0
2. SequenceID: 0, ChunkID: 1
3. SequenceID: 0, ChunkID: 1
4. SequenceID: 0, ChunkID: 2
Such four chunks cannot be processed cor
>IIUC, this may change the existing behavior and may introduce inconsistencies.
>Suppose that we have a large message with 3 chunks. But the producer
>crashes and resends the message after sending the chunk-1. It will
>send a total of 5 messages to the Pulsar topic:
>
>1. SequenceID: 0, ChunkID: 0
>> This solution also cannot solve the out-of-order messages inside the
>>chunks. For example, the above five messages will still be persisted.
>The consumer already handles this case. The above 5 messages will all
>be persisted but the consumer will skip message 1 and 2.
>For messages 3, 4, and 5.
Hi Heesung,
> I believe in this PIP "similar to the existing "sequence ID map",
to facilitate effective filtering" actually means tracking the last
chunkId(not all chunk ids) on the broker side.
With this simple solution, I think we don't need to track the
(sequenceID, chunkID) on the broker side
Hi, xiangying
> it will find that the message
is out of order and rewind the cursor. Loop this operation, and
discard this message after it expires instead of assembling 3, 4, 5
into a message.
Could you point out where the implementation for this? From my
understanding, there should not be any
> 1. SequenceID: 0, ChunkID: 0
> 2. SequenceID: 0, ChunkID: 1
> 3. SequenceID: 0, ChunkID: 0
> 4. SequenceID: 0, ChunkID: 1
> 5. SequenceID: 0, ChunkID: 2
> For the existing behavior, the consumer assembles
> messages 3,4,5 into
> the original large message. But the changes brought
> about by this
> I think a simple better approach is to only check the deduplication
for the last chunk of the large message. The consumer only gets the
whole message after receiving the last chunk. We don't need to check
the deduplication for all previous chunks. Also by doing this we only
need bug fixes, we don
Hi Zike
Thank you for your attention.
>For the existing behavior, the consumer assembles messages 3,4,5 into the
>original large message. But the changes brought about by this PIP will cause
>the consumer to use messages 1,2,5 for assembly. There is no guarantee that
>the The producer will split
Hi, xiangying,
Thanks for your PIP.
IIUC, this may change the existing behavior and may introduce inconsistencies.
Suppose that we have a large message with 3 chunks. But the producer
crashes and resends the message after sending the chunk-1. It will
send a total of 5 messages to the Pulsar topic
Dear Community,
I hope this email finds you well. I'd like to address an important
issue related to Apache Pulsar and discuss a solution I've proposed on
GitHub. The problem pertains to the handling of Chunk Messages after
enabling deduplication.
In the current version of Apache Pulsar, all chunk
33 matches
Mail list logo