github.com/apache/beam/sdks/v2@v2.48.2/python/apache_beam/io/gcp/bigtableio.py (about) 1 # 2 # Licensed to the Apache Software Foundation (ASF) under one or more 3 # contributor license agreements. See the NOTICE file distributed with 4 # this work for additional information regarding copyright ownership. 5 # The ASF licenses this file to You under the Apache License, Version 2.0 6 # (the "License"); you may not use this file except in compliance with 7 # the License. You may obtain a copy of the License at 8 # 9 # http://www.apache.org/licenses/LICENSE-2.0 10 # 11 # Unless required by applicable law or agreed to in writing, software 12 # distributed under the License is distributed on an "AS IS" BASIS, 13 # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 # See the License for the specific language governing permissions and 15 # limitations under the License. 16 # 17 18 """BigTable connector 19 20 This module implements writing to BigTable tables. 21 The default mode is to set row data to write to BigTable tables. 22 The syntax supported is described here: 23 https://cloud.google.com/bigtable/docs/quickstart-cbt 24 25 BigTable connector can be used as main outputs. A main output 26 (common case) is expected to be massive and will be split into 27 manageable chunks and processed in parallel. In the example below 28 we created a list of rows then passed to the GeneratedDirectRows 29 DoFn to set the Cells and then we call the BigTableWriteFn to insert 30 those generated rows in the table. 31 32 main_table = (p 33 | beam.Create(self._generate()) 34 | WriteToBigTable(project_id, 35 instance_id, 36 table_id)) 37 """ 38 # pytype: skip-file 39 40 import logging 41 42 import apache_beam as beam 43 from apache_beam.internal.metrics.metric import ServiceCallMetric 44 from apache_beam.io.gcp import resource_identifiers 45 from apache_beam.metrics import Metrics 46 from apache_beam.metrics import monitoring_infos 47 from apache_beam.transforms.display import DisplayDataItem 48 49 _LOGGER = logging.getLogger(__name__) 50 51 try: 52 from google.cloud.bigtable import Client 53 from google.cloud.bigtable.batcher import MutationsBatcher 54 55 FLUSH_COUNT = 1000 56 MAX_ROW_BYTES = 5242880 # 5MB 57 58 class _MutationsBatcher(MutationsBatcher): 59 def __init__( 60 self, table, flush_count=FLUSH_COUNT, max_row_bytes=MAX_ROW_BYTES): 61 super().__init__(table, flush_count, max_row_bytes) 62 self.rows = [] 63 64 def set_flush_callback(self, callback_fn): 65 self.callback_fn = callback_fn 66 67 def flush(self): 68 if len(self.rows) != 0: 69 status_list = self.table.mutate_rows(self.rows) 70 self.callback_fn(status_list) 71 72 # If even one request fails we retry everything. BigTable mutations are 73 # idempotent so this should be correct. 74 # TODO(https://github.com/apache/beam/issues/21396): make this more 75 # efficient by retrying only re-triable failed requests. 76 for status in status_list: 77 if not status: 78 # BigTable client may return 'None' instead of a valid status in 79 # some cases due to 80 # https://github.com/googleapis/python-bigtable/issues/485 81 raise Exception( 82 'Failed to write a batch of %r records' % len(self.rows)) 83 elif status.code != 0: 84 raise Exception( 85 'Failed to write a batch of %r records due to %r' % ( 86 len(self.rows), 87 ServiceCallMetric.bigtable_error_code_to_grpc_status_string( 88 status.code))) 89 90 self.total_mutation_count = 0 91 self.total_size = 0 92 self.rows = [] 93 94 except ImportError: 95 _LOGGER.warning( 96 'ImportError: from google.cloud.bigtable import Client', exc_info=True) 97 98 __all__ = ['WriteToBigTable'] 99 100 101 class _BigTableWriteFn(beam.DoFn): 102 """ Creates the connector can call and add_row to the batcher using each 103 row in beam pipe line 104 Args: 105 project_id(str): GCP Project ID 106 instance_id(str): GCP Instance ID 107 table_id(str): GCP Table ID 108 109 """ 110 def __init__(self, project_id, instance_id, table_id): 111 """ Constructor of the Write connector of Bigtable 112 Args: 113 project_id(str): GCP Project of to write the Rows 114 instance_id(str): GCP Instance to write the Rows 115 table_id(str): GCP Table to write the `DirectRows` 116 """ 117 super().__init__() 118 self.beam_options = { 119 'project_id': project_id, 120 'instance_id': instance_id, 121 'table_id': table_id 122 } 123 self.table = None 124 self.batcher = None 125 self.service_call_metric = None 126 self.written = Metrics.counter(self.__class__, 'Written Row') 127 128 def __getstate__(self): 129 return self.beam_options 130 131 def __setstate__(self, options): 132 self.beam_options = options 133 self.table = None 134 self.batcher = None 135 self.service_call_metric = None 136 self.written = Metrics.counter(self.__class__, 'Written Row') 137 138 def write_mutate_metrics(self, status_list): 139 for status in status_list: 140 code = status.code if status else None 141 grpc_status_string = ( 142 ServiceCallMetric.bigtable_error_code_to_grpc_status_string(code)) 143 self.service_call_metric.call(grpc_status_string) 144 145 def start_service_call_metrics(self, project_id, instance_id, table_id): 146 resource = resource_identifiers.BigtableTable( 147 project_id, instance_id, table_id) 148 labels = { 149 monitoring_infos.SERVICE_LABEL: 'BigTable', 150 # TODO(JIRA-11985): Add Ptransform label. 151 monitoring_infos.METHOD_LABEL: 'google.bigtable.v2.MutateRows', 152 monitoring_infos.RESOURCE_LABEL: resource, 153 monitoring_infos.BIGTABLE_PROJECT_ID_LABEL: ( 154 self.beam_options['project_id']), 155 monitoring_infos.INSTANCE_ID_LABEL: self.beam_options['instance_id'], 156 monitoring_infos.TABLE_ID_LABEL: self.beam_options['table_id'] 157 } 158 return ServiceCallMetric( 159 request_count_urn=monitoring_infos.API_REQUEST_COUNT_URN, 160 base_labels=labels) 161 162 def start_bundle(self): 163 if self.table is None: 164 client = Client(project=self.beam_options['project_id']) 165 instance = client.instance(self.beam_options['instance_id']) 166 self.table = instance.table(self.beam_options['table_id']) 167 self.service_call_metric = self.start_service_call_metrics( 168 self.beam_options['project_id'], 169 self.beam_options['instance_id'], 170 self.beam_options['table_id']) 171 self.batcher = _MutationsBatcher(self.table) 172 self.batcher.set_flush_callback(self.write_mutate_metrics) 173 174 def process(self, row): 175 self.written.inc() 176 # You need to set the timestamp in the cells in this row object, 177 # when we do a retry we will mutating the same object, but, with this 178 # we are going to set our cell with new values. 179 # Example: 180 # direct_row.set_cell('cf1', 181 # 'field1', 182 # 'value1', 183 # timestamp=datetime.datetime.now()) 184 self.batcher.mutate(row) 185 186 def finish_bundle(self): 187 self.batcher.flush() 188 self.batcher = None 189 190 def display_data(self): 191 return { 192 'projectId': DisplayDataItem( 193 self.beam_options['project_id'], label='Bigtable Project Id'), 194 'instanceId': DisplayDataItem( 195 self.beam_options['instance_id'], label='Bigtable Instance Id'), 196 'tableId': DisplayDataItem( 197 self.beam_options['table_id'], label='Bigtable Table Id') 198 } 199 200 201 class WriteToBigTable(beam.PTransform): 202 """ A transform to write to the Bigtable Table. 203 204 A PTransform that write a list of `DirectRow` into the Bigtable Table 205 206 """ 207 def __init__(self, project_id=None, instance_id=None, table_id=None): 208 """ The PTransform to access the Bigtable Write connector 209 Args: 210 project_id(str): GCP Project of to write the Rows 211 instance_id(str): GCP Instance to write the Rows 212 table_id(str): GCP Table to write the `DirectRows` 213 """ 214 super().__init__() 215 self.beam_options = { 216 'project_id': project_id, 217 'instance_id': instance_id, 218 'table_id': table_id 219 } 220 221 def expand(self, pvalue): 222 beam_options = self.beam_options 223 return ( 224 pvalue 225 | beam.ParDo( 226 _BigTableWriteFn( 227 beam_options['project_id'], 228 beam_options['instance_id'], 229 beam_options['table_id'])))