[ 
https://issues.apache.org/jira/browse/BEAM-7230?focusedWorklogId=246039&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-246039
 ]

ASF GitHub Bot logged work on BEAM-7230:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 21/May/19 13:27
            Start Date: 21/May/19 13:27
    Worklog Time Spent: 10m 
      Work Description: jbonofre commented on pull request #8635: [BEAM-7230] 
Make PoolableDataSourceProvider a static singleton
URL: https://github.com/apache/beam/pull/8635#discussion_r286024117
 
 

 ##########
 File path: 
sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java
 ##########
 @@ -1112,39 +1072,50 @@ public void process(ProcessContext c) {
     }
   }
 
-  private static class DataSourceProviderFromDataSourceConfiguration
+  /** Wraps a {@link DataSourceConfiguration} to provide a {@link 
PoolingDataSource}. */
+  public static class PoolableDataSourceProvider
       implements SerializableFunction<Void, DataSource>, HasDisplayData {
-    private final DataSourceConfiguration config;
-    private static DataSourceProviderFromDataSourceConfiguration instance;
+    private static PoolableDataSourceProvider instance;
+    private static transient DataSource source;
+    private static SerializableFunction<Void, DataSource> dataSourceProviderFn;
 
-    private 
DataSourceProviderFromDataSourceConfiguration(DataSourceConfiguration config) {
-      this.config = config;
+    private PoolableDataSourceProvider(DataSourceConfiguration config) {
+      dataSourceProviderFn = 
DataSourceProviderFromDataSourceConfiguration.of(config);
     }
 
-    public static SerializableFunction<Void, DataSource> 
of(DataSourceConfiguration config) {
+    public static synchronized SerializableFunction<Void, DataSource> of(
+        DataSourceConfiguration config) {
       if (instance == null) {
-        instance = new DataSourceProviderFromDataSourceConfiguration(config);
+        instance = new PoolableDataSourceProvider(config);
       }
       return instance;
     }
 
     @Override
     public DataSource apply(Void input) {
-      return config.buildDatasource();
-    }
-
-    @Override
-    public void populateDisplayData(DisplayData.Builder builder) {
-      config.populateDisplayData(builder);
-    }
-  }
-
-  private abstract static class BaseDataSourceProvider
-      implements SerializableFunction<Void, DataSource>, HasDisplayData {
-    private final SerializableFunction<Void, DataSource> dataSourceProviderFn;
-
-    BaseDataSourceProvider(SerializableFunction<Void, DataSource> 
dataSourceProviderFn) {
-      this.dataSourceProviderFn = dataSourceProviderFn;
+      return buildDataSource(input);
+    }
+
+    static synchronized DataSource buildDataSource(Void input) {
+      if (source == null) {
+        DataSource basicSource = dataSourceProviderFn.apply(input);
+        DataSourceConnectionFactory connectionFactory =
+            new DataSourceConnectionFactory(basicSource);
+        PoolableConnectionFactory poolableConnectionFactory =
+            new PoolableConnectionFactory(connectionFactory, null);
+        GenericObjectPoolConfig poolConfig = new GenericObjectPoolConfig();
+        poolConfig.setMaxTotal(1);
+        poolConfig.setMinIdle(0);
+        poolConfig.setMinEvictableIdleTimeMillis(10000);
+        poolConfig.setSoftMinEvictableIdleTimeMillis(30000);
+        GenericObjectPool connectionPool =
+            new GenericObjectPool(poolableConnectionFactory, poolConfig);
+        poolableConnectionFactory.setPool(connectionPool);
+        poolableConnectionFactory.setDefaultAutoCommit(false);
+        poolableConnectionFactory.setDefaultReadOnly(false);
+        source = new PoolingDataSource(connectionPool);
+      }
+      return source;
 
 Review comment:
   If `source` is provided but not a `PoolableDataSource`, we just use it, 
right (we don't wrap as a poolable datasource) ?
 
----------------------------------------------------------------
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]


Issue Time Tracking
-------------------

    Worklog Id:     (was: 246039)
    Time Spent: 50m  (was: 40m)

> Using JdbcIO creates huge amount of connections
> -----------------------------------------------
>
>                 Key: BEAM-7230
>                 URL: https://issues.apache.org/jira/browse/BEAM-7230
>             Project: Beam
>          Issue Type: Bug
>          Components: runner-dataflow
>    Affects Versions: 2.11.0
>            Reporter: Brachi Packter
>            Assignee: Ismaël Mejía
>            Priority: Major
>             Fix For: 2.13.0
>
>          Time Spent: 50m
>  Remaining Estimate: 0h
>
> I want to write form DataFlow to GCP cloud SQL, I'm using connection pool, 
> and still I see huge amount of connections in GCP SQL (4k while I set 
> connection pool to 300), and most of them in sleep.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to