kmjung commented on a change in pull request #15185:
URL: https://github.com/apache/beam/pull/15185#discussion_r678642676



##########
File path: sdks/python/apache_beam/io/gcp/bigquery.py
##########
@@ -883,6 +890,184 @@ def _export_files(self, bq):
     return table.schema, metadata_list
 
 
+class _CustomBigQueryStorageStreamSourceBase(BoundedSource):
+  """A base class for BoundedSource implementations which read from BigQuery
+  using the BigQuery Storage API."""
+  def __init__(
+      self,
+      project=None,
+      dataset=None,
+      table=None,
+      selected_fields=None,
+      row_restriction=None,
+      pipeline_options=None):
+
+    if table is None:
+      raise ValueError('A BigQuery table must be specified')
+    else:
+      self.table_reference = bigquery_tools.parse_table_reference(
+          table, dataset, project)
+
+    self.project = self.table_reference.projectId
+    self.dataset = self.table_reference.datasetId
+    self.table = self.table_reference.tableId
+    self.selected_fields = selected_fields
+    self.row_restriction = row_restriction
+    self.pipeline_options = pipeline_options
+    self.split_result = None
+    # The maximum number of streams which will be requested when creating a 
read
+    # session, regardless of the desired bundle size.
+    self.MAX_SPLIT_COUNT = 10000
+    # The minimum number of streams which will be requested when creating a 
read
+    # session, regardless of the desired bundle size. Note that the server may
+    # still choose to return fewer than ten streams based on the layout of the
+    # table.
+    self.MIN_SPLIT_COUNT = 10
+
+  def _get_project(self):
+    """Returns the project that will be billed."""
+    project = self.pipeline_options.view_as(GoogleCloudOptions).project
+    if isinstance(project, vp.ValueProvider):
+      project = project.get()
+    if not project:
+      project = self.project
+    return project
+
+  def _get_table_size(self, table, dataset, project):
+    if project is None:
+      project = self._get_project()
+
+    bq = bigquery_tools.BigQueryWrapper()
+    table = bq.get_table(project, dataset, table)
+    return table.numBytes
+
+  def display_data(self):
+    return {
+        'project': str(self.project),
+        'dataset': str(self.dataset),
+        'table': str(self.table),
+        'export_file_format': 'AVRO',
+    }
+
+  def estimate_size(self):
+    # The size of stream source cannot be estimate due to server-side liquid
+    # sharding
+    return None
+
+  def split(self, desired_bundle_size, start_position=None, 
stop_position=None):
+    requested_session = bq_storage.types.ReadSession()
+    requested_session.table = 'projects/{}/datasets/{}/tables/{}'.format(
+        self.project,
+        self.dataset,
+        self.table)
+    requested_session.data_format = bq_storage.types.DataFormat.AVRO
+
+    if (self.selected_fields is not None or self.row_restriction is not None):
+      table_read_options = requested_session.TableReadOptions()
+      if self.selected_fields is not None:
+        table_read_options.selected_fields = self.selected_fields
+      if self.row_restriction is not None:
+        table_read_options.row_restriction = self.row_restriction
+      requested_session.read_options = table_read_options
+
+    storage_client = bq_storage.BigQueryReadClient()
+    stream_count = 0
+    if (desired_bundle_size > 0):
+      stream_count = min(
+          int(
+              self._get_table_size(self.table, self.dataset, self.project) /
+              desired_bundle_size),
+          self.MAX_SPLIT_COUNT)
+
+    stream_count = max(stream_count, self.MIN_SPLIT_COUNT)
+    parent = 'projects/{}'.format(self.project)
+    read_session = storage_client.create_read_session(
+        parent=parent,
+        read_session=requested_session,
+        max_stream_count=stream_count)
+    self.split_result = [
+        _CustomBigQueryStorageStreamSource(stream.name)
+        for stream in read_session.streams
+    ]
+
+    for source in self.split_result:
+      yield SourceBundle(1.0, source, None, None)
+
+  def get_range_tracker(self, start_position, stop_position):
+    class CustomBigQuerySourceRangeTracker(RangeTracker):
+      """A RangeTracker that always returns positions as None."""
+      def start_position(self):
+        return None
+
+      def stop_position(self):
+        return None
+
+    return CustomBigQuerySourceRangeTracker()
+
+  def read(self, range_tracker):
+    raise NotImplementedError(
+        'BigQuery storage source must be split before being read')
+
+
+class _CustomBigQueryStorageStreamSource(BoundedSource):
+  """A source representing a single stream in a read session."""
+  def __init__(self, read_stream_name):
+    self.read_stream_name = read_stream_name
+    self.deserialized_rows = []
+
+  def display_data(self):
+    return {
+        'read_stream': str(self.read_stream_name),
+    }
+
+  def estimate_size(self):
+    # The size of stream source cannot be estimate due to server-side liquid
+    # sharding
+    return None
+
+  def split(self, desired_bundle_size, start_position=None, 
stop_position=None):
+    # A stream source can't be split without reading from it due to
+    # server-side liquid sharding.
+    raise NotImplementedError('BigQuery storage stream source cannot be 
split.')
+
+  def get_range_tracker(self, start_position, stop_position):
+    if start_position is None:
+      # Defaulting to the start of the stream.
+      start_position = 0
+
+    # Since the streams are unsplittable we choose OFFSET_INFINITY as the
+    # default end offset so that all data of the source gets read.
+    stop_position = range_trackers.OffsetRangeTracker.OFFSET_INFINITY
+    range_tracker = range_trackers.OffsetRangeTracker(
+        start_position, stop_position)
+    # Ensuring that all try_split() calls will be ignored by the Rangetracker.
+    range_tracker = range_trackers.UnsplittableRangeTracker(range_tracker)
+
+    return range_tracker
+
+  def deserialize_rows(self, read_rows_response, reader):

Review comment:
       Right, but in this CL you're processing the entire stream before 
returning any rows to the caller, materializing the full decoded rows list in 
memory, and then returning an iterator over the materialized rows list. This 
won't work as we scale to stream sizes larger than Dataflow worker memory. 
   
   An alternative would be to define your own custom iterator object which 
decoded exactly one record for each call to __next__, and which read the next 
ReadRowsResponse only when the end of the previous one was reached. I don't 
know if Beam Python has the equivalent of a BoundedReader -- it sounds like 
maybe it doesn't -- but this would effectively be the equivalent. See 
https://wiki.python.org/moin/Iterator for an idea of how this might work.

##########
File path: sdks/python/apache_beam/io/gcp/bigquery.py
##########
@@ -883,6 +890,184 @@ def _export_files(self, bq):
     return table.schema, metadata_list
 
 
+class _CustomBigQueryStorageStreamSourceBase(BoundedSource):
+  """A base class for BoundedSource implementations which read from BigQuery
+  using the BigQuery Storage API."""
+  def __init__(
+      self,
+      project=None,

Review comment:
       Your explanation makes sense.




-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to