You signed in with another tab or window. Reload to refresh your session.You signed out in another tab or window. Reload to refresh your session.You switched accounts on another tab or window. Reload to refresh your session.Dismiss alert
Currently, chunk messages producing fails if topic level maxMessageSize is set [1]. The root cause of this issue is because chunk message is using broker level maxMessageSize as chunk size. And topic level maxMessageSize is always <= broker level maxMessageSize. So once it is set, the on-going chunk message producing fails.
Goal
Resolve topic level maxMessageSize compatibility issue with chunking messages.
Implementation
Current best solution would be just skipping topic level maxMessageSize check in org.apache.pulsar.broker.service.AbstractTopic#isExceedMaximumMessageSize.
Topic level maxMessageSize is introduced in [2], for the purpose of "easier to plan resource quotas for client allocation". And IMO this change will not bring further complex into this.
Reject Alternatives
Add a client side topic level maxMessageSize and keep it synced with broker.
Required changes:
[client] Add a new field org.apache.pulsar.client.impl.ProducerBase#maxMessageSize to store this client side topic level maxMessageSize.
[PulsarApi.proto] Add a new field maxMessageSize in the CommandProducerSuccess for the initial value of ProducerBase#maxMessageSize
[PulsarApi.proto] Add a new Command like CommandUpdateClientPolicy{producerId, maxMessageSize} to update ProducerBase#maxMessageSize when topic level maxMessageSize is updated.
Further more, some other data consistency issues need be handled very carefully when maxMessageSize is updated.
This alternative is complex but can also solve other topic level maxMessageSize issue [3] when batching is enabled (non-batching case is solved with PR [4]).
Discussion thread: https://lists.apache.org/thread/pp1lgpbhy07z5mg2hrosrbjwksxx3qq8
Vote thread: https://lists.apache.org/thread/cgww0m8dkrb14dbmkmw2qq3v5kkgvdmc
Motivation
Currently, chunk messages producing fails if topic level maxMessageSize is set [1]. The root cause of this issue is because chunk message is using broker level maxMessageSize as chunk size. And topic level maxMessageSize is always <= broker level maxMessageSize. So once it is set, the on-going chunk message producing fails.
Goal
Resolve topic level maxMessageSize compatibility issue with chunking messages.
Implementation
Current best solution would be just skipping topic level maxMessageSize check in org.apache.pulsar.broker.service.AbstractTopic#isExceedMaximumMessageSize.
Topic level maxMessageSize is introduced in [2], for the purpose of "easier to plan resource quotas for client allocation". And IMO this change will not bring further complex into this.
Reject Alternatives
Add a client side topic level maxMessageSize and keep it synced with broker.
Required changes:
Further more, some other data consistency issues need be handled very carefully when maxMessageSize is updated.
This alternative is complex but can also solve other topic level maxMessageSize issue [3] when batching is enabled (non-batching case is solved with PR [4]).
[1] #13360
[2] #8732
[3] #12958
[4] #13147
The text was updated successfully, but these errors were encountered: