apache_beam.io.jdbc module

PTransforms for supporting Jdbc in Python pipelines.

These transforms are currently supported by Beam portable Flink, Spark, and Dataflow v2 runners.

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 Jdbc 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 Jdbc transforms. This option is only available for Beam 2.24.0 and later.

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 Jdbc transforms use the ‘beam-sdks-java-io-expansion-service’ jar for this purpose.

The transforms in this file support an extra classpath argument, where one can specify any extra JARs to be included in the classpath for the expansion service. Users will need to specify this option to include the JDBC driver for the database that you’re trying to use. By default, a Postgres JDBC driver is included (i.e. the Java package “org.postgresql:postgresql:42.2.16”).

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 Jdbc 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.jdbc.WriteToJdbc(table_name, driver_class_name, jdbc_url, username, password, statement=None, connection_properties=None, connection_init_sqls=None, autosharding=False, max_connections=None, driver_jars=None, expansion_service=None, classpath=None)[source]

Bases: ExternalTransform

A PTransform which writes Rows to the specified database via JDBC.

This transform receives Rows defined as NamedTuple type and registered in the coders registry, e.g.:

ExampleRow = typing.NamedTuple('ExampleRow',
                               [('id', int), ('name', unicode)])
coders.registry.register_coder(ExampleRow, coders.RowCoder)

with TestPipeline() as p:
  _ = (
      p
      | beam.Create([ExampleRow(1, 'abc')])
          .with_output_types(ExampleRow)
      | 'Write to jdbc' >> WriteToJdbc(
          table_name='jdbc_external_test_write'
          driver_class_name='org.postgresql.Driver',
          jdbc_url='jdbc:postgresql://localhost:5432/example',
          username='postgres',
          password='postgres',
      ))

table_name is a required paramater, and by default, the write_statement is generated from it.

The generated write_statement can be overridden by passing in a write_statment.

Experimental; no backwards compatibility guarantees.

Initializes a write operation to Jdbc.

Parameters:
  • driver_class_name – name of the jdbc driver class

  • jdbc_url – full jdbc url to the database.

  • username – database username

  • password – database password

  • statement – sql statement to be executed

  • connection_properties – properties of the jdbc connection passed as string with format [propertyName=property;]*

  • connection_init_sqls – required only for MySql and MariaDB. passed as list of strings

  • autosharding – enable automatic re-sharding of bundles to scale the number of shards with the number of workers.

  • max_connections – sets the maximum total number of connections. use a negative value for no limit.

  • driver_jars – comma separated paths for JDBC drivers. if not specified, the default classloader is used to load the driver jars.

  • expansion_service – The address (host:port) of the ExpansionService.

  • classpath – A list of JARs or Java packages to include in the classpath for the expansion service. This option is usually needed for jdbc to include extra JDBC driver packages. The packages can be in these three formats: (1) A local file, (2) A URL, (3) A gradle-style identifier of a Maven package (e.g. “org.postgresql:postgresql:42.3.1”). By default, this argument includes a Postgres SQL JDBC driver.

URN = 'beam:transform:org.apache.beam:schemaio_jdbc_write:v1'
class apache_beam.io.jdbc.ReadFromJdbc(table_name, driver_class_name, jdbc_url, username, password, query=None, output_parallelization=None, fetch_size=None, partition_column=None, partitions=None, connection_properties=None, connection_init_sqls=None, max_connections=None, driver_jars=None, expansion_service=None, classpath=None)[source]

Bases: ExternalTransform

A PTransform which reads Rows from the specified database via JDBC.

This transform delivers Rows defined as NamedTuple registered in the coders registry, e.g.:

ExampleRow = typing.NamedTuple('ExampleRow',
                               [('id', int), ('name', unicode)])
coders.registry.register_coder(ExampleRow, coders.RowCoder)

with TestPipeline() as p:
  result = (
      p
      | 'Read from jdbc' >> ReadFromJdbc(
          table_name='jdbc_external_test_read'
          driver_class_name='org.postgresql.Driver',
          jdbc_url='jdbc:postgresql://localhost:5432/example',
          username='postgres',
          password='postgres',
      ))

table_name is a required paramater, and by default, the read_query is generated from it.

The generated read_query can be overridden by passing in a read_query.

Experimental; no backwards compatibility guarantees.

Initializes a read operation from Jdbc.

Parameters:
  • driver_class_name – name of the jdbc driver class

  • jdbc_url – full jdbc url to the database.

  • username – database username

  • password – database password

  • query – sql query to be executed

  • output_parallelization – is output parallelization on

  • fetch_size – how many rows to fetch

  • partition_column – enable partitioned reads by splitting on this column

  • partitions – override the default number of splits when using partition_column

  • connection_properties – properties of the jdbc connection passed as string with format [propertyName=property;]*

  • connection_init_sqls – required only for MySql and MariaDB. passed as list of strings

  • max_connections – sets the maximum total number of connections. use a negative value for no limit.

  • driver_jars – comma separated paths for JDBC drivers. if not specified, the default classloader is used to load the driver jars.

  • expansion_service – The address (host:port) of the ExpansionService.

  • classpath – A list of JARs or Java packages to include in the classpath for the expansion service. This option is usually needed for jdbc to include extra JDBC driver packages. The packages can be in these three formats: (1) A local file, (2) A URL, (3) A gradle-style identifier of a Maven package (e.g. “org.postgresql:postgresql:42.3.1”). By default, this argument includes a Postgres SQL JDBC driver.

URN = 'beam:transform:org.apache.beam:schemaio_jdbc_read:v1'