github.com/apache/beam/sdks/v2@v2.48.2/python/apache_beam/io/snowflake.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 """Snowflake transforms tested against Flink portable runner. 19 20 **Setup** 21 22 Transforms provided in this module are cross-language transforms 23 implemented in the Beam Java SDK. During the pipeline construction, Python SDK 24 will connect to a Java expansion service to expand these transforms. 25 To facilitate this, a small amount of setup is needed before using these 26 transforms in a Beam Python pipeline. 27 28 There are several ways to setup cross-language Snowflake transforms. 29 30 * Option 1: use the default expansion service 31 * Option 2: specify a custom expansion service 32 33 See below for details regarding each of these options. 34 35 *Option 1: Use the default expansion service* 36 37 This is the recommended and easiest setup option for using Python Snowflake 38 transforms.This option requires following pre-requisites 39 before running the Beam pipeline. 40 41 * Install Java runtime in the computer from where the pipeline is constructed 42 and make sure that 'java' command is available. 43 44 In this option, Python SDK will either download (for released Beam version) or 45 build (when running from a Beam Git clone) a expansion service jar and use 46 that to expand transforms. Currently Snowflake transforms use the 47 'beam-sdks-java-io-snowflake-expansion-service' jar for this purpose. 48 49 *Option 2: specify a custom expansion service* 50 51 In this option, you startup your own expansion service and provide that as 52 a parameter when using the transforms provided in this module. 53 54 This option requires following pre-requisites before running the Beam 55 pipeline. 56 57 * Startup your own expansion service. 58 * Update your pipeline to provide the expansion service address when 59 initiating Snowflake transforms provided in this module. 60 61 Flink Users can use the built-in Expansion Service of the Flink Runner's 62 Job Server. If you start Flink's Job Server, the expansion service will be 63 started on port 8097. For a different address, please set the 64 expansion_service parameter. 65 66 **More information** 67 68 For more information regarding cross-language transforms see: 69 - https://beam.apache.org/roadmap/portability/ 70 71 For more information specific to Flink runner see: 72 - https://beam.apache.org/documentation/runners/flink/ 73 """ 74 75 # pytype: skip-file 76 77 from typing import Iterable 78 from typing import NamedTuple 79 from typing import Optional 80 81 import apache_beam as beam 82 from apache_beam.transforms.external import BeamJarExpansionService 83 from apache_beam.transforms.external import ExternalTransform 84 from apache_beam.transforms.external import NamedTupleBasedPayloadBuilder 85 86 __all__ = [ 87 'ReadFromSnowflake', 88 'WriteToSnowflake', 89 'WriteDisposition', 90 'CreateDisposition', 91 ] 92 93 94 def default_io_expansion_service(): 95 return BeamJarExpansionService( 96 'sdks:java:io:snowflake:expansion-service:shadowJar') 97 98 99 ReadFromSnowflakeSchema = NamedTuple( 100 'ReadFromSnowflakeSchema', 101 [ 102 ('server_name', str), 103 ('schema', str), 104 ('database', str), 105 ('staging_bucket_name', str), 106 ('storage_integration_name', str), 107 ('username', Optional[str]), 108 ('password', Optional[str]), 109 ('private_key_path', Optional[str]), 110 ('raw_private_key', Optional[str]), 111 ('private_key_passphrase', Optional[str]), 112 ('o_auth_token', Optional[str]), 113 ('table', Optional[str]), 114 ('query', Optional[str]), 115 ('role', Optional[str]), 116 ('warehouse', Optional[str]), 117 ]) 118 119 120 class ReadFromSnowflake(beam.PTransform): 121 """ 122 An external PTransform which reads from Snowflake. 123 """ 124 125 URN = 'beam:transform:org.apache.beam:snowflake_read:v1' 126 127 def __init__( 128 self, 129 server_name, 130 schema, 131 database, 132 staging_bucket_name, 133 storage_integration_name, 134 csv_mapper, 135 username=None, 136 password=None, 137 private_key_path=None, 138 raw_private_key=None, 139 private_key_passphrase=None, 140 o_auth_token=None, 141 table=None, 142 query=None, 143 role=None, 144 warehouse=None, 145 expansion_service=None): 146 """ 147 Initializes a read operation from Snowflake. 148 149 Required parameters: 150 151 :param server_name: full Snowflake server name with the following format 152 https://account.region.gcp.snowflakecomputing.com. 153 :param schema: name of the Snowflake schema in the database to use. 154 :param database: name of the Snowflake database to use. 155 :param staging_bucket_name: name of the Google Cloud Storage bucket. 156 Bucket will be used as a temporary location for storing CSV files. 157 Those temporary directories will be named 158 'sf_copy_csv_DATE_TIME_RANDOMSUFFIX' 159 and they will be removed automatically once Read operation finishes. 160 :param storage_integration_name: is the name of storage integration 161 object created according to Snowflake documentation. 162 :param csv_mapper: specifies a function which must translate 163 user-defined object to array of strings. 164 SnowflakeIO uses a COPY INTO <location> statement to move data from 165 a Snowflake table to Google Cloud Storage as CSV files.These files 166 are then downloaded via FileIO and processed line by line. 167 Each line is split into an array of Strings using the OpenCSV 168 The csv_mapper function job is to give the user the possibility to 169 convert the array of Strings to a user-defined type, 170 ie. GenericRecord for Avro or Parquet files, or custom objects. 171 Example: 172 def csv_mapper(strings_array) 173 return User(strings_array[0], int(strings_array[1]))) 174 :param table: specifies a Snowflake table name. 175 :param query: specifies a Snowflake custom SQL query. 176 :param role: specifies a Snowflake role. 177 :param warehouse: specifies a Snowflake warehouse name. 178 :param expansion_service: specifies URL of expansion service. 179 180 Authentication parameters: 181 182 :param username: specifies username for 183 username/password authentication method. 184 :param password: specifies password for 185 username/password authentication method. 186 :param private_key_path: specifies a private key file for 187 key/ pair authentication method. 188 :param raw_private_key: specifies a private key for 189 key/ pair authentication method. 190 :param private_key_passphrase: specifies password for 191 key/ pair authentication method. 192 :param o_auth_token: specifies access token for 193 OAuth authentication method. 194 """ 195 verify_credentials( 196 username=username, 197 password=password, 198 private_key_path=private_key_path, 199 raw_private_key=raw_private_key, 200 o_auth_token=o_auth_token, 201 ) 202 203 self.params = ReadFromSnowflakeSchema( 204 server_name=server_name, 205 schema=schema, 206 database=database, 207 staging_bucket_name=staging_bucket_name, 208 storage_integration_name=storage_integration_name, 209 username=username, 210 password=password, 211 private_key_path=private_key_path, 212 raw_private_key=raw_private_key, 213 private_key_passphrase=private_key_passphrase, 214 o_auth_token=o_auth_token, 215 table=table, 216 query=query, 217 role=role, 218 warehouse=warehouse, 219 ) 220 self.csv_mapper = csv_mapper 221 self.expansion_service = expansion_service or default_io_expansion_service() 222 223 def expand(self, pbegin): 224 return ( 225 pbegin 226 | ExternalTransform( 227 self.URN, 228 NamedTupleBasedPayloadBuilder(self.params), 229 self.expansion_service, 230 ) 231 | 'CSV to array mapper' >> beam.Map(lambda csv: csv.split(b',')) 232 | 'CSV mapper' >> beam.Map(self.csv_mapper)) 233 234 235 WriteToSnowflakeSchema = NamedTuple( 236 'WriteToSnowflakeSchema', 237 [ 238 ('server_name', str), 239 ('schema', str), 240 ('database', str), 241 ('staging_bucket_name', str), 242 ('storage_integration_name', str), 243 ('create_disposition', str), 244 ('write_disposition', str), 245 ('table_schema', str), 246 ('username', Optional[str]), 247 ('password', Optional[str]), 248 ('private_key_path', Optional[str]), 249 ('raw_private_key', Optional[str]), 250 ('private_key_passphrase', Optional[str]), 251 ('o_auth_token', Optional[str]), 252 ('table', Optional[str]), 253 ('query', Optional[str]), 254 ('role', Optional[str]), 255 ('warehouse', Optional[str]), 256 ], 257 ) 258 259 260 class WriteToSnowflake(beam.PTransform): 261 """ 262 An external PTransform which writes to Snowflake. 263 """ 264 265 URN = 'beam:transform:org.apache.beam:snowflake_write:v1' 266 267 def __init__( 268 self, 269 server_name, 270 schema, 271 database, 272 staging_bucket_name, 273 storage_integration_name, 274 create_disposition, 275 write_disposition, 276 table_schema, 277 user_data_mapper, 278 username=None, 279 password=None, 280 private_key_path=None, 281 raw_private_key=None, 282 private_key_passphrase=None, 283 o_auth_token=None, 284 table=None, 285 query=None, 286 role=None, 287 warehouse=None, 288 expansion_service=None, 289 ): 290 # pylint: disable=line-too-long 291 292 """ 293 Initializes a write operation to Snowflake. 294 295 Required parameters: 296 297 :param server_name: full Snowflake server name with the following format 298 https://account.region.gcp.snowflakecomputing.com. 299 :param schema: name of the Snowflake schema in the database to use. 300 :param database: name of the Snowflake database to use. 301 :param staging_bucket_name: name of the Google Cloud Storage bucket. 302 Bucket will be used as a temporary location for storing CSV files. 303 :param storage_integration_name: is the name of a Snowflake storage 304 integration object created according to Snowflake documentation for the 305 GCS bucket. 306 :param user_data_mapper: specifies a function which maps data from 307 a PCollection to an array of String values before the write operation 308 saves the data to temporary .csv files. 309 Example: 310 def user_data_mapper(user): 311 return [user.name, str(user.age)] 312 :param table: specifies a Snowflake table name 313 :param query: specifies a custom SQL query 314 :param role: specifies a Snowflake role. 315 :param warehouse: specifies a Snowflake warehouse name. 316 :param expansion_service: specifies URL of expansion service. 317 318 Authentication parameters: 319 320 :param username: specifies username for 321 username/password authentication method. 322 :param password: specifies password for 323 username/password authentication method. 324 :param private_key_path: specifies a private key file for 325 key/ pair authentication method. 326 :param raw_private_key: specifies a private key for 327 key/ pair authentication method. 328 :param private_key_passphrase: specifies password for 329 key/ pair authentication method. 330 :param o_auth_token: specifies access token for 331 OAuth authentication method. 332 333 Additional parameters: 334 335 :param create_disposition: Defines the behaviour of the write operation if 336 the target table does not exist. The following values are supported: 337 CREATE_IF_NEEDED - default behaviour. The write operation checks whether 338 the specified target table exists; if it does not, the write operation 339 attempts to create the table Specify the schema for the target table 340 using the table_schema parameter. 341 CREATE_NEVER - The write operation fails if the target table does not 342 exist. 343 :param write_disposition: Defines the write behaviour based on the table 344 where data will be written to. The following values are supported: 345 APPEND - Default behaviour. Written data is added to the existing rows 346 in the table, 347 EMPTY - The target table must be empty; otherwise, the write operation 348 fails, 349 TRUNCATE - The write operation deletes all rows from the target table 350 before writing to it. 351 :param table_schema: When the create_disposition parameter is set to 352 CREATE_IF_NEEDED, the table_schema parameter enables specifying the 353 schema for the created target table. A table schema is as JSON with the 354 following structure: 355 356 .. code-block:: none 357 358 {"schema":[ 359 { 360 "dataType":{"type":"<COLUMN DATA TYPE>"}, 361 "name":"<COLUMN NAME>", 362 "nullable": <NULLABLE> 363 }, 364 ]} 365 366 All supported data types: 367 368 .. code-block:: json 369 370 {"schema":[ 371 {"dataType":{"type":"date"},"name":"","nullable":false}, 372 {"dataType":{"type":"datetime"},"name":"","nullable":false}, 373 {"dataType":{"type":"time"},"name":"","nullable":false}, 374 {"dataType":{"type":"timestamp"},"name":"","nullable":false}, 375 {"dataType":{"type":"timestamp_ltz"},"name":"","nullable":false}, 376 {"dataType":{"type":"timestamp_ntz"},"name":"","nullable":false}, 377 {"dataType":{"type":"timestamp_tz"},"name":"","nullable":false}, 378 {"dataType":{"type":"boolean"},"name":"","nullable":false}, 379 {"dataType":{"type":"decimal","precision":38,"scale":1},"name":"","nullable":true}, 380 {"dataType":{"type":"double"},"name":"","nullable":false}, 381 {"dataType":{"type":"float"},"name":"","nullable":false}, 382 {"dataType":{"type":"integer","precision":38,"scale":0},"name":"","nullable":false}, 383 {"dataType":{"type":"number","precision":38,"scale":1},"name":"","nullable":false}, 384 {"dataType":{"type":"numeric","precision":38,"scale":2},"name":"","nullable":false}, 385 {"dataType":{"type":"real"},"name":"","nullable":false}, 386 {"dataType":{"type":"array"},"name":"","nullable":false}, 387 {"dataType":{"type":"object"},"name":"","nullable":false}, 388 {"dataType":{"type":"variant"},"name":"","nullable":true}, 389 {"dataType":{"type":"binary","size":null},"name":"","nullable":false}, 390 {"dataType":{"type":"char","length":1},"name":"","nullable":false}, 391 {"dataType":{"type":"string","length":null},"name":"","nullable":false}, 392 {"dataType":{"type":"text","length":null},"name":"","nullable":false}, 393 {"dataType":{"type":"varbinary","size":null},"name":"","nullable":false}, 394 {"dataType":{"type":"varchar","length":100},"name":"","nullable":false}, 395 {"dataType":{"type":"geography"},"name":"","nullable":true}] 396 } 397 """ 398 verify_credentials( 399 username=username, 400 password=password, 401 private_key_path=private_key_path, 402 raw_private_key=raw_private_key, 403 o_auth_token=o_auth_token, 404 ) 405 WriteDisposition.VerifyParam(write_disposition) 406 CreateDisposition.VerifyParam(create_disposition) 407 408 self.params = WriteToSnowflakeSchema( 409 server_name=server_name, 410 schema=schema, 411 database=database, 412 staging_bucket_name=staging_bucket_name, 413 storage_integration_name=storage_integration_name, 414 create_disposition=create_disposition, 415 write_disposition=write_disposition, 416 table_schema=table_schema, 417 username=username, 418 password=password, 419 private_key_path=private_key_path, 420 raw_private_key=raw_private_key, 421 private_key_passphrase=private_key_passphrase, 422 o_auth_token=o_auth_token, 423 table=table, 424 query=query, 425 role=role, 426 warehouse=warehouse, 427 ) 428 self.user_data_mapper = user_data_mapper 429 self.expansion_service = expansion_service or default_io_expansion_service() 430 431 def expand(self, pbegin): 432 return ( 433 pbegin 434 | 'User data mapper' >> beam.Map( 435 self.user_data_mapper).with_output_types(Iterable[bytes]) 436 | ExternalTransform( 437 self.URN, 438 NamedTupleBasedPayloadBuilder(self.params), 439 self.expansion_service)) 440 441 442 class CreateDisposition: 443 """ 444 Enum class for possible values of create dispositions: 445 CREATE_IF_NEEDED: default behaviour. The write operation checks whether 446 the specified target table exists; if it does not, the write operation 447 attempts to create the table Specify the schema for the target table 448 using the table_schema parameter. 449 CREATE_NEVER: The write operation fails if the target table does not exist. 450 """ 451 CREATE_IF_NEEDED = 'CREATE_IF_NEEDED' 452 CREATE_NEVER = 'CREATE_NEVER' 453 454 @staticmethod 455 def VerifyParam(field): 456 if field and not hasattr(CreateDisposition, field): 457 raise RuntimeError( 458 'Create disposition has to be one of the following values:' 459 'CREATE_IF_NEEDED, CREATE_NEVER. Got: {}'.format(field)) 460 461 462 class WriteDisposition: 463 """ 464 Enum class for possible values of write dispositions: 465 APPEND: Default behaviour. Written data is added to the existing rows 466 in the table, 467 EMPTY: The target table must be empty; otherwise, the write operation fails, 468 TRUNCATE: The write operation deletes all rows from the target table 469 before writing to it. 470 """ 471 APPEND = 'APPEND' 472 EMPTY = 'EMPTY' 473 TRUNCATE = 'TRUNCATE' 474 475 @staticmethod 476 def VerifyParam(field): 477 if field and not hasattr(WriteDisposition, field): 478 raise RuntimeError( 479 'Write disposition has to be one of the following values:' 480 'APPEND, EMPTY, TRUNCATE. Got: {}'.format(field)) 481 482 483 def verify_credentials( 484 username, password, private_key_path, raw_private_key, o_auth_token): 485 if not (o_auth_token or (username and password) or 486 (username and (private_key_path or raw_private_key))): 487 raise RuntimeError('Snowflake credentials are not set correctly.')