Skip to content

[Java Client] Fix wrong behavior of deduplication for key based batching #15413

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Conversation

BewareMyPower
Copy link
Contributor

Motivation

Currently message deduplication doesn't work well for key based
batching. First, the key based batch container doesn't update the
lastSequenceIdPushed. So a batch could contain both duplicated and not
duplicated messages. Second, when createOpSendMsgs is called, the
OpSendMsg objects are sorted by the lowest sequence ids, and the
highest sequence id is not set. If a batch contains sequence id 0,1,2,
then the message with sequence id 1 or 2 won't be dropped.

Modifications

  • Refactor the key based batch container that the
    BatchMessageContainerImpl is reused instead of maintaining a
    KeyedBatch class.
  • When createOpSendMsgs is called, clear the highest sequence id field
    and configure the sequence id field with the highest sequence id to fix
    the second issue described before.
  • Add testKeyBasedBatchingOrder to show and verify the current
    behavior.
  • Add test for key based batching into
    testProducerDeduplicationWithDiscontinuousSequenceId to verify
    lastSlastSequenceIdPushed is updated correctly.

Documentation

Check the box below or label this PR directly.

Need to update docs?

  • doc-required
    (Your PR needs to update docs and you will update later)

  • no-need-doc
    (Please explain why)

  • doc
    (Your PR contains doc changes)

  • doc-added
    (Docs have been already added)

@BewareMyPower BewareMyPower added type/bug The PR fixed a bug or issue reported a bug component/client-java labels May 2, 2022
@BewareMyPower BewareMyPower added this to the 2.11.0 milestone May 2, 2022
@BewareMyPower BewareMyPower self-assigned this May 2, 2022
@github-actions github-actions bot added the doc-not-needed Your PR changes do not impact docs label May 2, 2022
@BewareMyPower BewareMyPower added release/2.9.3 release/2.8.4 release/2.10.1 and removed doc-not-needed Your PR changes do not impact docs labels May 2, 2022
@github-actions
Copy link

github-actions bot commented May 2, 2022

@BewareMyPower:Thanks for your contribution. For this PR, do we need to update docs?
(The PR template contains info about doc, which helps others know more about the changes. Can you provide doc-related info in this and future PR descriptions? Thanks)

@BewareMyPower
Copy link
Contributor Author

Though this PR changed the current behavior that the OpSendMsgs are now sorted by the highest sequence id instead of the lowest sequence id. I think it should be a bug fix rather than a feature. So I added the labels for older branches. If anyone worried about this behavior change, feel free to point it out.

@BewareMyPower BewareMyPower added the doc-not-needed Your PR changes do not impact docs label May 2, 2022
@github-actions
Copy link

github-actions bot commented May 2, 2022

@BewareMyPower:Thanks for providing doc info!

@poorbarcode
Copy link
Contributor

poorbarcode commented May 2, 2022

Hi @BewareMyPower. I think it will cause other problems.

  • I'm trying to understand what you're trying to say: when the code is executed like this, the message which is sequence_3 was repeated.
// batch 1
send [seq_1, seq_3]
// batch 2
send [seq_2, seq_3]
  • But after the code changes, when the code is executed like this, the message which is seq_1 was repeated.
// batch 1
send [seq_1, seq_3]
// batch 2
send [seq_1, seq_5]

I don't know whether I have correctly understood your meaning. If there is any misunderstanding, thank you for telling me.

@BewareMyPower
Copy link
Contributor Author

First, both cases should not be valid. User should be responsible to make the sequence id unique.

What this PR wants to solve is the resend case of an existing message. For example, 4 messages might be split into two groups in key based batching:

  • A: 0, 3 (i.e. messages whose keys are all "A", and the sequence ids are 0 and 3)
  • B: 1, 2

If the client didn't receive the ack receipt of batch A, it might resend batch A (msg-0 and msg-3). There are two possible cases:

  1. batch A has been persisted -> broker should not accept the resend
  2. batch A has not been persisted -> broker should accept the resend

Assuming msg-0 and msg-3 are sent individually (without batch).

For the 1st case, we can verify it easily with the unit test.

    public void test() throws Exception {
        final String topic = "persistent://my-property/my-ns/test";
        admin.namespaces().setDeduplicationStatus("my-property/my-ns", true);

        final Producer<String> producer = pulsarClient.newProducer(Schema.STRING)
                .topic(topic)
                .batcherBuilder(BatcherBuilder.KEY_BASED)
                .create();
        final List<TypedMessageBuilder<String>> messagesToSend = new ArrayList<>();
        messagesToSend.add(producer.newMessage().value("msg-0").key("A").sequenceId(0));
        messagesToSend.add(producer.newMessage().value("msg-1").key("B").sequenceId(1));
        messagesToSend.add(producer.newMessage().value("msg-2").key("B").sequenceId(2));
        messagesToSend.add(producer.newMessage().value("msg-3").key("A").sequenceId(3));
        final List<CompletableFuture<MessageId>> futures = new ArrayList<>();
        messagesToSend.forEach(msg -> futures.add(msg.sendAsync()));
        producer.flush(); // flush the messages in the batch container
        final List<MessageId> ids = futures.stream().map(CompletableFuture::join).collect(Collectors.toList());
        // Resend messages in batch A
        ids.add(messagesToSend.get(0).send());
        ids.add(messagesToSend.get(3).send());
        // TODO: print the ids
    }
