Skip to content
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

Support writing to Pubsub with ordering key; Add PubsubMessage SchemaCoder #31608

Open
wants to merge 29 commits into
base: master
Choose a base branch
from

Conversation

ahmedabu98
Copy link
Contributor

@ahmedabu98 ahmedabu98 commented Jun 15, 2024

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.

@ahmedabu98
Copy link
Contributor Author

Confirmed that ordering key is preserved with both direct runner and dataflow runner

@egalpin
Copy link
Member

egalpin commented Jun 15, 2024

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!

Copy link
Contributor

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).

Comment on lines 1510 to 1514
if (getNeedsOrderingKey()) {
pubsubMessages.setCoder(PubsubMessageSchemaCoder.getSchemaCoder());
} else {
pubsubMessages.setCoder(new PubsubMessageWithTopicCoder());
}
Copy link
Contributor Author

Choose a reason for hiding this comment

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

This fork is required to not break update compatibility

@sjvanrossum
Copy link
Contributor

sjvanrossum commented Jun 17, 2024

@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:

  1. This does not work with DataflowRunner unless --experiments=enable_custom_pubsub_sink is specified since Dataflow's native implementation for PubsubUnboundedSink omits the ordering key before publishing.
  2. PubsubUnboundedSink.PubsubSink and PubsubUnboundedSink.PubsubDynamicSink do not group on the ordering key property, which will cause multiple ordering keys to end up in the same batch for publishing.
  3. PubsubUnboundedSink sets a fixed number of shards (100) on both PubsubUnboundedSink.PubsubSink and PubsubUnboundedSink.PubsubDynamicSink to 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.

@ahmedabu98
Copy link
Contributor Author

@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.

@sjvanrossum
Copy link
Contributor

@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.

@scwhittle
Copy link
Contributor

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.

@sjvanrossum
Copy link
Contributor

sjvanrossum commented Jun 18, 2024

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.

Copy link
Contributor

Reminder, please take a look at this pr: @damondouglas @shunping

Copy link
Contributor

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)

Copy link
Contributor

github-actions bot commented Jul 7, 2024

Reminder, please take a look at this pr: @robertwb @Abacn

@sjvanrossum
Copy link
Contributor

sjvanrossum commented Jul 8, 2024

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:

  1. Fix the validation issue in PreparePubsubWriteDoFn for both REST and gRPC clients.
  2. Use a more ergonomic batching mechanism than I had initially proposed in Additional patches for apache#31608 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#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?

@Abacn
Copy link
Contributor

Abacn commented Jul 8, 2024

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.

@sjvanrossum
Copy link
Contributor

sjvanrossum commented Jul 8, 2024

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#427 before US business hours start tomorrow and I'll defer the rest to separate PRs. 👍

Copy link
Contributor

Reminder, please take a look at this pr: @damondouglas @damondouglas

@ahmedabu98
Copy link
Contributor Author

Friendly ping @scwhittle @sjvanrossum

Copy link
Contributor

@scwhittle scwhittle left a comment

Choose a reason for hiding this comment

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

Sorry for the delay

K key = keyFunction.apply(ThreadLocalRandom.current().nextInt(numShards), topic);
@Nullable String orderingKey = message.getOrderingKey();
int shard =
Strings.isNullOrEmpty(orderingKey)
Copy link
Contributor

Choose a reason for hiding this comment

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

can we use the new logic only if sink is configured to care about ordering keys to avoid changing the batching for existing cases where key is set?

Copy link
Contributor

Choose a reason for hiding this comment

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

That makes sense, after reverting the strict validation on ordering keys requires a change here as well to retain the existing behavior.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

See response in the comment below. I think it makes sense to revert this new logic

Copy link
Contributor

Choose a reason for hiding this comment

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

it seems we may want to group by ordering key so that we can do better at sequencing publishing to the ordering key if that prevents pubsub ingestion issues.

However to fix the issue that this new sharding is used even if we are not publishing the ordering key, perhaps we should instead clear the ordering key in the verification dofn if we don't want to publish the ordering key? That could be done before we include the ordering key in message size validation etc which doesn't make sense if it will later be ignored.

