[ https://issues.apache.org/jira/browse/BEAM-3342?focusedWorklogId=289305&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-289305 ]
ASF GitHub Bot logged work on BEAM-3342: ---------------------------------------- Author: ASF GitHub Bot Created on: 06/Aug/19 01:07 Start Date: 06/Aug/19 01:07 Worklog Time Spent: 10m Work Description: pabloem commented on pull request #8457: [BEAM-3342] Create a Cloud Bigtable IO connector for Python URL: https://github.com/apache/beam/pull/8457#discussion_r310842434 ########## File path: sdks/python/apache_beam/io/gcp/bigtableio.py ########## @@ -122,22 +129,148 @@ class WriteToBigTable(beam.PTransform): A PTransform that write a list of `DirectRow` into the Bigtable Table """ - def __init__(self, project_id=None, instance_id=None, - table_id=None): + def __init__(self, project_id=None, instance_id=None, table_id=None): """ The PTransform to access the Bigtable Write connector Args: project_id(str): GCP Project of to write the Rows instance_id(str): GCP Instance to write the Rows table_id(str): GCP Table to write the `DirectRows` """ super(WriteToBigTable, self).__init__() - self.beam_options = {'project_id': project_id, + self._beam_options = {'project_id': project_id, 'instance_id': instance_id, 'table_id': table_id} def expand(self, pvalue): - beam_options = self.beam_options + beam_options = self._beam_options return (pvalue | beam.ParDo(_BigTableWriteFn(beam_options['project_id'], beam_options['instance_id'], beam_options['table_id']))) + + +class _BigtableReadFn(beam.DoFn): + """ Creates the connector that can read rows for Beam pipeline + + Args: + project_id(str): GCP Project ID + instance_id(str): GCP Instance ID + table_id(str): GCP Table ID + + """ + + def __init__(self, project_id, instance_id, table_id, start_key=None, end_key=None, filter_=b''): + """ Constructor of the Read connector of Bigtable + + Args: + project_id: [str] GCP Project of to write the Rows + instance_id: [str] GCP Instance to write the Rows + table_id: [str] GCP Table to write the `DirectRows` + filter_: [RowFilter] Filter to apply to columns in a row. + """ + super(self.__class__, self).__init__() + self._initialize({'project_id': project_id, + 'instance_id': instance_id, + 'table_id': table_id, + 'start_key': start_key, + 'end_key': end_key, + 'filter_': filter_}) + + def __getstate__(self): + return self._beam_options + + def __setstate__(self, options): + self._initialize(options) + + def _initialize(self, options): + self._beam_options = options + self.table = None + self.sample_row_keys = None + self.row_count = Metrics.counter(self.__class__.__name__, 'Rows read') + + def start_bundle(self): + if self.table is None: + self.table = Client(project=self._beam_options['project_id'])\ + .instance(self._beam_options['instance_id'])\ + .table(self._beam_options['table_id']) + + def process(self, element, **kwargs): + for row in self.table.read_rows(start_key=self._beam_options['start_key'], + end_key=self._beam_options['end_key'], + filter_=self._beam_options['filter_']): + self.written.inc() Review comment: This metric doesn't seem to have been defined. Also, I'd think its name could be better. ---------------------------------------------------------------- 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org Issue Time Tracking ------------------- Worklog Id: (was: 289305) Time Spent: 30h 20m (was: 30h 10m) > Create a Cloud Bigtable IO connector for Python > ----------------------------------------------- > > Key: BEAM-3342 > URL: https://issues.apache.org/jira/browse/BEAM-3342 > Project: Beam > Issue Type: Bug > Components: sdk-py-core > Reporter: Solomon Duskis > Assignee: Solomon Duskis > Priority: Major > Time Spent: 30h 20m > Remaining Estimate: 0h > > I would like to create a Cloud Bigtable python connector. -- This message was sent by Atlassian JIRA (v7.6.14#76016)