Before this patch After this patch
msg-0 rejected rejected
msg-3 accepted rejected

Before this patch, the resend of msg-3 is accepted, this behavior must be wrong. Because msg-0 and msg-3 have already been persisted.

For the 2nd case, we can modify the test to:

        final List<CompletableFuture<MessageId>> futures = new ArrayList<>();
        // Assume msg-0 and msg-3 are lost
        futures.add(messagesToSend.get(1).sendAsync());
        futures.add(messagesToSend.get(2).sendAsync());
        producer.flush(); // flush the messages in the batch container
Before this patch After this patch
msg-0 rejected rejected
msg-3 accepted accepted

The behaviors are the same. Though before this patch, msg-3 is accepted because 3 > 1, while after this patch, msg-3 is accepted because 3 > 2.

It's a pity to see msg-0 is rejected as the duplicated message. It's true that this PR cannot solve this problem as well.

However, if we resent msg-0 and msg-3 as a batch,

        // Resend messages in batch A
        futures.add(messagesToSend.get(0).sendAsync());
        futures.add(messagesToSend.get(3).sendAsync());
        final List<MessageId> newIds = futures.stream().map(CompletableFuture::join).collect(Collectors.toList());
Before this patch After this patch
msg-0 rejected Never Completed
msg-3 rejected Never Completed

After this patch, the futures of sendAsync never completed. It might be another bug to fix.

2022-05-03T04:17:20,988+0800 [main] WARN org.apache.pulsar.client.impl.ProducerImpl - Message with sequence id 0 is definitely a duplicate


In short, this PR solves the problem that duplicated messages might be accepted even if the previous messages were persisted successfully.

  • With the default batching builder, the highest sequence id is updated in broker.
  • (Before this patch) With the key based batching builder, the lowest sequence id is updated in broker.

@BewareMyPower
Copy link
Contributor Author

To make it more clear and simple, #5491 introduced the highest sequence id to MessageMetadata so that broker could update the latest sequence id to the sequence id of the sequence id of the latest message in the batch. However, the current implementation of the key based batching never tells the broker the sequence id of the latest message in the batch.

@BewareMyPower
Copy link
Contributor Author

/pulsarbot run-failure-checks

@BewareMyPower BewareMyPower force-pushed the bewaremypower/key-based-batcher-highest-seq-id branch from db6ad4f to dd62fff Compare May 5, 2022 11:57
@codelipenghui
Copy link
Contributor

messagesToSend.add(producer.newMessage().value("msg-0").key("A").sequenceId(0));
messagesToSend.add(producer.newMessage().value("msg-1").key("B").sequenceId(1));
messagesToSend.add(producer.newMessage().value("msg-2").key("B").sequenceId(2));
messagesToSend.add(producer.newMessage().value("msg-3").key("A").sequenceId(3));

What is the expected behavior for the above case?

The deduplication depends on the monotonically increasing sequence ID, but the key-based
batcher will break the rule.

@BewareMyPower
Copy link
Contributor Author

BewareMyPower commented May 6, 2022

messagesToSend.add(producer.newMessage().value("msg-0").key("A").sequenceId(0));
messagesToSend.add(producer.newMessage().value("msg-1").key("B").sequenceId(1));
messagesToSend.add(producer.newMessage().value("msg-2").key("B").sequenceId(2));
messagesToSend.add(producer.newMessage().value("msg-3").key("A").sequenceId(3));

What is the expected behavior for the above case?

The deduplication depends on the monotonically increasing sequence ID, but the key-based batcher will break the rule.

Before this patch After this patch
message order A-0, A-3, B-1, B-2 B-1, B-2, A-0, A-3
highest sequence id pushed 1 3

I've explained in details in the previous comment. It's true that even after this patch, message deduplication cannot achieve effectively-once, instead, it's at-most once. But it can avoid the duplicated messages. It doesn't solve the problem fundamentally, but it makes it better.

I noticed that when #4435 added the key based batch container, it also sorted the batches by sequence id.

list.sort(((o1, o2) -> ComparisonChain.start()
.compare(o1.sequenceId, o2.sequenceId)
.result()));

