vachan-shetty commented on a change in pull request #15185:
URL: https://github.com/apache/beam/pull/15185#discussion_r680207651



##########
File path: sdks/python/apache_beam/io/gcp/bigquery.py
##########
@@ -883,6 +890,206 @@ def _export_files(self, bq):
     return table.schema, metadata_list
 
 
+class _CustomBigQueryStorageSourceBase(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 dataset is None:
+      raise ValueError('A BigQuery dataset must be specified.')
+    elif 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)
+    }
+
+  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()

Review comment:
       Using pysize [1] each instance of the client is ~ 16,000 bytes. The 
serialization/deserialization that occurs in BEAM pipelines prevents us from 
being able to re-use the same instance of the client object.
   
   `split` is invoked whenever the source needs to be split into a smaller 
bundle which depends on the characteristics of the pipelines.
   
   [1]: https://github.com/bosswissam/pysize/blob/master/pysize.py




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