You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by GitBox <gi...@apache.org> on 2022/07/26 10:20:04 UTC

[GitHub] [pulsar] Gleiphir2769 opened a new pull request, #16797: [fix][client] Fix incorrect out-of-order check for chunked message in consumerImpl

Gleiphir2769 opened a new pull request, #16797:
URL: https://github.com/apache/pulsar/pull/16797

   Fixes #[16796](https://github.com/apache/pulsar/issues/16796)
   
   ### Motivation
   
   There is a incorrect out-of-order check for chunked message in `ConsumerImpl`. For the last check should compare the result of `chunkedMsgCtx.chunkedMsgBuffer.readableBytes() + compressedPayload.readableBytes()` with `TotalChunkMsgSize` instead of `ChunkId` with `TotalChunkMsgSize`.
   
   ### Modifications
   
   Fix the out-of-order check for chunked message in `ConsumerImpl`.
   
   ### 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)
     
   - [x] `doc-not-needed` 
   (Please explain why)
     
   - [ ] `doc` 
   (Your PR contains doc changes)
   
   - [ ] `doc-complete`
   (Docs have been already added)


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] Gleiphir2769 commented on pull request #16797: [fix][client] Fix incorrect out-of-order check for chunked message in consumerImpl

Posted by GitBox <gi...@apache.org>.
Gleiphir2769 commented on PR #16797:
URL: https://github.com/apache/pulsar/pull/16797#issuecomment-1203468320

   > IIUC
   
   Hi, the rebase is done.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] Gleiphir2769 commented on pull request #16797: [fix][client] Fix incorrect out-of-order check for chunked message in consumerImpl

Posted by GitBox <gi...@apache.org>.
Gleiphir2769 commented on PR #16797:
URL: https://github.com/apache/pulsar/pull/16797#issuecomment-1203462287

   > requested changes
   
   Hi, thanks for your review. The problem for `TotalChunkMsgSize ` will be caused by the message with incorrect metadata, which is caused by the incorrect implementation of the producer client. So I think it should check each chunk because it's not necessarily only the last chunk will cause the problem.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] Technoboy- commented on pull request #16797: [fix][client] Remove redundant check for chunked message TotalChunkMsgSize in ConsumerImpl

Posted by GitBox <gi...@apache.org>.
Technoboy- commented on PR #16797:
URL: https://github.com/apache/pulsar/pull/16797#issuecomment-1207582715

   > @Technoboy- Could this PR be included in 2.11.0 release? If not, we can add a `release/2.11.1` label for it.
   
   Will cherry-pick to 2.11.0


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] BewareMyPower commented on pull request #16797: [fix][client] Remove redundant check for chunked message TotalChunkMsgSize in ConsumerImpl

Posted by GitBox <gi...@apache.org>.
BewareMyPower commented on PR #16797:
URL: https://github.com/apache/pulsar/pull/16797#issuecomment-1206763468

   @Technoboy- Could this PR be included in 2.11.0 release? If not, we can add a `release/2.11.1` label for it.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] BewareMyPower commented on pull request #16797: [fix][client] Fix incorrect out-of-order check for chunked message in consumerImpl

Posted by GitBox <gi...@apache.org>.
BewareMyPower commented on PR #16797:
URL: https://github.com/apache/pulsar/pull/16797#issuecomment-1203502113

   @RobertIndie @Gleiphir2769 **In short, I think there is no need to check the chunk msg size.**
   
   I guess the original code of `msgMetadata.getChunkId() >= msgMetadata.getTotalChunkMsgSize()` might be `msgMetadata.getChunkId() >= msgMetadata.getNumChunksFromMsg()` to check if the chunk id is invalid. See the following log:
   
   https://github.com/apache/pulsar/blob/2e8bd3d7b17190d6fb45d5b35eff598948975385/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java#L1439-L1441
   
   Intuitively, the "total-chunks" should be the number of chunks, but `getTotalChunkMsgSize()` might be somehow misleading. I believe the original purpose is to apply the range check for chunk id.
   
   Back to the discussion, the `total_chunk_msg_size` field is designed only for a better initial size of the byte buffer, see https://github.com/apache/pulsar/blob/2e8bd3d7b17190d6fb45d5b35eff598948975385/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java#L1421-L1422
   
   Only the 1st chunk's size is used.
   
   To verify if a chunk is valid, we have already used `uuid` for it https://github.com/apache/pulsar/blob/2e8bd3d7b17190d6fb45d5b35eff598948975385/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java#L1424-L1425
   
   If you are very suspicious about if the producer has set these metadata correctly, checking the `total_chunk_msg_size` is **far not enough**.
   
   First, we only check `num_chunks_for_msg` for whether it's a chunk.
   
   https://github.com/apache/pulsar/blob/2e8bd3d7b17190d6fb45d5b35eff598948975385/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java#L1302-L1303
   
   However, what if
   - `chunk_id`, `uuid` or `total_chunk_msg_size` was not set but `num_chunks_from_msg` was set?
   - `uuid` fields are different among the chunks of the same message?
   - `total_chunk_msg_size` fields are different among the chunks of the same message?
   
   In these cases above, we should fail fast instead of only logging a line and skip these chunks. These error cases should be detected in the test phase. It's an **ERROR**, not an **EXCEPTION**.
   
   We check the order of chunk id because even if the producer has set chunk related metadata correctly, if there is a bug at broker side, which might be caused by a rare case when dispatching messages, the chunks might be out or order. Discarding them might be helpful to find the bug so that we can fix it in broker.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] Gleiphir2769 commented on pull request #16797: [fix][client] Fix incorrect out-of-order check for chunked message in consumerImpl