At that time, #5491 was not pushed and there is no highest sequence id field in MessageMetadata. In #5491, it looks like the key based batch container was forgotten so that there were no changes.

Even without considering the enhancement (maybe meaningless at this time) to the deduplication, the refactor of this PR that reuses the BatchMessageContainerImpl rather than the KeyedBatch could make the code easier to maintain. For example, when currentTxnidMostBits and currentTxnidLeastBits were added in #8415, these two fields must be added to both default and key based containers.

BTW, maybe in future, we can support deduplication by keys and Key_Shared mode can also support effectively-once as well.

@BewareMyPower
Copy link
Contributor Author

BewareMyPower commented May 6, 2022

The initial purpose of this PR was that I'm wondering why adding a KeyedBatch class in BatchMessageKeyBasedContainer rather than reusing the BatchMessageContainerImpl. Then I found the difference that BatchMessageKeyBasedContainer

  1. doesn't update the ProducerImpl#lastSequenceIdPushed
  2. sorts the batches by sequence id, not highest sequence id

So during the refactoring, I tend to adopt the more correct behavior rather than keeping the previous behavior (might not be on purpose, but a mistake).

@codelipenghui
Copy link
Contributor

Oh, I see. I thought the PR was for support deduplication for key-based batching before.

Copy link
Contributor

@codelipenghui codelipenghui left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The change looks good to me, just left some minor comments.

@BewareMyPower
Copy link
Contributor Author

@codelipenghui All comments are addressed, PTAL again.

@codelipenghui codelipenghui requested a review from hangc0276 May 7, 2022 15:19
@BewareMyPower
Copy link
Contributor Author

@hangc0276 Could you also take a look?

### Motivation

Currently message deduplication doesn't work well for key based
batching. First, the key based batch container doesn't update the
`lastSequenceIdPushed`. So a batch could contain both duplicated and not
duplicated messages. Second, when `createOpSendMsgs` is called, the
`OpSendMsg` objects are sorted by the lowest sequence ids, and the
highest sequence id is not set. If a batch contains sequence id 0,1,2,
then the message with sequence id 1 or 2 won't be dropped.

### Modifications

- Refactor the key based batch container that the
  `BatchMessageContainerImpl` is reused instead of maintaining a
  `KeyedBatch` class.
- When `createOpSendMsgs` is called, clear the highest sequence id field
  and configure the sequence id field with the highest sequence id to fix
  the second issue described before.
- Add `testKeyBasedBatchingOrder` to show and verify the current
  behavior.
- Add test for key based batching into
  `testProducerDeduplicationWithDiscontinuousSequenceId` to verify
  `lastSlastSequenceIdPushed` is updated correctly.
@BewareMyPower BewareMyPower force-pushed the bewaremypower/key-based-batcher-highest-seq-id branch from 79e7e47 to 3ad7fa9 Compare May 9, 2022 12:44
@BewareMyPower
Copy link
Contributor Author

I accidentally found (triggered) a bug of GitHub that I merged a PR twice because of the network issue.:sweat_smile:

image

This PR is still in an Open state, However, please don't merge this PR again.

