-
Notifications
You must be signed in to change notification settings - Fork 4.4k
feat(pubsub): support batch mode in WriteToPubSub transform #36027
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
base: master
Are you sure you want to change the base?
Changes from 10 commits
4492401
d5f553a
291e2e8
7af30e4
9fb516a
1b9fce5
499e34e
518c783
585a67e
d2e9150
4a5b4b2
43808d6
3b7e445
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -17,8 +17,9 @@ | |
|
||
"""Google Cloud PubSub sources and sinks. | ||
|
||
Cloud Pub/Sub sources and sinks are currently supported only in streaming | ||
pipelines, during remote execution. | ||
Cloud Pub/Sub sources are currently supported only in streaming pipelines, | ||
during remote execution. Cloud Pub/Sub sinks (WriteToPubSub) support both | ||
streaming and batch pipelines. | ||
|
||
This API is currently under development and is subject to change. | ||
|
||
|
@@ -42,7 +43,6 @@ | |
from apache_beam import coders | ||
from apache_beam.io import iobase | ||
from apache_beam.io.iobase import Read | ||
from apache_beam.io.iobase import Write | ||
from apache_beam.metrics.metric import Lineage | ||
from apache_beam.transforms import DoFn | ||
from apache_beam.transforms import Flatten | ||
|
@@ -376,7 +376,12 @@ def report_lineage_once(self): | |
|
||
|
||
class WriteToPubSub(PTransform): | ||
"""A ``PTransform`` for writing messages to Cloud Pub/Sub.""" | ||
"""A ``PTransform`` for writing messages to Cloud Pub/Sub. | ||
|
||
This transform supports both streaming and batch pipelines. In streaming mode, | ||
messages are written continuously as they arrive. In batch mode, all messages | ||
are written when the pipeline completes. | ||
""" | ||
|
||
# Implementation note: This ``PTransform`` is overridden by Directrunner. | ||
|
||
|
@@ -435,7 +440,7 @@ def expand(self, pcoll): | |
self.bytes_to_proto_str, self.project, | ||
self.topic_name)).with_input_types(Union[bytes, str]) | ||
pcoll.element_type = bytes | ||
return pcoll | Write(self._sink) | ||
return pcoll | ParDo(_PubSubWriteDoFn(self)) | ||
|
||
def to_runner_api_parameter(self, context): | ||
# Required as this is identified by type in PTransformOverrides. | ||
|
@@ -541,11 +546,67 @@ def is_bounded(self): | |
return False | ||
|
||
|
||
# TODO(BEAM-27443): Remove in favor of a proper WriteToPubSub transform. | ||
class _PubSubWriteDoFn(DoFn): | ||
"""DoFn for writing messages to Cloud Pub/Sub. | ||
|
||
This DoFn handles both streaming and batch modes by buffering messages | ||
and publishing them in batches to optimize performance. | ||
""" | ||
BUFFER_SIZE_ELEMENTS = 100 | ||
FLUSH_TIMEOUT_SECS = BUFFER_SIZE_ELEMENTS * 0.5 | ||
|
||
def __init__(self, transform): | ||
self.project = transform.project | ||
self.short_topic_name = transform.topic_name | ||
self.id_label = transform.id_label | ||
self.timestamp_attribute = transform.timestamp_attribute | ||
self.with_attributes = transform.with_attributes | ||
|
||
# TODO(https://github.com/apache/beam/issues/18939): Add support for | ||
# id_label and timestamp_attribute. | ||
if transform.id_label: | ||
raise NotImplementedError('id_label is not supported for PubSub writes') | ||
if transform.timestamp_attribute: | ||
raise NotImplementedError( | ||
'timestamp_attribute is not supported for PubSub writes') | ||
|
||
def start_bundle(self): | ||
self._buffer = [] | ||
|
||
def process(self, elem): | ||
self._buffer.append(elem) | ||
if len(self._buffer) >= self.BUFFER_SIZE_ELEMENTS: | ||
self._flush() | ||
|
||
def finish_bundle(self): | ||
self._flush() | ||
|
||
def _flush(self): | ||
if not self._buffer: | ||
return | ||
|
||
from google.cloud import pubsub | ||
import time | ||
|
||
pub_client = pubsub.PublisherClient() | ||
topic = pub_client.topic_path(self.project, self.short_topic_name) | ||
|
||
# The elements in buffer are already serialized bytes from the previous | ||
# transforms | ||
futures = [pub_client.publish(topic, elem) for elem in self._buffer] | ||
|
||
timer_start = time.time() | ||
for future in futures: | ||
remaining = self.FLUSH_TIMEOUT_SECS - (time.time() - timer_start) | ||
future.result(remaining) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. why is there a flush timeout? Completing processing without waiting for all of the messages to be consumed by pubsub could lead to data loss There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I added the timeout exception, which should trigger Dataflow to retry for batch jobs. The idea is to avoid any stuckness when publishing the messages. |
||
self._buffer = [] | ||
|
||
|
||
class _PubSubSink(object): | ||
"""Sink for a Cloud Pub/Sub topic. | ||
|
||
This ``NativeSource`` is overridden by a native Pubsub implementation. | ||
This sink works for both streaming and batch pipelines by using a DoFn | ||
that buffers and batches messages for efficient publishing. | ||
""" | ||
def __init__( | ||
self, | ||
|
Uh oh!
There was an error while loading. Please reload this page.