Posted by GitBox <gi...@apache.org>.
Gleiphir2769 commented on PR #16797:
URL: https://github.com/apache/pulsar/pull/16797#issuecomment-1199591594

   > Could you please add a test to cover the new change?
   
   Hi, this out-of-bounds check will not be triggered in the correct client implementation. It will only be triggered if the producer client implement does not set the TotalChunkMsgSize correctly. So I think it's not neccessary to add a test.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] RobertIndie commented on pull request #16797: [fix][client] Fix incorrect out-of-order check for chunked message in consumerImpl

Posted by GitBox <gi...@apache.org>.
RobertIndie commented on PR #16797:
URL: https://github.com/apache/pulsar/pull/16797#issuecomment-1205046098

   @Gleiphir2769 Please change the PR title.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] Gleiphir2769 commented on pull request #16797: [fix][client] Fix incorrect out-of-order check for chunked message in consumerImpl

Posted by GitBox <gi...@apache.org>.
Gleiphir2769 commented on PR #16797:
URL: https://github.com/apache/pulsar/pull/16797#issuecomment-1203469646

   > > requested changes
   > 
   > Hi, thanks for your review. The problem for `TotalChunkMsgSize ` will be caused by the message with incorrect metadata, which is caused by the incorrect implementation of the producer client. So I think it should check each chunk because it's not necessarily only the last chunk will cause the problem.
   
   
   
   
   > out-of-order chunks would not make the total bytes greater than the `total_chunk_msg_size`, we should only check chunk id here.
   > 
   > If the total size of `chunkedMsgBuffer` was greater than the `total_chunk_msg_size` field, which is set by producer, it might be caused by the corrupted message metadata or wrong implementations of producer. (e.g. there are two chunked messages with the same UUID and those chunks are mixed)
   > 
   > In this case, we should not check the size here. Instead, if you tend to validate whether the `total_chunk_msg_size` is right, you should check it when the chunked message is complete.
   > 
   > ```java
   >         if (msgMetadata.getChunkId() != (msgMetadata.getNumChunksFromMsg() - 1)) {
   >             /* ... */
   >             return null;
   >         }
   > 
   >         if (chunkedMsgCtx.chunkedMsgBuffer.readableBytes() != msgMetadata.getTotalChunkMsgSize()) {
   >             log.error("Received a chunked message whose size is {} while the total_chunk_msg_size field is {}",
   >                     chunkedMsgCtx.chunkedMsgBuffer.readableBytes(), msgMetadata.getTotalChunkMsgSize());
   >             chunkedMsgCtx.chunkedMsgBuffer.release();
   >             compressedPayload.release();
   >             // TODO: we need to remove the current chunkedMsgCtx, maybe we should throw an exception here?
   >             return null;
   >         }
   > ```
   > 
   > If the total
   
   Hi, thanks for your review. The problem for TotalChunkMsgSize will be caused by the message with incorrect metadata, which is caused by the incorrect implementation of the producer client. So I think it should check each chunk because it's not necessarily only the last chunk will cause the problem.
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] BewareMyPower commented on pull request #16797: [fix][client] Fix incorrect out-of-order check for chunked message in consumerImpl

Posted by GitBox <gi...@apache.org>.
BewareMyPower commented on PR #16797:
URL: https://github.com/apache/pulsar/pull/16797#issuecomment-1203580501

   I agree, we can simply remove this line.
   
   Actually, when I migrated the chunking feature into C++ client, I only checked the chunk id and the current number of chunks. Maybe this implementation would be better
   
   https://github.com/apache/pulsar/blob/68e454544d45734129e67ab3032eae3e40ff664c/pulsar-client-cpp/lib/ConsumerImpl.h#L240
   
   https://github.com/apache/pulsar/blob/68e454544d45734129e67ab3032eae3e40ff664c/pulsar-client-cpp/lib/ConsumerImpl.h#L262-L264
   
   https://github.com/apache/pulsar/blob/68e454544d45734129e67ab3032eae3e40ff664c/pulsar-client-cpp/lib/ConsumerImpl.cc#L354-L355


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] Gleiphir2769 commented on pull request #16797: [fix][client] Remove redundant check for chunked message TotalChunkMsgSize in ConsumerImpl

