You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by ie...@apache.org on 2019/01/09 10:46:44 UTC

[beam] branch master updated (16a2e8d -> 3b10890)

This is an automated email from the ASF dual-hosted git repository.

iemejia pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/beam.git.


    from 16a2e8d  Merge pull request #7440: [BEAM-6388] Fix flaky AtomicInsertTest#testAtomicInsert
     new 93d0882  [BEAM-6348] Add ValueProvider support for Statement in JdbcIO.write()
     new 54341bd  [BEAM-6348] Reuse ValueProvider methods on method invocation on JdbcIO
     new 3b10890  Merge pull request #7434: [BEAM-6348] Add ValueProvider support for Statement in JdbcIO.write()

The 3 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../java/org/apache/beam/sdk/io/jdbc/JdbcIO.java   | 25 +++++++++++-----------
 1 file changed, 12 insertions(+), 13 deletions(-)


[beam] 03/03: Merge pull request #7434: [BEAM-6348] Add ValueProvider support for Statement in JdbcIO.write()

Posted by ie...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

iemejia pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/beam.git

commit 3b108907c6a98a76f14b78f697d192c205b5723c
Merge: 16a2e8d 54341bd
Author: Ismaël Mejía <ie...@gmail.com>
AuthorDate: Wed Jan 9 11:46:08 2019 +0100

    Merge pull request #7434: [BEAM-6348] Add ValueProvider support for Statement in JdbcIO.write()

 .../java/org/apache/beam/sdk/io/jdbc/JdbcIO.java   | 25 +++++++++++-----------
 1 file changed, 12 insertions(+), 13 deletions(-)


[beam] 02/03: [BEAM-6348] Reuse ValueProvider methods on method invocation on JdbcIO

Posted by ie...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

iemejia pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/beam.git

commit 54341bd8d92dd97d3673b000bb3006ffd24913ca
Author: Ismaël Mejía <ie...@gmail.com>
AuthorDate: Wed Jan 9 11:43:55 2019 +0100

    [BEAM-6348] Reuse ValueProvider methods on method invocation on JdbcIO
---
 .../src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java | 15 +++++----------
 1 file changed, 5 insertions(+), 10 deletions(-)

diff --git a/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java b/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java
index 82872c8..a189123 100644
--- a/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java
+++ b/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java
@@ -268,10 +268,7 @@ public class JdbcIO {
     public static DataSourceConfiguration create(String driverClassName, String url) {
       checkArgument(driverClassName != null, "driverClassName can not be null");
       checkArgument(url != null, "url can not be null");
-      return new AutoValue_JdbcIO_DataSourceConfiguration.Builder()
-          .setDriverClassName(ValueProvider.StaticValueProvider.of(driverClassName))
-          .setUrl(ValueProvider.StaticValueProvider.of(url))
-          .build();
+      return create(ValueProvider.StaticValueProvider.of(driverClassName), ValueProvider.StaticValueProvider.of(url));
     }
 
     public static DataSourceConfiguration create(
@@ -285,7 +282,7 @@ public class JdbcIO {
     }
 
     public DataSourceConfiguration withUsername(String username) {
-      return builder().setUsername(ValueProvider.StaticValueProvider.of(username)).build();
+      return withUsername(ValueProvider.StaticValueProvider.of(username));
     }
 
     public DataSourceConfiguration withUsername(ValueProvider<String> username) {
@@ -293,7 +290,7 @@ public class JdbcIO {
     }
 
     public DataSourceConfiguration withPassword(String password) {
-      return builder().setPassword(ValueProvider.StaticValueProvider.of(password)).build();
+      return withPassword(ValueProvider.StaticValueProvider.of(password));
     }
 
     public DataSourceConfiguration withPassword(ValueProvider<String> password) {
@@ -309,9 +306,7 @@ public class JdbcIO {
      */
     public DataSourceConfiguration withConnectionProperties(String connectionProperties) {
       checkArgument(connectionProperties != null, "connectionProperties can not be null");
-      return builder()
-          .setConnectionProperties(ValueProvider.StaticValueProvider.of(connectionProperties))
-          .build();
+      return withConnectionProperties(ValueProvider.StaticValueProvider.of(connectionProperties));
     }
 
     /** Same as {@link #withConnectionProperties(String)} but accepting a ValueProvider. */
@@ -722,7 +717,7 @@ public class JdbcIO {
     }
 
     public Write<T> withStatement(String statement) {
-      return toBuilder().setStatement(ValueProvider.StaticValueProvider.of(statement)).build();
+      return withStatement(ValueProvider.StaticValueProvider.of(statement));
     }
 
     public Write<T> withStatement(ValueProvider<String> statement) {


[beam] 01/03: [BEAM-6348] Add ValueProvider support for Statement in JdbcIO.write()

Posted by ie...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

iemejia pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/beam.git

commit 93d08824f49014e27b1d10a938e2c4ee864086eb
Author: Prem Kumar Karunakaran <p....@metrosystems.net>
AuthorDate: Tue Jan 8 13:24:44 2019 +0530

    [BEAM-6348] Add ValueProvider support for Statement in JdbcIO.write()
---
 .../src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java    | 12 ++++++++----
 1 file changed, 8 insertions(+), 4 deletions(-)

diff --git a/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java b/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java
index 2423b2a..82872c8 100644
--- a/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java
+++ b/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java
@@ -690,7 +690,7 @@ public class JdbcIO {
     abstract DataSourceConfiguration getDataSourceConfiguration();
 
     @Nullable
-    abstract String getStatement();
+    abstract ValueProvider<String> getStatement();
 
     abstract long getBatchSize();
 
@@ -706,7 +706,7 @@ public class JdbcIO {
     abstract static class Builder<T> {
       abstract Builder<T> setDataSourceConfiguration(DataSourceConfiguration config);
 
-      abstract Builder<T> setStatement(String statement);
+      abstract Builder<T> setStatement(ValueProvider<String> statement);
 
       abstract Builder<T> setBatchSize(long batchSize);
 
@@ -722,6 +722,10 @@ public class JdbcIO {
     }
 
     public Write<T> withStatement(String statement) {
+      return toBuilder().setStatement(ValueProvider.StaticValueProvider.of(statement)).build();
+    }
+
+    public Write<T> withStatement(ValueProvider<String> statement) {
       return toBuilder().setStatement(statement).build();
     }
 
@@ -789,7 +793,7 @@ public class JdbcIO {
       public void startBundle() throws Exception {
         connection = dataSource.getConnection();
         connection.setAutoCommit(false);
-        preparedStatement = connection.prepareStatement(spec.getStatement());
+        preparedStatement = connection.prepareStatement(spec.getStatement().get());
       }
 
       @ProcessElement
@@ -835,7 +839,7 @@ public class JdbcIO {
         BackOff backoff = BUNDLE_WRITE_BACKOFF.backoff();
         while (true) {
           try (PreparedStatement preparedStatement =
-              connection.prepareStatement(spec.getStatement())) {
+              connection.prepareStatement(spec.getStatement().get())) {
             try {
               // add each record in the statement batch
               for (T record : records) {