int shard =
Strings.isNullOrEmpty(orderingKey)
? ThreadLocalRandom.current().nextInt(numShards)
: Hashing.murmur3_32_fixed().hashString(orderingKey, StandardCharsets.UTF_8).asInt();
Copy link
Contributor

Choose a reason for hiding this comment

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

in general using a huge # of keys hurts performance. It seems like it would be better to still limit to numShards but be deterministic shard. The user can still for now increase the numShards very high if needed for performance.

If we need more publishing paralellism that seems like it should be done below by just adding publishBatch to scheduled executor and then joining, not via key parallelism.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Makes sense. I see we already group messages into different batches based on ordering key down in WriterFn. In other words, downstream steps don't rely on bundles being grouped by ordering key.

Do we necessarily need to shard by ordering key here? I wonder if we can revert this section and keep to numShards CC @sjvanrossum

Copy link
Contributor

Choose a reason for hiding this comment

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

Agreed, that was the intention based on an earlier comment of yours, but I guess it slipped my mind to apply bucketing.

Copy link
Contributor

@sjvanrossum sjvanrossum Sep 28, 2024

Choose a reason for hiding this comment

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

@ahmedabu98 a deterministic shard number based on the ordering key ensures that the writers only operate on distinct key ranges, thus avoiding the creation of multiple buffers per ordering key with a potentially small number of batched elements per bundle. Volume skews across keys is as much a problem for Beam as it is for Pub/Sub and does not fit well with the feature since throughput is capped at 1MB/s per ordering key.

One thing that does concern me is ending up with a skewed distribution of keys across buckets, but that could be fixed by tweaking the hashing or bucketing algorithm. The suggestion below switches the hashing algorithm to 64-bit FarmHash since consistentHash will pad or shrink the provided hash code to long.

Suggested change
: Hashing.murmur3_32_fixed().hashString(orderingKey, StandardCharsets.UTF_8).asInt();
: Hashing.consistentHash(
Hashing.farmHashFingerprint64().hashString(orderingKey, StandardCharsets.UTF_8),
numShards);

@sjvanrossum
Copy link
Contributor

@scwhittle I just realized that the PubSubClient implementations used by the writer transforms do not apply flow control to publish calls. The throughput quota per ordering key is capped to 1MB/s and would only be enforced by server-side rejections if exceeded by this connector. The official Pub/Sub client library for Java provides client-side flow control and batching per ordering key as well as an executor service that sends batches for different ordering keys concurrently and sequentially sending batches for a single ordering key.

Would it make sense to change the client implementation to the official client library or should we duplicate that functionality by extending the Beam clients to handle this?

Copy link
Contributor

github-actions bot commented Oct 6, 2024

Reminder, please take a look at this pr: @damondouglas @damondouglas

@scwhittle
Copy link
Contributor

@scwhittle I just realized that the PubSubClient implementations used by the writer transforms do not apply flow control to publish calls. The throughput quota per ordering key is capped to 1MB/s and would only be enforced by server-side rejections if exceeded by this connector. The official Pub/Sub client library for Java provides client-side flow control and batching per ordering key as well as an executor service that sends batches for different ordering keys concurrently and sequentially sending batches for a single ordering key.

Would it make sense to change the client implementation to the official client library or should we duplicate that functionality by extending the Beam clients to handle this?

I think using the pubsub provided client would be good to do and was wondering why we weren't using it. I'm guessing it might not have been available when the original Beam implementation was done. Perhaps this could go in with some minimal backoff sleeping to retry such errors and that can be done separately?

Copy link
Contributor

Reminder, please take a look at this pr: @damondouglas @damondouglas

Copy link
Contributor

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)

Copy link
Contributor

Reminder, please take a look at this pr: @robertwb @Abacn

Copy link
Contributor

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)

Copy link
Contributor

github-actions bot commented Nov 5, 2024

Reminder, please take a look at this pr: @kennknowles @damondouglas

Copy link
Contributor

github-actions bot commented Nov 7, 2024

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: @Abacn 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)

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

Add ability to Write to GCP PubSub with an orderingKey
6 participants