cbornet pushed a commit to cbornet/pulsar that referenced this pull request May 13, 2022
…ing (apache#15413)

### Motivation

Currently message deduplication doesn't work well for key based
batching. First, the key based batch container doesn't update the
`lastSequenceIdPushed`. So a batch could contain both duplicated and not
duplicated messages. Second, when `createOpSendMsgs` is called, the
`OpSendMsg` objects are sorted by the lowest sequence ids, and the
highest sequence id is not set. If a batch contains sequence id 0,1,2,
then the message with sequence id 1 or 2 won't be dropped.

### Modifications

- Refactor the key based batch container that the
  `BatchMessageContainerImpl` is reused instead of maintaining a
  `KeyedBatch` class.
- When `createOpSendMsgs` is called, clear the highest sequence id field
  and configure the sequence id field with the highest sequence id to fix
  the second issue described before.
- Add `testKeyBasedBatchingOrder` to show and verify the current
  behavior.
- Add test for key based batching into
  `testProducerDeduplicationWithDiscontinuousSequenceId` to verify
  `lastSlastSequenceIdPushed` is updated correctly.
cbornet pushed a commit to cbornet/pulsar that referenced this pull request May 13, 2022
…ing (apache#15413)

### Motivation

Currently message deduplication doesn't work well for key based
batching. First, the key based batch container doesn't update the
`lastSequenceIdPushed`. So a batch could contain both duplicated and not
duplicated messages. Second, when `createOpSendMsgs` is called, the
`OpSendMsg` objects are sorted by the lowest sequence ids, and the
highest sequence id is not set. If a batch contains sequence id 0,1,2,
then the message with sequence id 1 or 2 won't be dropped.

### Modifications

- Refactor the key based batch container that the
  `BatchMessageContainerImpl` is reused instead of maintaining a
  `KeyedBatch` class.
- When `createOpSendMsgs` is called, clear the highest sequence id field
  and configure the sequence id field with the highest sequence id to fix
  the second issue described before.
- Add `testKeyBasedBatchingOrder` to show and verify the current
  behavior.
- Add test for key based batching into
  `testProducerDeduplicationWithDiscontinuousSequenceId` to verify
  `lastSlastSequenceIdPushed` is updated correctly.
@BewareMyPower BewareMyPower deleted the bewaremypower/key-based-batcher-highest-seq-id branch May 16, 2022 17:17
gaozhangmin pushed a commit to gaozhangmin/pulsar that referenced this pull request May 17, 2022
…ing (apache#15413)

### Motivation

Currently message deduplication doesn't work well for key based
batching. First, the key based batch container doesn't update the
`lastSequenceIdPushed`. So a batch could contain both duplicated and not
duplicated messages. Second, when `createOpSendMsgs` is called, the
`OpSendMsg` objects are sorted by the lowest sequence ids, and the
highest sequence id is not set. If a batch contains sequence id 0,1,2,
then the message with sequence id 1 or 2 won't be dropped.

### Modifications

- Refactor the key based batch container that the
  `BatchMessageContainerImpl` is reused instead of maintaining a
  `KeyedBatch` class.
- When `createOpSendMsgs` is called, clear the highest sequence id field
  and configure the sequence id field with the highest sequence id to fix
  the second issue described before.
- Add `testKeyBasedBatchingOrder` to show and verify the current
  behavior.
- Add test for key based batching into
  `testProducerDeduplicationWithDiscontinuousSequenceId` to verify
  `lastSlastSequenceIdPushed` is updated correctly.
BewareMyPower added a commit that referenced this pull request May 24, 2022
…ing (#15413)

### Motivation

Currently message deduplication doesn't work well for key based
batching. First, the key based batch container doesn't update the
`lastSequenceIdPushed`. So a batch could contain both duplicated and not
duplicated messages. Second, when `createOpSendMsgs` is called, the
`OpSendMsg` objects are sorted by the lowest sequence ids, and the
highest sequence id is not set. If a batch contains sequence id 0,1,2,
then the message with sequence id 1 or 2 won't be dropped.

### Modifications

- Refactor the key based batch container that the
  `BatchMessageContainerImpl` is reused instead of maintaining a
  `KeyedBatch` class.
- When `createOpSendMsgs` is called, clear the highest sequence id field
  and configure the sequence id field with the highest sequence id to fix
  the second issue described before.
- Add `testKeyBasedBatchingOrder` to show and verify the current
  behavior.
- Add test for key based batching into
  `testProducerDeduplicationWithDiscontinuousSequenceId` to verify
  `lastSlastSequenceIdPushed` is updated correctly.

(cherry picked from commit a773337)
@BewareMyPower BewareMyPower added the cherry-picked/branch-2.9 Archived: 2.9 is end of life label May 24, 2022
BewareMyPower added a commit that referenced this pull request Jul 27, 2022
…ing (#15413)

### Motivation

Currently message deduplication doesn't work well for key based
batching. First, the key based batch container doesn't update the
`lastSequenceIdPushed`. So a batch could contain both duplicated and not
duplicated messages. Second, when `createOpSendMsgs` is called, the
`OpSendMsg` objects are sorted by the lowest sequence ids, and the
highest sequence id is not set. If a batch contains sequence id 0,1,2,
then the message with sequence id 1 or 2 won't be dropped.

### Modifications

- Refactor the key based batch container that the
  `BatchMessageContainerImpl` is reused instead of maintaining a
  `KeyedBatch` class.
- When `createOpSendMsgs` is called, clear the highest sequence id field
  and configure the sequence id field with the highest sequence id to fix
  the second issue described before.
- Add `testKeyBasedBatchingOrder` to show and verify the current
  behavior.
- Add test for key based batching into
  `testProducerDeduplicationWithDiscontinuousSequenceId` to verify
  `lastSlastSequenceIdPushed` is updated correctly.

(cherry picked from commit a773337)
@BewareMyPower BewareMyPower added the cherry-picked/branch-2.8 Archived: 2.8 is end of life label Jul 27, 2022
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
cherry-picked/branch-2.8 Archived: 2.8 is end of life cherry-picked/branch-2.9 Archived: 2.9 is end of life doc-not-needed Your PR changes do not impact docs release/2.8.4 release/2.9.3 release/2.10.1 type/bug The PR fixed a bug or issue reported a bug
Projects
None yet
Development

Successfully merging this pull request may close these issues.

3 participants