chamikaramj commented on a change in pull request #12145:
URL: https://github.com/apache/beam/pull/12145#discussion_r453020473



##########
File path: sdks/java/io/expansion-service/build.gradle
##########
@@ -33,6 +33,9 @@ ext.summary = "Expansion service serving several Java IOs"
 dependencies {
   compile project(":sdks:java:expansion-service")
   compile project(":sdks:java:io:kafka")
+  runtime project(":sdks:java:io:jdbc")
   runtime library.java.kafka_clients
+  // Include postgres so it can be used with external JDBC
+  runtime library.java.postgres

Review comment:
       These runtime dependencies are not included in the shadow jar ? How do 
they get staged for the runner ?

##########
File path: 
sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/external/JdbcExternalRead.java
##########
@@ -0,0 +1,85 @@
+/*
+ * 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.
+ */
+package org.apache.beam.sdk.io.jdbc.external;
+
+import com.google.auto.service.AutoService;
+import java.util.Map;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.annotations.Experimental.Kind;
+import org.apache.beam.sdk.expansion.ExternalTransformRegistrar;
+import org.apache.beam.sdk.io.jdbc.JdbcIO;
+import org.apache.beam.sdk.io.jdbc.JdbcIO.DataSourceConfiguration;
+import org.apache.beam.sdk.transforms.ExternalTransformBuilder;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.Row;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
+
+/** Exposes {@link JdbcIO.ReadRows} as an external transform for 
cross-language usage. */
+@Experimental(Kind.PORTABILITY)
+@AutoService(ExternalTransformRegistrar.class)
+public class JdbcExternalRead implements ExternalTransformRegistrar {

Review comment:
       How about renaming this to "JdbcReadRowsRegistrar". Current name make 
this sounds like a transform while in reality this is just a utility to 
register the existing transform with the expansion service.

##########
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.

Review comment:
       Is support for Dataflow runner confirmed and tested ?

##########
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 = false 
->
+        project.evaluationDependsOn(":sdks:java:io:expansion-service")
         project.tasks.create(name) {
           dependsOn 'setupVirtualenv'
           dependsOn ':sdks:python:sdist'
+          if (needsExpansionServiceJar) {
+            dependsOn ':sdks:java:io:expansion-service:shadowJar'
+          }
 
           doLast {
             // Python source directory is also tox execution workspace, We want
             // to isolate them per tox suite to avoid conflict when running
             // multiple tox suites in parallel.
             project.copy { from project.pythonSdkDeps; into copiedSrcRoot }
-
+            if (needsExpansionServiceJar) {
+              def expansionServiceJar =  
project.project(':sdks:java:io:expansion-service').shadowJar.archivePath
+              def expansionServiceDestinationDir = 
"${copiedSrcRoot}/sdks/java/io/expansion-service/build/libs"

Review comment:
       Could you clarify why this copy is needed ? Is building 
":sdks:java:io:expansion-service" inadequate for some reason ?

##########
File path: 
sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/external/JdbcExternalRead.java
##########
@@ -0,0 +1,85 @@
+/*
+ * 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.
+ */
+package org.apache.beam.sdk.io.jdbc.external;
+
+import com.google.auto.service.AutoService;
+import java.util.Map;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.annotations.Experimental.Kind;
+import org.apache.beam.sdk.expansion.ExternalTransformRegistrar;
+import org.apache.beam.sdk.io.jdbc.JdbcIO;
+import org.apache.beam.sdk.io.jdbc.JdbcIO.DataSourceConfiguration;
+import org.apache.beam.sdk.transforms.ExternalTransformBuilder;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.Row;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
+
+/** Exposes {@link JdbcIO.ReadRows} as an external transform for 
cross-language usage. */
+@Experimental(Kind.PORTABILITY)
+@AutoService(ExternalTransformRegistrar.class)
+public class JdbcExternalRead implements ExternalTransformRegistrar {
+
+  public static final String URN = "beam:external:java:jdbc:read:v1";

Review comment:
       Probably "beam:external:java:jdbc:read_rows:v1" is more unique to this 
transform

##########
File path: 
sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/external/JdbcExternalWrite.java
##########
@@ -0,0 +1,121 @@
+/*
+ * 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.
+ */
+package org.apache.beam.sdk.io.jdbc.external;
+
+import com.google.auto.service.AutoService;
+import java.sql.Date;
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.annotations.Experimental.Kind;
+import org.apache.beam.sdk.expansion.ExternalTransformRegistrar;
+import org.apache.beam.sdk.io.jdbc.JdbcIO;
+import org.apache.beam.sdk.io.jdbc.JdbcIO.DataSourceConfiguration;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.transforms.ExternalTransformBuilder;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PDone;
+import org.apache.beam.sdk.values.Row;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
+
+/** Exposes {@link JdbcIO.Write} as an external transform for cross-language 
usage. */
+@Experimental(Kind.PORTABILITY)
+@AutoService(ExternalTransformRegistrar.class)
+public class JdbcExternalWrite implements ExternalTransformRegistrar {

Review comment:
       JdbcWriteRegistrar

##########
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

Review comment:
       Please move this to io module directly. "apache_beam/io/jdbc.py"

##########
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):
+  """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)
+
+  An example can be found in

Review comment:
       Is this example being added in a separate PR ?

##########
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

Review comment:
       This should be Beam 2.24.0 for 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/
+"""
+
+# 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):
+  """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)
+
+  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):
+  """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)

Review comment:
       Ditto regarding adding a simple code example here.

##########
File path: 
sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/external/JdbcExternalWrite.java
##########
@@ -0,0 +1,121 @@
+/*
+ * 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.
+ */
+package org.apache.beam.sdk.io.jdbc.external;
+
+import com.google.auto.service.AutoService;
+import java.sql.Date;
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.annotations.Experimental.Kind;
+import org.apache.beam.sdk.expansion.ExternalTransformRegistrar;
+import org.apache.beam.sdk.io.jdbc.JdbcIO;
+import org.apache.beam.sdk.io.jdbc.JdbcIO.DataSourceConfiguration;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.transforms.ExternalTransformBuilder;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PDone;
+import org.apache.beam.sdk.values.Row;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
+
+/** Exposes {@link JdbcIO.Write} as an external transform for cross-language 
usage. */
+@Experimental(Kind.PORTABILITY)
+@AutoService(ExternalTransformRegistrar.class)
+public class JdbcExternalWrite implements ExternalTransformRegistrar {
+
+  public static final String URN = "beam:external:java:jdbc:write:v1";
+
+  @Override
+  public Map<String, Class<? extends ExternalTransformBuilder>> 
knownBuilders() {
+    return ImmutableMap.of(URN, JdbcExternalWrite.Builder.class);
+  }
+
+  /** Parameters class to expose the Write transform to an external SDK. */
+  public static class WriteConfiguration extends Configuration {
+    private String statement;
+
+    public void setStatement(String statement) {
+      this.statement = statement;
+    }
+  }
+
+  public static class Builder
+      implements ExternalTransformBuilder<WriteConfiguration, 
PCollection<Row>, PDone> {
+    @Override
+    public PTransform<PCollection<Row>, PDone> 
buildExternal(WriteConfiguration configuration) {
+      DataSourceConfiguration dataSourceConfiguration = 
configuration.getDataSourceConfiguration();
+
+      // TODO: BEAM-10396 use writeRows() when it's available
+      return JdbcIO.<Row>write()
+          .withDataSourceConfiguration(dataSourceConfiguration)
+          .withStatement(configuration.statement)
+          .withPreparedStatementSetter(new XlangPreparedStatementSetter());
+    }
+
+    private static class XlangPreparedStatementSetter

Review comment:
       How about calling this "BeamRowPreparedStatementSetter" and moving this 
out of external directory ? This sounds like a more generic utility.

##########
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):
+  """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)

Review comment:
       Let's add a simple code example here to showcase the usage of this 
transform.

##########
File path: 
sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/external/JdbcExternalRead.java
##########
@@ -0,0 +1,85 @@
+/*
+ * 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.
+ */
+package org.apache.beam.sdk.io.jdbc.external;
+
+import com.google.auto.service.AutoService;
+import java.util.Map;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.annotations.Experimental.Kind;
+import org.apache.beam.sdk.expansion.ExternalTransformRegistrar;
+import org.apache.beam.sdk.io.jdbc.JdbcIO;
+import org.apache.beam.sdk.io.jdbc.JdbcIO.DataSourceConfiguration;
+import org.apache.beam.sdk.transforms.ExternalTransformBuilder;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.Row;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
+
+/** Exposes {@link JdbcIO.ReadRows} as an external transform for 
cross-language usage. */
+@Experimental(Kind.PORTABILITY)
+@AutoService(ExternalTransformRegistrar.class)
+public class JdbcExternalRead implements ExternalTransformRegistrar {
+
+  public static final String URN = "beam:external:java:jdbc:read:v1";
+
+  @Override
+  public Map<String, Class<? extends ExternalTransformBuilder>> 
knownBuilders() {
+    return ImmutableMap.of(URN, JdbcExternalRead.Builder.class);
+  }
+
+  /** Parameters class to expose the Read transform to an external SDK. */
+  public static class ReadConfiguration extends Configuration {
+    private String query;
+    private Integer fetchSize;
+    private Boolean outputParallelization;
+
+    public void setOutputParallelization(Boolean outputParallelization) {
+      this.outputParallelization = outputParallelization;
+    }
+
+    public void setFetchSize(Integer fetchSize) {
+      this.fetchSize = fetchSize;
+    }
+
+    public void setQuery(String query) {
+      this.query = query;
+    }
+  }
+
+  public static class Builder
+      implements ExternalTransformBuilder<ReadConfiguration, PBegin, 
PCollection<Row>> {
+    @Override
+    public PTransform<PBegin, PCollection<Row>> 
buildExternal(ReadConfiguration configuration) {
+      DataSourceConfiguration dataSourceConfiguration = 
configuration.getDataSourceConfiguration();
+
+      JdbcIO.ReadRows readRows =

Review comment:
       How about adding unit tests for Builder classes for read and write ?

##########
File path: 
sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/external/JdbcExternalWrite.java
##########
@@ -0,0 +1,121 @@
+/*
+ * 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.
+ */
+package org.apache.beam.sdk.io.jdbc.external;
+
+import com.google.auto.service.AutoService;
+import java.sql.Date;
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.annotations.Experimental.Kind;
+import org.apache.beam.sdk.expansion.ExternalTransformRegistrar;
+import org.apache.beam.sdk.io.jdbc.JdbcIO;
+import org.apache.beam.sdk.io.jdbc.JdbcIO.DataSourceConfiguration;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.transforms.ExternalTransformBuilder;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PDone;
+import org.apache.beam.sdk.values.Row;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
+
+/** Exposes {@link JdbcIO.Write} as an external transform for cross-language 
usage. */
+@Experimental(Kind.PORTABILITY)
+@AutoService(ExternalTransformRegistrar.class)
+public class JdbcExternalWrite implements ExternalTransformRegistrar {
+
+  public static final String URN = "beam:external:java:jdbc:write:v1";
+
+  @Override
+  public Map<String, Class<? extends ExternalTransformBuilder>> 
knownBuilders() {
+    return ImmutableMap.of(URN, JdbcExternalWrite.Builder.class);
+  }
+
+  /** Parameters class to expose the Write transform to an external SDK. */
+  public static class WriteConfiguration extends Configuration {
+    private String statement;
+
+    public void setStatement(String statement) {
+      this.statement = statement;
+    }
+  }
+
+  public static class Builder
+      implements ExternalTransformBuilder<WriteConfiguration, 
PCollection<Row>, PDone> {
+    @Override
+    public PTransform<PCollection<Row>, PDone> 
buildExternal(WriteConfiguration configuration) {
+      DataSourceConfiguration dataSourceConfiguration = 
configuration.getDataSourceConfiguration();
+
+      // TODO: BEAM-10396 use writeRows() when it's available
+      return JdbcIO.<Row>write()
+          .withDataSourceConfiguration(dataSourceConfiguration)
+          .withStatement(configuration.statement)
+          .withPreparedStatementSetter(new XlangPreparedStatementSetter());
+    }
+
+    private static class XlangPreparedStatementSetter
+        implements JdbcIO.PreparedStatementSetter<Row> {
+      @Override
+      public void setParameters(Row row, PreparedStatement statement) throws 
SQLException {
+        List<Schema.Field> fieldTypes = row.getSchema().getFields();
+        for (int i = 0; i < fieldTypes.size(); ++i) {
+          Schema.TypeName typeName = fieldTypes.get(i).getType().getTypeName();
+          switch (typeName) {
+            case DATETIME:

Review comment:
       Please add unit tests for these conversions.




----------------------------------------------------------------
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]


Reply via email to