Posted by GitBox <gi...@apache.org>.
Gleiphir2769 commented on PR #16797:
URL: https://github.com/apache/pulsar/pull/16797#issuecomment-1205999658

   > @Gleiphir2769 Please change the PR title.
   
   OK, it's done. @RobertIndie 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] RobertIndie merged pull request #16797: [fix][client] Remove redundant check for chunked message TotalChunkMsgSize in ConsumerImpl

Posted by GitBox <gi...@apache.org>.
RobertIndie merged PR #16797:
URL: https://github.com/apache/pulsar/pull/16797


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] RobertIndie commented on pull request #16797: [fix][client] Fix incorrect out-of-order check for chunked message in consumerImpl

Posted by GitBox <gi...@apache.org>.
RobertIndie commented on PR #16797:
URL: https://github.com/apache/pulsar/pull/16797#issuecomment-1203452715

   > In this case, we should not check the size here. Instead, if you tend to validate whether the total_chunk_msg_size is right, you should check it when the chunked message is complete.
   
   Hi @BewareMyPower . IIUC, This approach has the same effect as this PR. 
   ```java
           if (chunkedMsgCtx == null || chunkedMsgCtx.chunkedMsgBuffer == null
                   || msgMetadata.getChunkId() != (chunkedMsgCtx.lastChunkedMessageId + 1)
                   || chunkedMsgCtx.chunkedMsgBuffer.readableBytes() + compressedPayload.readableBytes()
                   > msgMetadata.getTotalChunkMsgSize()) {
   ```
   If we reach the last validation: `chunkedMsgCtx.chunkedMsgBuffer.readableBytes() + compressedPayload.readableBytes()
                   > msgMetadata.getTotalChunkMsgSize()`, that means we are handling the last chunk. 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] BewareMyPower commented on pull request #16797: [fix][client] Fix incorrect out-of-order check for chunked message in consumerImpl

Posted by GitBox <gi...@apache.org>.
BewareMyPower commented on PR #16797:
URL: https://github.com/apache/pulsar/pull/16797#issuecomment-1203443604

   BTW, it's not a serious bug that we don't need to include it in Pulsar 2.11.0 in a hurry. Because with the existing code, `msgMetadata.getChunkId() >= msgMetadata.getTotalChunkMsgSize()` always returns false unless each chunk has only 1 byte.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] Gleiphir2769 commented on pull request #16797: [fix][client] Fix incorrect out-of-order check for chunked message in consumerImpl

Posted by GitBox <gi...@apache.org>.
Gleiphir2769 commented on PR #16797:
URL: https://github.com/apache/pulsar/pull/16797#issuecomment-1203468908

   > Could you rebase the master branch ?
   
   Hi, the rebase is done.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] RobertIndie commented on pull request #16797: [fix][client] Fix incorrect out-of-order check for chunked message in consumerImpl

Posted by GitBox <gi...@apache.org>.
RobertIndie commented on PR #16797:
URL: https://github.com/apache/pulsar/pull/16797#issuecomment-1203541387

   > In short, I think there is no need to check the chunk msg size.
   
   You're right. If we need to give the client the responsibility to check the correctness of the metadata or the correctness of the client's implementation, then many things need to be checked.
   
   > I guess the original code of `msgMetadata.getChunkId() >= msgMetadata.getTotalChunkMsgSize()` might be `msgMetadata.getChunkId() >= msgMetadata.getNumChunksFromMsg()` to check if the chunk id is invalid. 
   
   I think we can change this PR by removing this validation to not confuse other developers. This validation must be false. It's better to remove such redundant validation. @Gleiphir2769 @BewareMyPower 
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] Gleiphir2769 commented on pull request #16797: [fix][client] Fix incorrect out-of-order check for chunked message in consumerImpl

Posted by GitBox <gi...@apache.org>.
Gleiphir2769 commented on PR #16797:
URL: https://github.com/apache/pulsar/pull/16797#issuecomment-1204692997

   >  In short, I think there is no need to check the chunk msg size.
   
   You are right, the check for totalchunk is no need.
   
   > This validation must be false. It's better to remove such redundant validation.
   
   I agree, it's better to remove it because even in original intention of original code, `msgMetadata.getChunkId() >= msgMetadata.getNumChunksFromMsg()` is also redundant.
   
   > We should also change the log in the if block as well.
   
   This reminds me, thanks. I will do this change.
   @BewareMyPower @RobertIndie 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] RobertIndie commented on pull request #16797: [fix][client] Remove redundant check for chunked message TotalChunkMsgSize in ConsumerImpl

Posted by GitBox <gi...@apache.org>.
RobertIndie commented on PR #16797:
URL: https://github.com/apache/pulsar/pull/16797#issuecomment-1206001154

   /pulsarbot run-failure-checks


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org