[
https://issues.apache.org/jira/browse/FLINK-23437?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17384293#comment-17384293
]
Maciej Bryński commented on FLINK-23437:
----------------------------------------
Sometimes I'm getting different error. Same reason: exceeded number of
connections.
{code:java}
16:09:39.988 [Source: Custom Source -> Sink: Output stream (1/1)#0] WARN
org.apache.flink.connector.jdbc.xa.JdbcXaSinkFunction - nothing to commit up to
checkpoint: 80
16:09:40.025 [Source: Custom Source -> Sink: Output stream (1/1)#0] DEBUG
org.apache.flink.connector.jdbc.xa.JdbcXaSinkFunction - snapshot state,
checkpointId=81
16:09:40.025 [Source: Custom Source -> Sink: Output stream (1/1)#0] DEBUG
org.apache.flink.connector.jdbc.xa.XaFacadeImpl - end,
xid=Optional[201:4b568b2fceb593e236dabae63f027e06000000005100000000000000:976b2337]
16:09:40.025 [Source: Custom Source -> Sink: Output stream (1/1)#0] DEBUG
org.apache.flink.connector.jdbc.xa.XaFacadeImpl - prepare,
xid=Optional[201:4b568b2fceb593e236dabae63f027e06000000005100000000000000:976b2337]
16:09:40.025 [Source: Custom Source -> Sink: Output stream (1/1)#0] INFO
org.apache.flink.connector.jdbc.xa.JdbcXaSinkFunction - empty XA transaction
(skip), xid:
201:4b568b2fceb593e236dabae63f027e06000000005100000000000000:976b2337,
checkpoint 81
java.sql.SQLException: Listener refused the connection with the following error:
ORA-12516, TNS:listener could not find available handler with matching protocol
stack at oracle.jdbc.driver.T4CConnection.logon(T4CConnection.java:874)
at
oracle.jdbc.driver.PhysicalConnection.connect(PhysicalConnection.java:807)
at
oracle.jdbc.driver.T4CDriverExtension.getConnection(T4CDriverExtension.java:77)
at oracle.jdbc.driver.OracleDriver.connect(OracleDriver.java:769)
at
oracle.jdbc.pool.OracleDataSource.getPhysicalConnection(OracleDataSource.java:450)
at
oracle.jdbc.xa.client.OracleXADataSource.getPooledConnection(OracleXADataSource.java:550)
at
oracle.jdbc.xa.client.OracleXADataSource.getPooledConnection(OracleXADataSource.java:270)
at
oracle.jdbc.xa.client.OracleXADataSource.getXAConnection(OracleXADataSource.java:159)
at
oracle.jdbc.xa.client.OracleXADataSource.getXAConnection(OracleXADataSource.java:116)
at
org.apache.flink.connector.jdbc.xa.XaFacadeImpl.open(XaFacadeImpl.java:98)
at
org.apache.flink.connector.jdbc.xa.XaFacadePoolingImpl.start(XaFacadePoolingImpl.java:80)
at
org.apache.flink.connector.jdbc.xa.JdbcXaSinkFunction.beginTx(JdbcXaSinkFunction.java:339)
at
org.apache.flink.connector.jdbc.xa.JdbcXaSinkFunction.snapshotState(JdbcXaSinkFunction.java:275)
at
org.apache.flink.streaming.util.functions.StreamingFunctionUtils.trySnapshotFunctionState(StreamingFunctionUtils.java:118)
at
org.apache.flink.streaming.util.functions.StreamingFunctionUtils.snapshotFunctionState(StreamingFunctionUtils.java:99)
at
org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator.snapshotState(AbstractUdfStreamOperator.java:89)
at
org.apache.flink.streaming.api.operators.StreamOperatorStateHandler.snapshotState(StreamOperatorStateHandler.java:218)
at
org.apache.flink.streaming.api.operators.StreamOperatorStateHandler.snapshotState(StreamOperatorStateHandler.java:169)
at
org.apache.flink.streaming.api.operators.AbstractStreamOperator.snapshotState(AbstractStreamOperator.java:371)
at
org.apache.flink.streaming.runtime.tasks.SubtaskCheckpointCoordinatorImpl.checkpointStreamOperator(SubtaskCheckpointCoordinatorImpl.java:706)
at
org.apache.flink.streaming.runtime.tasks.SubtaskCheckpointCoordinatorImpl.buildOperatorSnapshotFutures(SubtaskCheckpointCoordinatorImpl.java:627)
at
org.apache.flink.streaming.runtime.tasks.SubtaskCheckpointCoordinatorImpl.takeSnapshotSync(SubtaskCheckpointCoordinatorImpl.java:590)
at
org.apache.flink.streaming.runtime.tasks.SubtaskCheckpointCoordinatorImpl.checkpointState(SubtaskCheckpointCoordinatorImpl.java:312)
at
org.apache.flink.streaming.runtime.tasks.StreamTask.lambda$performCheckpoint$8(StreamTask.java:1089)
at
org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor$SynchronizedStreamTaskActionExecutor.runThrowing(StreamTaskActionExecutor.java:93)
at
org.apache.flink.streaming.runtime.tasks.StreamTask.performCheckpoint(StreamTask.java:1073)
at
org.apache.flink.streaming.runtime.tasks.StreamTask.triggerCheckpointAsyncInMailbox(StreamTask.java:991)
at
org.apache.flink.streaming.runtime.tasks.StreamTask.lambda$triggerCheckpointAsync$7(StreamTask.java:955)
at
org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor$SynchronizedStreamTaskActionExecutor.runThrowing(StreamTaskActionExecutor.java:93)
at
org.apache.flink.streaming.runtime.tasks.mailbox.Mail.run(Mail.java:90)
at
org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.processMailsWhenDefaultActionUnavailable(MailboxProcessor.java:344)
at
org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.processMail(MailboxProcessor.java:330)
at
org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.runMailboxLoop(MailboxProcessor.java:202)
at
org.apache.flink.streaming.runtime.tasks.StreamTask.runMailboxLoop(StreamTask.java:681)
at
org.apache.flink.streaming.runtime.tasks.StreamTask.executeInvoke(StreamTask.java:636)
at
org.apache.flink.streaming.runtime.tasks.StreamTask.runWithCleanUpOnFail(StreamTask.java:647)
at
org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:620)
at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:779)
at org.apache.flink.runtime.taskmanager.Task.run(Task.java:566)
at java.base/java.lang.Thread.run(Thread.java:829)
Caused by: oracle.net.ns.NetException: Listener refused the connection with the
following error:
ORA-12516, TNS:listener could not find available handler with matching protocol
stack at
oracle.net.ns.NSProtocolNIO.negotiateConnection(NSProtocolNIO.java:289)
at oracle.net.ns.NSProtocol.connect(NSProtocol.java:364)
at oracle.jdbc.driver.T4CConnection.connect(T4CConnection.java:1625)
at oracle.jdbc.driver.T4CConnection.logon(T4CConnection.java:606)
... 39 more
{code}
> Connection leak in XaFacadePoolingImpl
> --------------------------------------
>
> Key: FLINK-23437
> URL: https://issues.apache.org/jira/browse/FLINK-23437
> Project: Flink
> Issue Type: Bug
> Components: Connectors / JDBC
> Affects Versions: 1.13.1
> Reporter: Maciej Bryński
> Priority: Major
>
> Hi,
> I'm using JDBC XA connector to put data into Oracle database.
> I'm facing issue with too many concurrent connection to database.
> I changed this method to return XaFacadeImpl instead of XaFacadePoolingImpl
> and problem was solved.
> {code:java}
> static XaFacade fromXaDataSourceSupplier(
> Supplier<XADataSource> dataSourceSupplier, Integer timeoutSec) {
> return new XaFacadePoolingImpl(() -> new XaFacadeImpl(dataSourceSupplier,
> timeoutSec));
> }
> {code}
>
--
This message was sent by Atlassian Jira
(v8.3.4#803005)