[ https://issues.apache.org/jira/browse/BEAM-8501?focusedWorklogId=335757&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-335757 ]
ASF GitHub Bot logged work on BEAM-8501: ---------------------------------------- Author: ASF GitHub Bot Created on: 29/Oct/19 21:11 Start Date: 29/Oct/19 21:11 Worklog Time Spent: 10m Work Description: lukecwik commented on pull request #9904: [BEAM-8501] Fixes NPE When Using JdbcIO.PoolableDataSourceProvider URL: https://github.com/apache/beam/pull/9904#discussion_r340327405 ########## File path: sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java ########## @@ -1395,7 +1395,7 @@ public DataSource apply(Void input) { static synchronized DataSource buildDataSource(Void input) { if (source == null) { - DataSource basicSource = dataSourceProviderFn.apply(input); + DataSource basicSource = dataSourceProviderFn.apply(null); Review comment: The issue is that PoolableDataSourceProvider stores dataSourceProviderFn as a static which means that it will never be serialized and once it is deserialized it is lost. ---------------------------------------------------------------- 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: us...@infra.apache.org Issue Time Tracking ------------------- Worklog Id: (was: 335757) Time Spent: 20m (was: 10m) > NPE When Using JdbcIO.PoolableDataSourceProvider > ------------------------------------------------ > > Key: BEAM-8501 > URL: https://issues.apache.org/jira/browse/BEAM-8501 > Project: Beam > Issue Type: Bug > Components: io-java-jdbc > Affects Versions: 2.16.0 > Environment: Dataflow Runner > Reporter: Mehdi Massoudi > Priority: Critical > Time Spent: 20m > Remaining Estimate: 0h > > When using a simple PoolableDataSourceProvider in the Dataflow Runner I get a > null pointer exception at runtime. > {code:java} > JdbcIO.<~>write() > .withDataSourceProviderFn( > JdbcIO.PoolableDataSourceProvider.of( > JdbcIO.DataSourceConfiguration > .create("org.postgresql.Driver", jdbcUrl) > .withUsername(jdbcUsername) > .withPassword(jdbcPassword) > ) > ) > {code} > Other users seem to have a similar issue: > https://issues.apache.org/jira/browse/BEAM-7230?focusedCommentId=16845769&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-16845769. > The stack trace is below. > {code} > java.lang.RuntimeException: org.apache.beam.sdk.util.UserCodeException: > java.lang.NullPointerException > > org.apache.beam.runners.dataflow.worker.IntrinsicMapTaskExecutorFactory$1.typedApply(IntrinsicMapTaskExecutorFactory.java:194) > > org.apache.beam.runners.dataflow.worker.IntrinsicMapTaskExecutorFactory$1.typedApply(IntrinsicMapTaskExecutorFactory.java:165) > > org.apache.beam.runners.dataflow.worker.graph.Networks$TypeSafeNodeFunction.apply(Networks.java:63) > > org.apache.beam.runners.dataflow.worker.graph.Networks$TypeSafeNodeFunction.apply(Networks.java:50) > > org.apache.beam.runners.dataflow.worker.graph.Networks.replaceDirectedNetworkNodes(Networks.java:87) > > org.apache.beam.runners.dataflow.worker.IntrinsicMapTaskExecutorFactory.create(IntrinsicMapTaskExecutorFactory.java:125) > > org.apache.beam.runners.dataflow.worker.StreamingDataflowWorker.process(StreamingDataflowWorker.java:1232) > > org.apache.beam.runners.dataflow.worker.StreamingDataflowWorker.access$1000(StreamingDataflowWorker.java:149) > > org.apache.beam.runners.dataflow.worker.StreamingDataflowWorker$6.run(StreamingDataflowWorker.java:1049) > > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) > > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) > java.lang.Thread.run(Thread.java:745) > Caused by: org.apache.beam.sdk.util.UserCodeException: > java.lang.NullPointerException > > org.apache.beam.sdk.util.UserCodeException.wrap(UserCodeException.java:34) > > org.apache.beam.sdk.io.jdbc.JdbcIO$WriteVoid$WriteFn$DoFnInvoker.invokeSetup(Unknown > Source) > > org.apache.beam.runners.dataflow.worker.DoFnInstanceManagers$ConcurrentQueueInstanceManager.deserializeCopy(DoFnInstanceManagers.java:80) > > org.apache.beam.runners.dataflow.worker.DoFnInstanceManagers$ConcurrentQueueInstanceManager.peek(DoFnInstanceManagers.java:62) > > org.apache.beam.runners.dataflow.worker.UserParDoFnFactory.create(UserParDoFnFactory.java:95) > > org.apache.beam.runners.dataflow.worker.DefaultParDoFnFactory.create(DefaultParDoFnFactory.java:75) > > org.apache.beam.runners.dataflow.worker.IntrinsicMapTaskExecutorFactory.createParDoOperation(IntrinsicMapTaskExecutorFactory.java:264) > > org.apache.beam.runners.dataflow.worker.IntrinsicMapTaskExecutorFactory.access$000(IntrinsicMapTaskExecutorFactory.java:86) > > org.apache.beam.runners.dataflow.worker.IntrinsicMapTaskExecutorFactory$1.typedApply(IntrinsicMapTaskExecutorFactory.java:183) > > org.apache.beam.runners.dataflow.worker.IntrinsicMapTaskExecutorFactory$1.typedApply(IntrinsicMapTaskExecutorFactory.java:165) > > org.apache.beam.runners.dataflow.worker.graph.Networks$TypeSafeNodeFunction.apply(Networks.java:63) > > org.apache.beam.runners.dataflow.worker.graph.Networks$TypeSafeNodeFunction.apply(Networks.java:50) > > org.apache.beam.runners.dataflow.worker.graph.Networks.replaceDirectedNetworkNodes(Networks.java:87) > > org.apache.beam.runners.dataflow.worker.IntrinsicMapTaskExecutorFactory.create(IntrinsicMapTaskExecutorFactory.java:125) > > org.apache.beam.runners.dataflow.worker.StreamingDataflowWorker.process(StreamingDataflowWorker.java:1232) > > org.apache.beam.runners.dataflow.worker.StreamingDataflowWorker.access$1000(StreamingDataflowWorker.java:149) > > org.apache.beam.runners.dataflow.worker.StreamingDataflowWorker$6.run(StreamingDataflowWorker.java:1049) > > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) > > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) > java.lang.Thread.run(Thread.java:745) > Caused by: java.lang.NullPointerException > > org.apache.beam.sdk.io.jdbc.JdbcIO$PoolableDataSourceProvider.buildDataSource(JdbcIO.java:1363) > > org.apache.beam.sdk.io.jdbc.JdbcIO$PoolableDataSourceProvider.apply(JdbcIO.java:1358) > > org.apache.beam.sdk.io.jdbc.JdbcIO$PoolableDataSourceProvider.apply(JdbcIO.java:1338) > > org.apache.beam.sdk.io.jdbc.JdbcIO$WriteVoid$WriteFn.setup(JdbcIO.java:1221) > {code} > -- This message was sent by Atlassian Jira (v8.3.4#803005)