apache_beam.io.snowflake module¶
Snowflake transforms tested against Flink portable runner.
Setup
Transforms provided in this module are cross-language transforms implemented in the Beam Java SDK. During the pipeline construction, Python SDK will connect to a Java expansion service to expand these transforms. To facilitate this, a small amount of setup is needed before using these transforms in a Beam Python pipeline.
There are several ways to setup cross-language Snowflake transforms.
Option 1: use the default expansion service
Option 2: specify a custom expansion service
See below for details regarding each of these options.
Option 1: Use the default expansion service
This is the recommended and easiest setup option for using Python Snowflake transforms.This option requires following pre-requisites before running the Beam pipeline.
Install Java runtime in the computer from where the pipeline is constructed and make sure that ‘java’ command is available.
In this option, Python SDK will either download (for released Beam version) or build (when running from a Beam Git clone) a expansion service jar and use that to expand transforms. Currently Snowflake transforms use the ‘beam-sdks-java-io-snowflake-expansion-service’ jar for this purpose.
Option 2: specify a custom expansion service
In this option, you startup your own expansion service and provide that as a parameter when using the transforms provided in this module.
This option requires following pre-requisites before running the Beam pipeline.
Startup your own expansion service.
Update your pipeline to provide the expansion service address when initiating Snowflake transforms provided in this module.
Flink Users can use the built-in Expansion Service of the Flink Runner’s Job Server. If you start Flink’s Job Server, the expansion service will be started on port 8097. For a different address, please set the expansion_service parameter.
More information
For more information regarding cross-language transforms see: - https://beam.apache.org/roadmap/portability/
For more information specific to Flink runner see: - https://beam.apache.org/documentation/runners/flink/
- class apache_beam.io.snowflake.ReadFromSnowflake(server_name, schema, database, staging_bucket_name, storage_integration_name, csv_mapper, username=None, password=None, private_key_path=None, raw_private_key=None, private_key_passphrase=None, o_auth_token=None, table=None, query=None, role=None, warehouse=None, expansion_service=None)[source]¶
Bases:
PTransform
An external PTransform which reads from Snowflake.
Initializes a read operation from Snowflake.
Required parameters:
- Parameters:
server_name – full Snowflake server name with the following format https://account.region.gcp.snowflakecomputing.com.
schema – name of the Snowflake schema in the database to use.
database – name of the Snowflake database to use.
staging_bucket_name – name of the Google Cloud Storage bucket. Bucket will be used as a temporary location for storing CSV files. Those temporary directories will be named ‘sf_copy_csv_DATE_TIME_RANDOMSUFFIX’ and they will be removed automatically once Read operation finishes.
storage_integration_name – is the name of storage integration object created according to Snowflake documentation.
csv_mapper – specifies a function which must translate user-defined object to array of strings. SnowflakeIO uses a COPY INTO <location> statement to move data from a Snowflake table to Google Cloud Storage as CSV files.These files are then downloaded via FileIO and processed line by line. Each line is split into an array of Strings using the OpenCSV The csv_mapper function job is to give the user the possibility to convert the array of Strings to a user-defined type, ie. GenericRecord for Avro or Parquet files, or custom objects. Example: def csv_mapper(strings_array) return User(strings_array[0], int(strings_array[1])))
table – specifies a Snowflake table name.
query – specifies a Snowflake custom SQL query.
role – specifies a Snowflake role.
warehouse – specifies a Snowflake warehouse name.
expansion_service – specifies URL of expansion service.
Authentication parameters:
- Parameters:
username – specifies username for username/password authentication method.
password – specifies password for username/password authentication method.
private_key_path – specifies a private key file for key/ pair authentication method.
raw_private_key – specifies a private key for key/ pair authentication method.
private_key_passphrase – specifies password for key/ pair authentication method.
o_auth_token – specifies access token for OAuth authentication method.
- URN = 'beam:transform:org.apache.beam:snowflake_read:v1'¶
- class apache_beam.io.snowflake.WriteToSnowflake(server_name, schema, database, staging_bucket_name, storage_integration_name, create_disposition, write_disposition, table_schema, user_data_mapper, username=None, password=None, private_key_path=None, raw_private_key=None, private_key_passphrase=None, o_auth_token=None, table=None, query=None, role=None, warehouse=None, expansion_service=None)[source]¶
Bases:
PTransform
An external PTransform which writes to Snowflake.
Initializes a write operation to Snowflake.
Required parameters:
- Parameters:
server_name – full Snowflake server name with the following format https://account.region.gcp.snowflakecomputing.com.
schema – name of the Snowflake schema in the database to use.
database – name of the Snowflake database to use.
staging_bucket_name – name of the Google Cloud Storage bucket. Bucket will be used as a temporary location for storing CSV files.
storage_integration_name – is the name of a Snowflake storage integration object created according to Snowflake documentation for the GCS bucket.
user_data_mapper – specifies a function which maps data from a PCollection to an array of String values before the write operation saves the data to temporary .csv files. Example: def user_data_mapper(user): return [user.name, str(user.age)]
table – specifies a Snowflake table name
query – specifies a custom SQL query
role – specifies a Snowflake role.
warehouse – specifies a Snowflake warehouse name.
expansion_service – specifies URL of expansion service.
Authentication parameters:
- Parameters:
username – specifies username for username/password authentication method.
password – specifies password for username/password authentication method.
private_key_path – specifies a private key file for key/ pair authentication method.
raw_private_key – specifies a private key for key/ pair authentication method.
private_key_passphrase – specifies password for key/ pair authentication method.
o_auth_token – specifies access token for OAuth authentication method.
Additional parameters:
- Parameters:
create_disposition – Defines the behaviour of the write operation if the target table does not exist. The following values are supported: CREATE_IF_NEEDED - default behaviour. The write operation checks whether the specified target table exists; if it does not, the write operation attempts to create the table Specify the schema for the target table using the table_schema parameter. CREATE_NEVER - The write operation fails if the target table does not exist.
write_disposition – Defines the write behaviour based on the table where data will be written to. The following values are supported: APPEND - Default behaviour. Written data is added to the existing rows in the table, EMPTY - The target table must be empty; otherwise, the write operation fails, TRUNCATE - The write operation deletes all rows from the target table before writing to it.
table_schema –
When the create_disposition parameter is set to CREATE_IF_NEEDED, the table_schema parameter enables specifying the schema for the created target table. A table schema is as JSON with the following structure:
{"schema":[ { "dataType":{"type":"<COLUMN DATA TYPE>"}, "name":"<COLUMN NAME>", "nullable": <NULLABLE> }, ]}
All supported data types:
{"schema":[ {"dataType":{"type":"date"},"name":"","nullable":false}, {"dataType":{"type":"datetime"},"name":"","nullable":false}, {"dataType":{"type":"time"},"name":"","nullable":false}, {"dataType":{"type":"timestamp"},"name":"","nullable":false}, {"dataType":{"type":"timestamp_ltz"},"name":"","nullable":false}, {"dataType":{"type":"timestamp_ntz"},"name":"","nullable":false}, {"dataType":{"type":"timestamp_tz"},"name":"","nullable":false}, {"dataType":{"type":"boolean"},"name":"","nullable":false}, {"dataType":{"type":"decimal","precision":38,"scale":1},"name":"","nullable":true}, {"dataType":{"type":"double"},"name":"","nullable":false}, {"dataType":{"type":"float"},"name":"","nullable":false}, {"dataType":{"type":"integer","precision":38,"scale":0},"name":"","nullable":false}, {"dataType":{"type":"number","precision":38,"scale":1},"name":"","nullable":false}, {"dataType":{"type":"numeric","precision":38,"scale":2},"name":"","nullable":false}, {"dataType":{"type":"real"},"name":"","nullable":false}, {"dataType":{"type":"array"},"name":"","nullable":false}, {"dataType":{"type":"object"},"name":"","nullable":false}, {"dataType":{"type":"variant"},"name":"","nullable":true}, {"dataType":{"type":"binary","size":null},"name":"","nullable":false}, {"dataType":{"type":"char","length":1},"name":"","nullable":false}, {"dataType":{"type":"string","length":null},"name":"","nullable":false}, {"dataType":{"type":"text","length":null},"name":"","nullable":false}, {"dataType":{"type":"varbinary","size":null},"name":"","nullable":false}, {"dataType":{"type":"varchar","length":100},"name":"","nullable":false}, {"dataType":{"type":"geography"},"name":"","nullable":true}] }
- URN = 'beam:transform:org.apache.beam:snowflake_write:v1'¶
- class apache_beam.io.snowflake.WriteDisposition[source]¶
Bases:
object
Enum class for possible values of write dispositions: APPEND: Default behaviour. Written data is added to the existing rows in the table, EMPTY: The target table must be empty; otherwise, the write operation fails, TRUNCATE: The write operation deletes all rows from the target table before writing to it.
- APPEND = 'APPEND'¶
- EMPTY = 'EMPTY'¶
- TRUNCATE = 'TRUNCATE'¶
- class apache_beam.io.snowflake.CreateDisposition[source]¶
Bases:
object
Enum class for possible values of create dispositions: CREATE_IF_NEEDED: default behaviour. The write operation checks whether the specified target table exists; if it does not, the write operation attempts to create the table Specify the schema for the target table using the table_schema parameter. CREATE_NEVER: The write operation fails if the target table does not exist.
- CREATE_IF_NEEDED = 'CREATE_IF_NEEDED'¶
- CREATE_NEVER = 'CREATE_NEVER'¶