TheNeuralBit commented on a change in pull request #12145: URL: https://github.com/apache/beam/pull/12145#discussion_r452418029
########## File path: CHANGES.md ########## @@ -78,6 +78,8 @@ is experimental. It reads data from BigQuery by exporting data to Avro files, and reading those files. It also supports reading data by exporting to JSON files. This has small differences in behavior for Time and Date-related fields. See Pydoc for more information. +* Add cross-language support to JdbcIO.ReadRows([BEAM-10135](https://issues.apache.org/jira/browse/BEAM-10135)). +* Add cross-language support to JdbcIO.Write([BEAM-10136](https://issues.apache.org/jira/browse/BEAM-10136)). Review comment: Looks like this will be in 2.24.0 ########## File path: sdks/java/container/boot.go ########## @@ -122,6 +122,7 @@ func main() { filepath.Join(jarsDir, "beam-sdks-java-harness.jar"), filepath.Join(jarsDir, "beam-sdks-java-io-kafka.jar"), filepath.Join(jarsDir, "kafka-clients.jar"), + filepath.Join(jarsDir, "beam-sdks-java-io-jdbc.jar"), Review comment: I think this change and the changes in sdks/java/container/build.gradle are no longer necessary (like the Dockerfile one from here: https://github.com/apache/beam/pull/12022#discussion_r447186177). I think they were required before we had artifact staging. ########## File path: build.gradle ########## @@ -273,6 +273,7 @@ task python37PostCommit() { task python38PostCommit() { dependsOn ":sdks:python:test-suites:portable:py38:crossLanguagePythonJavaKafkaIOFlink" + dependsOn ":sdks:python:test-suites:portable:py38:crossLanguagePythonJavaJdbcIO" Review comment: Ah sorry, I see now that this is how you know it's running on Flink and Spark :) ########## File path: sdks/python/apache_beam/io/external/jdbc.py ########## @@ -0,0 +1,254 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +"""PTransforms for supporting Jdbc in Python pipelines. + + These transforms are currently supported by Beam portable runners (for + example, portable Flink and Spark) as well as Dataflow 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 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.22.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. + + *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/ +""" + +# pytype: skip-file + +from __future__ import absolute_import + +import typing + +from past.builtins import unicode + +from apache_beam.transforms.external import BeamJarExpansionService +from apache_beam.transforms.external import ExternalTransform +from apache_beam.transforms.external import NamedTupleBasedPayloadBuilder + +__all__ = [ + 'WriteToJdbc', + 'ReadFromJdbc', +] + + +def default_io_expansion_service(): + return BeamJarExpansionService('sdks:java:io:expansion-service:shadowJar') + + +WriteToJdbcSchema = typing.NamedTuple( + 'WriteToJdbcSchema', + [ + ('driver_class_name', unicode), + ('jdbc_url', unicode), + ('username', unicode), + ('password', unicode), + ('connection_properties', typing.Optional[unicode]), + ('connection_init_sqls', typing.Optional[typing.List[unicode]]), + ('statement', unicode), + ], +) + + +class WriteToJdbc(ExternalTransform): + """An external PTransform which writes Rows to the specified database. Review comment: ```suggestion """A PTransform which writes Rows to the specified database via JDBC. ``` ########## File path: sdks/python/apache_beam/io/external/jdbc.py ########## @@ -0,0 +1,254 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +"""PTransforms for supporting Jdbc in Python pipelines. + + These transforms are currently supported by Beam portable runners (for + example, portable Flink and Spark) as well as Dataflow 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 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.22.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. + + *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/ Review comment: This looks great, thanks! ########## File path: sdks/java/io/jdbc/build.gradle ########## @@ -25,6 +25,7 @@ description = "Apache Beam :: SDKs :: Java :: IO :: JDBC" ext.summary = "IO to read and write on JDBC datasource." dependencies { + compile library.java.postgres Review comment: Could you add this dependency in :sdks:java:io:expansion-service instead? I think that would have the same effect, but saves users who don't need it from pulling it in. ########## File path: buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy ########## @@ -1982,17 +1982,25 @@ class BeamModulePlugin implements Plugin<Project> { return argList.join(' ') } - project.ext.toxTask = { name, tox_env -> + project.ext.toxTask = { name, tox_env, needsExpansionServiceJar = true -> Review comment: wouldn't it be preferable to make false the default? I'd think most tox tasks do not need this. ########## File path: sdks/python/apache_beam/io/external/jdbc.py ########## @@ -0,0 +1,254 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +"""PTransforms for supporting Jdbc in Python pipelines. + + These transforms are currently supported by Beam portable runners (for + example, portable Flink and Spark) as well as Dataflow 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 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.22.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. + + *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/ +""" + +# pytype: skip-file + +from __future__ import absolute_import + +import typing + +from past.builtins import unicode + +from apache_beam.transforms.external import BeamJarExpansionService +from apache_beam.transforms.external import ExternalTransform +from apache_beam.transforms.external import NamedTupleBasedPayloadBuilder + +__all__ = [ + 'WriteToJdbc', + 'ReadFromJdbc', +] + + +def default_io_expansion_service(): + return BeamJarExpansionService('sdks:java:io:expansion-service:shadowJar') + + +WriteToJdbcSchema = typing.NamedTuple( + 'WriteToJdbcSchema', + [ + ('driver_class_name', unicode), + ('jdbc_url', unicode), + ('username', unicode), + ('password', unicode), + ('connection_properties', typing.Optional[unicode]), + ('connection_init_sqls', typing.Optional[typing.List[unicode]]), + ('statement', unicode), + ], +) + + +class WriteToJdbc(ExternalTransform): + """An external PTransform which writes Rows to the specified database. + + 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) + + An example can be found in + `apache_beam.examples.xlang_jdbcio_it_test` + """ + + URN = 'beam:external:java:jdbc:write:v1' + + def __init__( + self, + driver_class_name, + jdbc_url, + username, + password, + statement, + connection_properties=None, + connection_init_sqls=None, + expansion_service=None, + ): + """ + Initializes a write operation to Jdbc. + + :param driver_class_name: name of the jdbc driver class + :param jdbc_url: full jdbc url to the database. + :param username: database username + :param password: database password + :param statement: sql statement to be executed + :param connection_properties: properties of the jdbc connection + passed as string with format + [propertyName=property;]* + :param connection_init_sqls: required only for MySql and MariaDB. + passed as list of strings + :param expansion_service: The address (host:port) of the ExpansionService. + """ + + super(WriteToJdbc, self).__init__( + self.URN, + NamedTupleBasedPayloadBuilder( + WriteToJdbcSchema( + driver_class_name=driver_class_name, + jdbc_url=jdbc_url, + username=username, + password=password, + statement=statement, + connection_properties=connection_properties, + connection_init_sqls=connection_init_sqls, + ), + ), + expansion_service or default_io_expansion_service(), + ) + + +ReadFromJdbcSchema = typing.NamedTuple( + 'ReadFromJdbcSchema', + [ + ('driver_class_name', unicode), + ('jdbc_url', unicode), + ('username', unicode), + ('password', unicode), + ('connection_properties', typing.Optional[unicode]), + ('connection_init_sqls', typing.Optional[typing.List[unicode]]), + ('query', unicode), + ('fetch_size', typing.Optional[int]), + ('output_parallelization', typing.Optional[bool]), + ], +) + + +class ReadFromJdbc(ExternalTransform): + """An external PTransform which reads Rows from the specified database. Review comment: ```suggestion """A PTransform which reads Rows from the specified database via JDBC. ``` ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: [email protected]
