[ 
https://issues.apache.org/jira/browse/FLINK-23437?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17384194#comment-17384194
 ] 

Maciej Bryński commented on FLINK-23437:
----------------------------------------

And the log:
{code:java}
10:44:21.338 [Source: Custom Source -> Sink: Output stream (1/1)#0] WARN  
org.apache.flink.connector.jdbc.xa.JdbcXaSinkFunction - nothing to commit up to 
checkpoint: 80
10:44:21.373 [Source: Custom Source -> Sink: Output stream (1/1)#0] DEBUG 
org.apache.flink.connector.jdbc.xa.JdbcXaSinkFunction - snapshot state, 
checkpointId=81
10:44:21.374 [Source: Custom Source -> Sink: Output stream (1/1)#0] DEBUG 
org.apache.flink.connector.jdbc.xa.XaFacadeImpl - NETWORKS: end, 
xid=Optional[201:099b749c5e88f0e650d968606dacd9ac000000005100000000000000:619cadea]
10:44:21.374 [Source: Custom Source -> Sink: Output stream (1/1)#0] DEBUG 
org.apache.flink.connector.jdbc.xa.XaFacadeImpl - NETWORKS: prepare, 
xid=Optional[201:099b749c5e88f0e650d968606dacd9ac000000005100000000000000:619cadea]
10:44:21.374 [Source: Custom Source -> Sink: Output stream (1/1)#0] INFO  
org.apache.flink.connector.jdbc.xa.JdbcXaSinkFunction - empty XA transaction 
(skip), xid: 
201:099b749c5e88f0e650d968606dacd9ac000000005100000000000000:619cadea, 
checkpoint 81
10:44:21.418 [Source: Custom Source -> Sink: Output stream (1/1)#0] WARN  
org.apache.flink.runtime.taskmanager.Task - Source: Custom Source -> Sink: 
Output stream (1/1)#0 (8353219eab3eaabd015a6e6b00a06b92) switched from RUNNING 
to FAILED with failure cause: java.lang.Exception: Could not perform checkpoint 
81 for operator Source: Custom Source -> Sink: Output stream (1/1)#0.
        at 
org.apache.flink.streaming.runtime.tasks.StreamTask.triggerCheckpoint(StreamTask.java:1000)
        at 
org.apache.flink.streaming.runtime.tasks.StreamTask.lambda$triggerCheckpointAsync$7(StreamTask.java:960)
        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: org.apache.flink.runtime.checkpoint.CheckpointException: Could not 
complete snapshot 81 for operator Source: Custom Source -> Sink: Output stream 
(1/1)#0. Failure reason: Checkpoint was declined.
        at 
org.apache.flink.streaming.api.operators.StreamOperatorStateHandler.snapshotState(StreamOperatorStateHandler.java:264)
        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:1086)
        at 
org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor$SynchronizedStreamTaskActionExecutor.runThrowing(StreamTaskActionExecutor.java:93)
        at 
org.apache.flink.streaming.runtime.tasks.StreamTask.performCheckpoint(StreamTask.java:1070)
        at 
org.apache.flink.streaming.runtime.tasks.StreamTask.triggerCheckpoint(StreamTask.java:988)
        ... 13 more
Caused by: java.sql.SQLRecoverableException: IO Error: Got minus one from a 
read call
        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:338)
        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)
        ... 23 more
Caused by: oracle.net.ns.NetException: Got minus one from a read call
        at oracle.net.ns.NSProtocolNIO.doSocketRead(NSProtocolNIO.java:562)
        at oracle.net.ns.NIOPacket.readNIOPacket(NIOPacket.java:408)
        at 
oracle.net.ns.NSProtocolNIO.negotiateConnection(NSProtocolNIO.java:132)
        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: Critical
>
> 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)

Reply via email to