[ https://issues.apache.org/jira/browse/BEAM-3342?focusedWorklogId=190190&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-190190 ]
ASF GitHub Bot logged work on BEAM-3342: ---------------------------------------- Author: ASF GitHub Bot Created on: 25/Jan/19 21:10 Start Date: 25/Jan/19 21:10 Worklog Time Spent: 10m Work Description: chamikaramj commented on pull request #7367: [BEAM-3342] Create a Cloud Bigtable Python connector Write URL: https://github.com/apache/beam/pull/7367#discussion_r251136921 ########## File path: sdks/python/apache_beam/io/gcp/bigtableio.py ########## @@ -0,0 +1,133 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +"""BigTable connector + +This module implements writing to BigTable tables. +The default mode is to set row data to write to BigTable tables. +The syntax supported is described here: +https://cloud.google.com/bigtable/docs/quickstart-cbt + +BigTable connector can be used as main outputs. A main output +(common case) is expected to be massive and will be split into +manageable chunks and processed in parallel. In the example below +we created a list of rows then passed to the GeneratedDirectRows +DoFn to set the Cells and then we call the BigTableWriteFn to insert +those generated rows in the table. + + main_table = (p + | 'Generate Direct Rows' >> GenerateDirectRows(number) + | 'Write to BT' >> beam.ParDo(BigTableWriteFn(config))) +""" +from __future__ import absolute_import + +import apache_beam as beam +from apache_beam.metrics import Metrics +from apache_beam.transforms.display import DisplayDataItem + +try: + from google.cloud.bigtable import Client +except ImportError: + pass + +__all__ = ['WriteToBigTable'] + + +class _BigTableWriteFn(beam.DoFn): + """ Creates the connector can call and add_row to the batcher using each + row in beam pipe line + + :type beam_options: class:`~bigtable_configuration.BigtableConfiguration` + :param beam_options: class `~bigtable_configuration.BigtableConfiguration` + """ + + def __init__(self, project_id, instance_id, table_id): + """ + Args: + project_id: GCP Project of to write the Rows + instance_id: GCP Instance to write the Rows + table_id: GCP Table to write the `DirectRows` + """ + super(_BigTableWriteFn, self).__init__() + self.beam_options = {'project_id': project_id, + 'instance_id': instance_id, + 'table_id': table_id} + self.table = None + self.batcher = None + self.written = Metrics.counter(self.__class__, 'Written Row') + + def __getstate__(self): + return self.beam_options + + def __setstate__(self, options): + self.beam_options = options + self.table = None + self.batcher = None + self.written = Metrics.counter(self.__class__, 'Written Row') + + def start_bundle(self): + if self.table is None: + client = Client(project=self.beam_options['project_id']) + instance = client.instance(self.beam_options['instance_id']) + self.table = instance.table(self.beam_options['table_id']) + self.batcher = self.table.mutations_batcher() + + def process(self, row): + self.written.inc() + # You need to set the timestamp in the cells in this row object, + # when we do a retry we will mutating the same object, but, with this + # we are going to set our cell with new values. + # Example: + # direct_row.set_cell('cf1', + # 'field1', + # 'value1', + # timestamp=datetime.datetime.now()) + self.batcher.mutate(row) + + def finish_bundle(self): + self.batcher.flush() + self.batcher = None + + def display_data(self): + return {'projectId': DisplayDataItem(self.beam_options['project_id'], + label='Bigtable Project Id'), + 'instanceId': DisplayDataItem(self.beam_options['instance_id'], + label='Bigtable Instance Id'), + 'tableId': DisplayDataItem(self.beam_options['table_id'], + label='Bigtable Table Id') + } + + +class WriteToBigTable(beam.PTransform): + """ A transform to write to the Bigtable Table. + + A PTransform that write a list of `DirectRow` into the Bigtable Table + + """ + def __init__(self, project_id=None, instance_id=None, Review comment: Please add a doc comment describing this public API. (probably update and move the comment in _BigTableWriteFn to here). ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on 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: 190190) Time Spent: 14h 10m (was: 14h) > Create a Cloud Bigtable Python connector > ---------------------------------------- > > 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: 14h 10m > Remaining Estimate: 0h > > I would like to create a Cloud Bigtable python connector. -- This message was sent by Atlassian JIRA (v7.6.3#76005)