github.com/apache/beam/sdks/v2@v2.48.2/python/apache_beam/io/gcp/spanner.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 """PTransforms for supporting Spanner in Python pipelines. 19 20 These transforms are currently supported by Beam portable 21 Flink and Spark runners. 22 23 **Setup** 24 25 Transforms provided in this module are cross-language transforms 26 implemented in the Beam Java SDK. During the pipeline construction, Python SDK 27 will connect to a Java expansion service to expand these transforms. 28 To facilitate this, a small amount of setup is needed before using these 29 transforms in a Beam Python pipeline. 30 31 There are several ways to setup cross-language Spanner transforms. 32 33 * Option 1: use the default expansion service 34 * Option 2: specify a custom expansion service 35 36 See below for details regarding each of these options. 37 38 *Option 1: Use the default expansion service* 39 40 This is the recommended and easiest setup option for using Python Spanner 41 transforms. This option is only available for Beam 2.26.0 and later. 42 43 This option requires following pre-requisites before running the Beam 44 pipeline. 45 46 * Install Java runtime in the computer from where the pipeline is constructed 47 and make sure that 'java' command is available. 48 49 In this option, Python SDK will either download (for released Beam version) or 50 build (when running from a Beam Git clone) a expansion service jar and use 51 that to expand transforms. Currently Spanner transforms use the 52 'beam-sdks-java-io-google-cloud-platform-expansion-service' jar for this 53 purpose. 54 55 *Option 2: specify a custom expansion service* 56 57 In this option, you startup your own expansion service and provide that as 58 a parameter when using the transforms provided in this module. 59 60 This option requires following pre-requisites before running the Beam 61 pipeline. 62 63 * Startup your own expansion service. 64 * Update your pipeline to provide the expansion service address when 65 initiating Spanner transforms provided in this module. 66 67 Flink Users can use the built-in Expansion Service of the Flink Runner's 68 Job Server. If you start Flink's Job Server, the expansion service will be 69 started on port 8097. For a different address, please set the 70 expansion_service parameter. 71 72 **More information** 73 74 For more information regarding cross-language transforms see: 75 - https://beam.apache.org/roadmap/portability/ 76 77 For more information specific to Flink runner see: 78 - https://beam.apache.org/documentation/runners/flink/ 79 """ 80 81 # pytype: skip-file 82 83 from enum import Enum 84 from enum import auto 85 from typing import NamedTuple 86 from typing import Optional 87 88 from apache_beam.transforms.external import BeamJarExpansionService 89 from apache_beam.transforms.external import ExternalTransform 90 from apache_beam.transforms.external import NamedTupleBasedPayloadBuilder 91 from apache_beam.typehints.schemas import named_tuple_to_schema 92 93 __all__ = [ 94 'ReadFromSpanner', 95 'SpannerDelete', 96 'SpannerInsert', 97 'SpannerInsertOrUpdate', 98 'SpannerReplace', 99 'SpannerUpdate', 100 'TimestampBoundMode', 101 'TimeUnit', 102 ] 103 104 105 def default_io_expansion_service(): 106 return BeamJarExpansionService( 107 'sdks:java:io:google-cloud-platform:expansion-service:shadowJar') 108 109 110 class TimeUnit(Enum): 111 NANOSECONDS = auto() 112 MICROSECONDS = auto() 113 MILLISECONDS = auto() 114 SECONDS = auto() 115 HOURS = auto() 116 DAYS = auto() 117 118 119 class TimestampBoundMode(Enum): 120 MAX_STALENESS = auto() 121 EXACT_STALENESS = auto() 122 READ_TIMESTAMP = auto() 123 MIN_READ_TIMESTAMP = auto() 124 STRONG = auto() 125 126 127 class ReadFromSpannerSchema(NamedTuple): 128 instance_id: str 129 database_id: str 130 schema: bytes 131 sql: Optional[str] 132 table: Optional[str] 133 project_id: Optional[str] 134 host: Optional[str] 135 emulator_host: Optional[str] 136 batching: Optional[bool] 137 timestamp_bound_mode: Optional[str] 138 read_timestamp: Optional[str] 139 staleness: Optional[int] 140 time_unit: Optional[str] 141 142 143 class ReadFromSpanner(ExternalTransform): 144 """ 145 A PTransform which reads from the specified Spanner instance's database. 146 147 This transform required type of the row it has to return to provide the 148 schema. Example:: 149 150 from typing import NamedTuple 151 from apache_beam import coders 152 153 class ExampleRow(NamedTuple): 154 id: int 155 name: unicode 156 157 coders.registry.register_coder(ExampleRow, coders.RowCoder) 158 159 with Pipeline() as p: 160 result = ( 161 p 162 | ReadFromSpanner( 163 instance_id='your_instance_id', 164 database_id='your_database_id', 165 project_id='your_project_id', 166 row_type=ExampleRow, 167 sql='SELECT * FROM some_table', 168 timestamp_bound_mode=TimestampBoundMode.MAX_STALENESS, 169 staleness=3, 170 time_unit=TimeUnit.HOURS, 171 ).with_output_types(ExampleRow)) 172 173 Experimental; no backwards compatibility guarantees. 174 """ 175 176 URN = 'beam:transform:org.apache.beam:spanner_read:v1' 177 178 def __init__( 179 self, 180 project_id, 181 instance_id, 182 database_id, 183 row_type=None, 184 sql=None, 185 table=None, 186 host=None, 187 emulator_host=None, 188 batching=None, 189 timestamp_bound_mode=None, 190 read_timestamp=None, 191 staleness=None, 192 time_unit=None, 193 expansion_service=None, 194 ): 195 """ 196 Initializes a read operation from Spanner. 197 198 :param project_id: Specifies the Cloud Spanner project. 199 :param instance_id: Specifies the Cloud Spanner instance. 200 :param database_id: Specifies the Cloud Spanner database. 201 :param row_type: Row type that fits the given query or table. Passed as 202 NamedTuple, e.g. NamedTuple('name', [('row_name', unicode)]) 203 :param sql: An sql query to execute. It's results must fit the 204 provided row_type. Don't use when table is set. 205 :param table: A spanner table. When provided all columns from row_type 206 will be selected to query. Don't use when query is set. 207 :param batching: By default Batch API is used to read data from Cloud 208 Spanner. It is useful to disable batching when the underlying query 209 is not root-partitionable. 210 :param host: Specifies the Cloud Spanner host. 211 :param emulator_host: Specifies Spanner emulator host. 212 :param timestamp_bound_mode: Defines how Cloud Spanner will choose a 213 timestamp for a read-only transaction or a single read/query. 214 Passed as TimestampBoundMode enum. Possible values: 215 STRONG: A timestamp bound that will perform reads and queries at a 216 timestamp where all previously committed transactions are visible. 217 READ_TIMESTAMP: Returns a timestamp bound that will perform reads 218 and queries at the given timestamp. 219 MIN_READ_TIMESTAMP: Returns a timestamp bound that will perform reads 220 and queries at a timestamp chosen to be at least given timestamp value. 221 EXACT_STALENESS: Returns a timestamp bound that will perform reads and 222 queries at an exact staleness. The timestamp is chosen soon after the 223 read is started. 224 MAX_STALENESS: Returns a timestamp bound that will perform reads and 225 queries at a timestamp chosen to be at most time_unit stale. 226 :param read_timestamp: Timestamp in string. Use only when 227 timestamp_bound_mode is set to READ_TIMESTAMP or MIN_READ_TIMESTAMP. 228 :param staleness: Staleness value as int. Use only when 229 timestamp_bound_mode is set to EXACT_STALENESS or MAX_STALENESS. 230 time_unit has to be set along with this param. 231 :param time_unit: Time unit for staleness_value passed as TimeUnit enum. 232 Possible values: NANOSECONDS, MICROSECONDS, MILLISECONDS, SECONDS, 233 HOURS, DAYS. 234 :param expansion_service: The address (host:port) of the ExpansionService. 235 """ 236 assert row_type 237 assert sql or table and not (sql and table) 238 staleness_value = int(staleness) if staleness else None 239 240 if staleness_value or time_unit: 241 assert staleness_value and time_unit and \ 242 timestamp_bound_mode is TimestampBoundMode.MAX_STALENESS or \ 243 timestamp_bound_mode is TimestampBoundMode.EXACT_STALENESS 244 245 if read_timestamp: 246 assert timestamp_bound_mode is TimestampBoundMode.MIN_READ_TIMESTAMP\ 247 or timestamp_bound_mode is TimestampBoundMode.READ_TIMESTAMP 248 249 super().__init__( 250 self.URN, 251 NamedTupleBasedPayloadBuilder( 252 ReadFromSpannerSchema( 253 instance_id=instance_id, 254 database_id=database_id, 255 sql=sql, 256 table=table, 257 schema=named_tuple_to_schema(row_type).SerializeToString(), 258 project_id=project_id, 259 host=host, 260 emulator_host=emulator_host, 261 batching=batching, 262 timestamp_bound_mode=_get_enum_name(timestamp_bound_mode), 263 read_timestamp=read_timestamp, 264 staleness=staleness, 265 time_unit=_get_enum_name(time_unit), 266 ), 267 ), 268 expansion_service or default_io_expansion_service(), 269 ) 270 271 272 class WriteToSpannerSchema(NamedTuple): 273 project_id: str 274 instance_id: str 275 database_id: str 276 table: str 277 max_batch_size_bytes: Optional[int] 278 max_number_mutations: Optional[int] 279 max_number_rows: Optional[int] 280 grouping_factor: Optional[int] 281 host: Optional[str] 282 emulator_host: Optional[str] 283 commit_deadline: Optional[int] 284 max_cumulative_backoff: Optional[int] 285 286 287 _CLASS_DOC = \ 288 """ 289 A PTransform which writes {operation} mutations to the specified Spanner 290 table. 291 292 This transform receives rows defined as NamedTuple. Example:: 293 294 from typing import NamedTuple 295 from apache_beam import coders 296 297 class {row_type}(NamedTuple): 298 id: int 299 name: unicode 300 301 coders.registry.register_coder({row_type}, coders.RowCoder) 302 303 with Pipeline() as p: 304 _ = ( 305 p 306 | 'Impulse' >> beam.Impulse() 307 | 'Generate' >> beam.FlatMap(lambda x: range(num_rows)) 308 | 'To row' >> beam.Map(lambda n: {row_type}(n, str(n)) 309 .with_output_types({row_type}) 310 | 'Write to Spanner' >> Spanner{operation_suffix}( 311 instance_id='your_instance', 312 database_id='existing_database', 313 project_id='your_project_id', 314 table='your_table')) 315 316 Experimental; no backwards compatibility guarantees. 317 """ 318 319 _INIT_DOC = \ 320 """ 321 Initializes {operation} operation to a Spanner table. 322 323 :param project_id: Specifies the Cloud Spanner project. 324 :param instance_id: Specifies the Cloud Spanner instance. 325 :param database_id: Specifies the Cloud Spanner database. 326 :param table: Specifies the Cloud Spanner table. 327 :param max_batch_size_bytes: Specifies the batch size limit (max number of 328 bytes mutated per batch). Default value is 1048576 bytes = 1MB. 329 :param max_number_mutations: Specifies the cell mutation limit (maximum 330 number of mutated cells per batch). Default value is 5000. 331 :param max_number_rows: Specifies the row mutation limit (maximum number of 332 mutated rows per batch). Default value is 500. 333 :param grouping_factor: Specifies the multiple of max mutation (in terms 334 of both bytes per batch and cells per batch) that is used to select a 335 set of mutations to sort by key for batching. This sort uses local 336 memory on the workers, so using large values can cause out of memory 337 errors. Default value is 1000. 338 :param host: Specifies the Cloud Spanner host. 339 :param emulator_host: Specifies Spanner emulator host. 340 :param commit_deadline: Specifies the deadline for the Commit API call. 341 Default is 15 secs. DEADLINE_EXCEEDED errors will prompt a backoff/retry 342 until the value of commit_deadline is reached. DEADLINE_EXCEEDED errors 343 are ar reported with logging and counters. Pass seconds as value. 344 :param max_cumulative_backoff: Specifies the maximum cumulative backoff 345 time when retrying after DEADLINE_EXCEEDED errors. Default is 900s 346 (15min). If the mutations still have not been written after this time, 347 they are treated as a failure, and handled according to the setting of 348 failure_mode. Pass seconds as value. 349 :param expansion_service: The address (host:port) of the ExpansionService. 350 """ 351 352 353 def _add_doc( 354 value, 355 operation=None, 356 row_type=None, 357 operation_suffix=None, 358 ): 359 def _doc(obj): 360 obj.__doc__ = value.format( 361 operation=operation, 362 row_type=row_type, 363 operation_suffix=operation_suffix, 364 ) 365 return obj 366 367 return _doc 368 369 370 @_add_doc( 371 _CLASS_DOC, 372 operation='delete', 373 row_type='ExampleKey', 374 operation_suffix='Delete', 375 ) 376 class SpannerDelete(ExternalTransform): 377 378 URN = 'beam:transform:org.apache.beam:spanner_delete:v1' 379 380 @_add_doc(_INIT_DOC, operation='a delete') 381 def __init__( 382 self, 383 project_id, 384 instance_id, 385 database_id, 386 table, 387 max_batch_size_bytes=None, 388 max_number_mutations=None, 389 max_number_rows=None, 390 grouping_factor=None, 391 host=None, 392 emulator_host=None, 393 commit_deadline=None, 394 max_cumulative_backoff=None, 395 expansion_service=None, 396 ): 397 max_cumulative_backoff = int( 398 max_cumulative_backoff) if max_cumulative_backoff else None 399 commit_deadline = int(commit_deadline) if commit_deadline else None 400 super().__init__( 401 self.URN, 402 NamedTupleBasedPayloadBuilder( 403 WriteToSpannerSchema( 404 project_id=project_id, 405 instance_id=instance_id, 406 database_id=database_id, 407 table=table, 408 max_batch_size_bytes=max_batch_size_bytes, 409 max_number_mutations=max_number_mutations, 410 max_number_rows=max_number_rows, 411 grouping_factor=grouping_factor, 412 host=host, 413 emulator_host=emulator_host, 414 commit_deadline=commit_deadline, 415 max_cumulative_backoff=max_cumulative_backoff, 416 ), 417 ), 418 expansion_service=expansion_service or default_io_expansion_service(), 419 ) 420 421 422 @_add_doc( 423 _CLASS_DOC, 424 operation='insert', 425 row_type='ExampleRow', 426 operation_suffix='Insert', 427 ) 428 class SpannerInsert(ExternalTransform): 429 430 URN = 'beam:transform:org.apache.beam:spanner_insert:v1' 431 432 @_add_doc(_INIT_DOC, operation='an insert') 433 def __init__( 434 self, 435 project_id, 436 instance_id, 437 database_id, 438 table, 439 max_batch_size_bytes=None, 440 max_number_mutations=None, 441 max_number_rows=None, 442 grouping_factor=None, 443 host=None, 444 emulator_host=None, 445 commit_deadline=None, 446 max_cumulative_backoff=None, 447 expansion_service=None, 448 ): 449 max_cumulative_backoff = int( 450 max_cumulative_backoff) if max_cumulative_backoff else None 451 commit_deadline = int(commit_deadline) if commit_deadline else None 452 super().__init__( 453 self.URN, 454 NamedTupleBasedPayloadBuilder( 455 WriteToSpannerSchema( 456 project_id=project_id, 457 instance_id=instance_id, 458 database_id=database_id, 459 table=table, 460 max_batch_size_bytes=max_batch_size_bytes, 461 max_number_mutations=max_number_mutations, 462 max_number_rows=max_number_rows, 463 grouping_factor=grouping_factor, 464 host=host, 465 emulator_host=emulator_host, 466 commit_deadline=commit_deadline, 467 max_cumulative_backoff=max_cumulative_backoff, 468 ), 469 ), 470 expansion_service=expansion_service or default_io_expansion_service(), 471 ) 472 473 474 @_add_doc( 475 _CLASS_DOC, 476 operation='replace', 477 row_type='ExampleRow', 478 operation_suffix='Replace', 479 ) 480 class SpannerReplace(ExternalTransform): 481 482 URN = 'beam:transform:org.apache.beam:spanner_replace:v1' 483 484 @_add_doc(_INIT_DOC, operation='a replace') 485 def __init__( 486 self, 487 project_id, 488 instance_id, 489 database_id, 490 table, 491 max_batch_size_bytes=None, 492 max_number_mutations=None, 493 max_number_rows=None, 494 grouping_factor=None, 495 host=None, 496 emulator_host=None, 497 commit_deadline=None, 498 max_cumulative_backoff=None, 499 expansion_service=None, 500 ): 501 max_cumulative_backoff = int( 502 max_cumulative_backoff) if max_cumulative_backoff else None 503 commit_deadline = int(commit_deadline) if commit_deadline else None 504 super().__init__( 505 self.URN, 506 NamedTupleBasedPayloadBuilder( 507 WriteToSpannerSchema( 508 project_id=project_id, 509 instance_id=instance_id, 510 database_id=database_id, 511 table=table, 512 max_batch_size_bytes=max_batch_size_bytes, 513 max_number_mutations=max_number_mutations, 514 max_number_rows=max_number_rows, 515 grouping_factor=grouping_factor, 516 host=host, 517 emulator_host=emulator_host, 518 commit_deadline=commit_deadline, 519 max_cumulative_backoff=max_cumulative_backoff, 520 ), 521 ), 522 expansion_service=expansion_service or default_io_expansion_service(), 523 ) 524 525 526 @_add_doc( 527 _CLASS_DOC, 528 operation='insert-or-update', 529 row_type='ExampleRow', 530 operation_suffix='InsertOrUpdate', 531 ) 532 class SpannerInsertOrUpdate(ExternalTransform): 533 534 URN = 'beam:transform:org.apache.beam:spanner_insert_or_update:v1' 535 536 @_add_doc(_INIT_DOC, operation='an insert-or-update') 537 def __init__( 538 self, 539 project_id, 540 instance_id, 541 database_id, 542 table, 543 max_batch_size_bytes=None, 544 max_number_mutations=None, 545 max_number_rows=None, 546 grouping_factor=None, 547 host=None, 548 emulator_host=None, 549 commit_deadline=None, 550 max_cumulative_backoff=None, 551 expansion_service=None, 552 ): 553 max_cumulative_backoff = int( 554 max_cumulative_backoff) if max_cumulative_backoff else None 555 commit_deadline = int(commit_deadline) if commit_deadline else None 556 super().__init__( 557 self.URN, 558 NamedTupleBasedPayloadBuilder( 559 WriteToSpannerSchema( 560 project_id=project_id, 561 instance_id=instance_id, 562 database_id=database_id, 563 table=table, 564 max_batch_size_bytes=max_batch_size_bytes, 565 max_number_mutations=max_number_mutations, 566 max_number_rows=max_number_rows, 567 grouping_factor=grouping_factor, 568 host=host, 569 emulator_host=emulator_host, 570 commit_deadline=commit_deadline, 571 max_cumulative_backoff=max_cumulative_backoff, 572 ), 573 ), 574 expansion_service=expansion_service or default_io_expansion_service(), 575 ) 576 577 578 @_add_doc( 579 _CLASS_DOC, 580 operation='update', 581 row_type='ExampleRow', 582 operation_suffix='Update', 583 ) 584 class SpannerUpdate(ExternalTransform): 585 586 URN = 'beam:transform:org.apache.beam:spanner_update:v1' 587 588 @_add_doc(_INIT_DOC, operation='an update') 589 def __init__( 590 self, 591 project_id, 592 instance_id, 593 database_id, 594 table, 595 max_batch_size_bytes=None, 596 max_number_mutations=None, 597 max_number_rows=None, 598 grouping_factor=None, 599 host=None, 600 emulator_host=None, 601 commit_deadline=None, 602 max_cumulative_backoff=None, 603 expansion_service=None, 604 ): 605 max_cumulative_backoff = int( 606 max_cumulative_backoff) if max_cumulative_backoff else None 607 commit_deadline = int(commit_deadline) if commit_deadline else None 608 super().__init__( 609 self.URN, 610 NamedTupleBasedPayloadBuilder( 611 WriteToSpannerSchema( 612 project_id=project_id, 613 instance_id=instance_id, 614 database_id=database_id, 615 table=table, 616 max_batch_size_bytes=max_batch_size_bytes, 617 max_number_mutations=max_number_mutations, 618 max_number_rows=max_number_rows, 619 grouping_factor=grouping_factor, 620 host=host, 621 emulator_host=emulator_host, 622 commit_deadline=commit_deadline, 623 max_cumulative_backoff=max_cumulative_backoff, 624 ), 625 ), 626 expansion_service=expansion_service or default_io_expansion_service(), 627 ) 628 629 630 def _get_enum_name(enum): 631 return None if enum is None else enum.name