You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@beam.apache.org by hg...@guerrillamail.com on 2019/03/29 01:53:44 UTC

JDBCIO Connection Pooling

Hello, I have recently created a streaming google dataflow program with apache beam using the java SDK. When files land in cloud-storage they fire off pubsub messages with the filename, which I consume and then write to a cloud sql database. Everything works great for the most part. However I've been testing it more thoroughly recently and noticed that if I start reading in multiple files that database connections slowly grow  and grow until they hit the default limit of 100 connections. Strangely the idle connections never seem to disappear and the program might run for hours watching for pubsub messages so this creates a problem. 

My initial idea was to create a c3p0 connection pool and pass that in as the datasource through the JdbcIO.DataSourceConfiguration.create method. I noticed this didn't seem to make a difference which perplexed me even with my aggressive pool connections. After some debugging I noticed that the datasource was still being wrapped in a pooling datasource..even through it already is a pooled datasource. I was wondering what strangeness this caused, so locally I hacked JdbcIO to just return my c3p0 datasource and do nothing else in the buildDatasource method ( https://github.com/apache/beam/blob/master/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java - line 331). It seemed to alleviate the connection problems and now I see the idle connections slowly start disappearing in cloud sql. Everything appears to be working smoothly. Obviously this isn't the solution I want moving forward. Is there some other way to achieve this? What grave mistakes have I done by bypassing the standard way of doing it?





----
Sent using Guerrillamail.com
Block or report abuse: https://www.guerrillamail.com//abuse/?a=VFJxFy0CRrUYxg%2Bk8X0Xd1rIX80%3D



Re: JDBCIO Connection Pooling

Posted by Jean-Baptiste Onofré <jb...@nanthrax.net>.
Hi,

You can directly provide the datasource in the datasource configuration,
the IO should use directly the datasource (not rewrap it). It's the way
I wrote the IO initially.

IMHO, the buildDatasource() method should not systematically wrap as a
poolable datasource. We should add a configuration in datasource
configuration to let the user control if he wants to wrap as poolable
datasource or not.

If you don't mind I will create a Jira about that and work on it.

Thoughts ?

Regards
JB

On 29/03/2019 02:53, hgu2hw+2g0aed6fdoszs@guerrillamail.com wrote:
> Hello, I have recently created a streaming google dataflow program with apache beam using the java SDK. When files land in cloud-storage they fire off pubsub messages with the filename, which I consume and then write to a cloud sql database. Everything works great for the most part. However I've been testing it more thoroughly recently and noticed that if I start reading in multiple files that database connections slowly grow  and grow until they hit the default limit of 100 connections. Strangely the idle connections never seem to disappear and the program might run for hours watching for pubsub messages so this creates a problem. 
> 
> My initial idea was to create a c3p0 connection pool and pass that in as the datasource through the JdbcIO.DataSourceConfiguration.create method. I noticed this didn't seem to make a difference which perplexed me even with my aggressive pool connections. After some debugging I noticed that the datasource was still being wrapped in a pooling datasource..even through it already is a pooled datasource. I was wondering what strangeness this caused, so locally I hacked JdbcIO to just return my c3p0 datasource and do nothing else in the buildDatasource method ( https://github.com/apache/beam/blob/master/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java - line 331). It seemed to alleviate the connection problems and now I see the idle connections slowly start disappearing in cloud sql. Everything appears to be working smoothly. Obviously this isn't the solution I want moving forward. Is there some other way to achieve this? What grave mistakes have I done by bypassing the standard way of doing it?
> 
> 
> 
> 
> 
> ----
> Sent using Guerrillamail.com
> Block or report abuse: https://www.guerrillamail.com//abuse/?a=VFJxFy0CRrUYxg%2Bk8X0Xd1rIX80%3D
> 
> 

-- 
Jean-Baptiste Onofré
jbonofre@apache.org
http://blog.nanthrax.net
Talend - http://www.talend.com