Support writing to Pubsub with ordering key; Add PubsubMessage SchemaCoder
Fixes #21162
I wasn't able to use the existing PubsubMessageWithAttributesAndMessageIdCoder because it doesn't encode/decode the message's topic, which is needed for dynamic destinations. There are already a number of existing coders (6) developed over the years. Every time a new feature/parameter is added to PubsubMessage, we need to make a new coder and fork the code to maintain update compatibility.
To mitigate this for the future, this PR introduces a SchemaCoder for PubsubMessage. SchemaCoder allows us to evolve the schema over time, so hopefully new features can be added in the future without breaking update compatibility.
Note that PubsubMessage's default coder is PubsubMessageWithAttributesCoder, which can't be updated without breaking backwards compatibility (see #23525). Wherever PubsubMessages are created in a pipeline, we would have to manually override the coder to PubsubMessageSchemaCoder.getSchemaCoder() or the ordering key will get lost.
Confirmed that ordering key is preserved with both direct runner and dataflow runner
Thanks @ahmedabu98! At first glance, this approach seems massively preferable to the set of bespoke coders that already exist, and those future ones that might need to exist later. I’d be happy to take a closer look next week!
Assigning reviewers. If you would like to opt out of this review, comment assign to next reviewer:
R: @damondouglas for label java. R: @shunping for label io.
Available commands:
-
stop reviewer notifications- opt out of the automated review tooling -
remind me after tests pass- tag the comment author after tests pass -
waiting on author- shift the attention set back to the author (any comment or push by the author will return the attention set to the reviewers)
The PR bot will only process comments in the main thread (not review comments).
@iht and I have been looking into this today for a Dataflow customer and we came across a few details that seem to be missing on this PR:
- This does not work with
DataflowRunnerunless--experiments=enable_custom_pubsub_sinkis specified since Dataflow's native implementation forPubsubUnboundedSinkomits the ordering key before publishing. -
PubsubUnboundedSink.PubsubSinkandPubsubUnboundedSink.PubsubDynamicSinkdo not group on the ordering key property, which will cause multiple ordering keys to end up in the same batch for publishing. -
PubsubUnboundedSinksets a fixed number of shards (100) on bothPubsubUnboundedSink.PubsubSinkandPubsubUnboundedSink.PubsubDynamicSinkto improve latency within the sink. Simply adding the ordering key as an additional property may result in many small batches being produced which can have a negative impact due to the per call overhead on batch publishing.
The issue we're working on is time-sensitive so we're trying to wrap up our patches today.
To avoid user confusion this PR must incorporate changes to PubsubUnboundedSink.ShardFn to avoid triggering this error in Pub/Sub:
In a single publish request, all messages must have no ordering key or they must all have the same ordering key. [code=539b]
A nice to have would be enabling users to customize the output sharding range based on ordering keys. Given the fact that throughput per ordering key is capped to 1 MBps (docs) I'd almost be inclined to say the ordering key should replace the output shard entirely.
@ahmedabu98 I'm happy to share our changes in a bit and I'll set up a PR against the source branch of this PR.
@sjvanrossum thank you for these insights, I'd be happy to take a look at your PR
I'm not familiar with the internal implementation and how it relates to this one, but looks like we'd need changes there too.
@scwhittle or @reuvenlax may be able to shed a light on Dataflow's implementation and the complexity of changes needed to accommodate this feature.
Context: To publish messages with an ordering key the sink must prepare publish batches per topic and ordering key. My proposal for the OSS sink is to reuse the dynamic destinations sink and to append the ordering key to the topic name in the sharding key. User configuration of the number of output shards or the use of a single output shard for messages with ordering keys (due to 1 MBps throughput limit per ordering key) is an open topic.
The DataflowRunner overrides the pubsub write transform using org.apache.beam.runners.dataflow.DataflowRunner.StreamingPubsubIOWrite so org.apache.beam.runners.dataflow.worker.PubsubSink is used. It would be nice to prevent using the ordering key for now with the DataflowRunner unless the experiment to use the beam implementation is present.
To add support for it to Dataflow, it appears that if PUBSUB_SERIALIZED_ATTRIBUTES_FN is set, that maps bytes to PubsubMessage which already includes the ordering key. But for the ordering key to be respected for publishing, additional changes would be needed in the dataflow service backend. Currently it looks like it would just be dropped but if it was respected the service would also need to be updated to ensure batching doesn't occur across ordering keys.
User configuration of the number of output shards or the use of a single output shard for messages with ordering keys (due to 1 MBps throughput limit per ordering key) is an open topic.
Are you considering producing to a single ordering key from multiple distinct grouped-by keys in parallel? Doesn't that defeat the purpose of the ordering provided? I'm also not sure it would increase the throughput beyond the 1Mb per ordering key limit. An alternative would be grouping by partitioning of the ordering keys (via deterministic hash buckets for example) and then batching just within a bundle.
The DataflowRunner overrides the pubsub write transform using org.apache.beam.runners.dataflow.DataflowRunner.StreamingPubsubIOWrite so org.apache.beam.runners.dataflow.worker.PubsubSink is used. It would be nice to prevent using the ordering key for now with the DataflowRunner unless the experiment to use the beam implementation is present.
Agreed, I'll throw an exception when the DataflowRunner runs StreamingPubsubSinkTranslators#translate() and the target transform has PubsubUnboundedSink#getPublishBatchWithOrderingKey() set with details and instructions to resolve the issue.
To add support for it to Dataflow, it appears that if PUBSUB_SERIALIZED_ATTRIBUTES_FN is set, that maps bytes to PubsubMessage which already includes the ordering key. But for the ordering key to be respected for publishing, additional changes would be needed in the dataflow service backend. Currently it looks like it would just be dropped but if it was respected the service would also need to be updated to ensure batching doesn't occur across ordering keys.
Agreed, I'll create a new bug for this to continue this discussion internally.
User configuration of the number of output shards or the use of a single output shard for messages with ordering keys (due to 1 MBps throughput limit per ordering key) is an open topic.
Are you considering producing to a single ordering key from multiple distinct grouped-by keys in parallel? Doesn't that defeat the purpose of the ordering provided? I'm also not sure it would increase the throughput beyond the 1Mb per ordering key limit. An alternative would be grouping by partitioning of the ordering keys (via deterministic hash buckets for example) and then batching just within a bundle.
The initial patch I wrote concatenated topic and ordering key and left output shards unchanged.
After I reviewed the ordering key limitations I realized there's almost nothing to be gained there because of the per key throughput limit.
Since messages with and without ordering keys can be published to the same topic and these limitations only apply to messages with ordering keys I'll leave the shard allocation as is for messages without an ordering key and apply murmur3_32 on the ordering key to determine the shard number for messages with an ordering key.
PubsubBoundedWriter batches by topic per bundle and I'll extend and reuse that in PubsubUnboundedSink to batch by topic and ordering key like you suggested.
That closely resembles what a KafkaProducer does behind the scenes to batch by topic and partition.
Reminder, please take a look at this pr: @damondouglas @shunping
Assigning new set of reviewers because Pr has gone too long without review. If you would like to opt out of this review, comment assign to next reviewer:
R: @robertwb for label java. R: @Abacn for label io.
Available commands:
-
stop reviewer notifications- opt out of the automated review tooling -
remind me after tests pass- tag the comment author after tests pass -
waiting on author- shift the attention set back to the author (any comment or push by the author will return the attention set to the reviewers)
Reminder, please take a look at this pr: @robertwb @Abacn
Highlighting this here as well, while trying to retrofit ordering keys onto the existing sinks I thought of rewriting the sink using GroupIntoBatches for ordering keys and GroupIntoBatches.WithShardedKeys for normal messages. This would unify the bounded and unbounded writers, allow runners to determine sharding for normal messages instead of 100 shards and allow ordering keys to be grouped exactly by topic and ordering key instead of explicitly bucketing with murmur3_32.
While writing that sink I stumbled on some issues regarding message size validation as documented in #31800. I've got a few fixes in progress which will:
- Fix the validation issue in
PreparePubsubWriteDoFnfor both REST and gRPC clients. - Use a more ergonomic batching mechanism than I had initially proposed in ahmedabu98/beam#427.
My thoughts on fixing the validation issue is to introduce a PubsubMessage.SizeValidator interface, a visitor over all fields of the message which returns size parts and is summed by PubsubMessage#validateSize(), allowing messages and attributes to be overridden to account for things like the size implicit attribute entries timestampAttribute (values are up to 20 B, millis since epoch Long#toString()) and idAttribute (values are 24 B for Dataflow's native sink, 36 B for PubsubUnboundedSink). The base implementation would validate explicit size with subclasses to consider encoding overhead for JSON or protobuf. The protobuf validator can be dropped when Pub/Sub switches to explicit size validation.
Coincidentally, the revised batching mechanism I had imagined turns out to be very close to the implementation found in Google Cloud Pub/Sub Client for Java (https://github.com/googleapis/java-pubsub/blob/main/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java) and would live in PubsubClient instead of the existing implementations in BoundedWriter and PubsubUnboundedSink. Side note: The batching mechanism in the client library does not account for the encoding overhead of the topic and messages field of PublishRequest (1-3 bytes for topic, 1-5 bytes per message) which could cause batched publish requests to fail today and may still cause them to fail when Pub/Sub switches to explicit size validation if this isn't fixed, but I'll happily raise an issue or propose a fix for them separately.
@ahmedabu98 the fixes to the batching mechanism should address the comments you had raised on ahmedabu98/beam#427 about my use of variable assignments in the condition of an if statement so I'll get those commits added to that PR.
A separate PR makes more sense for the other bugfix.
In a separate comment we discussed including a GIB/GIB-WSK sink as the default sink for writes with ordering keys, but the design and implementation of that sink may add unnecessary bloat to this PR and reviews since it could be toggled for all types of writes if exposed through PubsubIO.Write#withAutoSharding() for example.
@robertwb, @Abacn any thoughts as reviewers?
I saw @scwhittle @egalpin already entered some ideas. Do you plan to finish the review in the near future? If not available I can do a first pass.
I see this new feature is guarded by a flag so won't affect existing uses if the flag is not set. So the current change looks fairly safe to get in.
I saw @scwhittle @egalpin already entered some ideas. Do you plan to finish the review in the near future? If not available I can do a first pass.
I'll have the batching fix added to ahmedabu98/beam#427 before US business hours start tomorrow and I'll defer the rest to separate PRs. 👍
@Abacn no need to hold anything up on my account, I'm +1 on the concept for sure (though I haven't taken a deep dive on the latest impl). Let me know if you need extra eyes for review, but don't let me hold anything up :-) Thanks for your consideration.
Can you explain the use case for ordering keys here? Apache Beam provides no guarantees on what order records will be written to the sink, so how are ordering keys being used?
On Mon, Jul 8, 2024 at 12:02 PM Evan Galpin @.***> wrote:
@Abacn https://github.com/Abacn no need to hold anything up on my account, I'm +1 on the concept for sure (though I haven't taken a deep dive on the latest impl). Let me know if you need extra eyes for review, but don't let me hold anything up :-) Thanks for your consideration.
— Reply to this email directly, view it on GitHub https://github.com/apache/beam/pull/31608#issuecomment-2214957009, or unsubscribe https://github.com/notifications/unsubscribe-auth/AFAYJVLDWXMFEBCXBXFYCYDZLLO3TAVCNFSM6AAAAABJLI4KDOVHI2DSMVQWIX3LMV43OSLTON2WKQ3PNVWWK3TUHMZDEMJUHE2TOMBQHE . You are receiving this because you were mentioned.Message ID: @.***>
Even though Beam itself makes no guarantee of publishing order, OrderingKeys can still be useful for creating partitions of data where the resulting partitions can be consumed (and re-consumed) in a deterministic order. It's not particularly about ordering from Beam to Pubsub, but ordering guarantees from Pubsub to downstream consumers thereafter.
So in this case you don't care exactly what the partitions are, as long as they are created?
On Mon, Jul 8, 2024 at 2:00 PM Evan Galpin @.***> wrote:
Even though Beam itself makes no guarantee of publishing order, OrderingKeys can still be useful for creating partitions of data where the resulting partitions can be consumed (and re-consumed) in a deterministic order. It's not particularly about ordering from Beam to Pubsub, but ordering guarantees from Pubsub to downstream consumers thereafter.
— Reply to this email directly, view it on GitHub https://github.com/apache/beam/pull/31608#issuecomment-2215291806, or unsubscribe https://github.com/notifications/unsubscribe-auth/AFAYJVN6Z2WLMMUGYDRQY7DZLL4W5AVCNFSM6AAAAABJLI4KDOVHI2DSMVQWIX3LMV43OSLTON2WKQ3PNVWWK3TUHMZDEMJVGI4TCOBQGY . You are receiving this because you were mentioned.Message ID: @.***>
Can you explain the use case for ordering keys here? Apache Beam provides no guarantees on what order records will be written to the sink, so how are ordering keys being used?
For a customer I'm currently working with they're aware of that and they can deal with out of order delivery through sequence IDs. Ordering keys would enable them to limit push message delivery to a downstream consumer since there can be only one outstanding message per ordering key. See https://cloud.google.com/pubsub/docs/ordering#subscriber_client_behavior_for_message_ordering for details on the limitations concerning ordering key subscribers.
So in this case you don't care exactly what the partitions are, as long as they are created?
I would want to be able to partition based on some attribute of the data (ex. user_id) such that data for user_id n always ended up in partition x. I believe that should be fully doable with support for ordering key here; I've done that before with great effort haha (enabling dataflow experiments, building local versions of pubsubIO, etc). And similar to what @sjvanrossum mentioned, I would deal with out-of-order data using application layer logic within consumers and using data within the message payloads to decide desired order.
@ahmedabu98 addressed your comments in ahmedabu98/beam#427.
Reminder, please take a look at this pr: @robertwb @Abacn
Assigning new set of reviewers because Pr has gone too long without review. If you would like to opt out of this review, comment assign to next reviewer:
R: @kennknowles for label java. R: @chamikaramj for label io.
Available commands:
-
stop reviewer notifications- opt out of the automated review tooling -
remind me after tests pass- tag the comment author after tests pass -
waiting on author- shift the attention set back to the author (any comment or push by the author will return the attention set to the reviewers)
Reminder, please take a look at this pr: @kennknowles @chamikaramj
Assigning new set of reviewers because Pr has gone too long without review. If you would like to opt out of this review, comment assign to next reviewer:
R: @robertwb for label java. R: @johnjcasey for label io.
Available commands:
-
stop reviewer notifications- opt out of the automated review tooling -
remind me after tests pass- tag the comment author after tests pass -
waiting on author- shift the attention set back to the author (any comment or push by the author will return the attention set to the reviewers)
Reminder, please take a look at this pr: @robertwb @johnjcasey
Assigning new set of reviewers because Pr has gone too long without review. If you would like to opt out of this review, comment assign to next reviewer:
R: @kennknowles for label java. R: @damondouglas for label io.
Available commands:
-
stop reviewer notifications- opt out of the automated review tooling -
remind me after tests pass- tag the comment author after tests pass -
waiting on author- shift the attention set back to the author (any comment or push by the author will return the attention set to the reviewers)
Reminder, please take a look at this pr: @kennknowles @damondouglas
Assigning new set of reviewers because Pr has gone too long without review. If you would like to opt out of this review, comment assign to next reviewer:
R: @robertwb for label java. R: @chamikaramj for label io.
Available commands:
-
stop reviewer notifications- opt out of the automated review tooling -
remind me after tests pass- tag the comment author after tests pass -
waiting on author- shift the attention set back to the author (any comment or push by the author will return the attention set to the reviewers)
Reminder, please take a look at this pr: @robertwb @chamikaramj