Skip to content

Commit 92e2b5f

Browse files
authored
[FLINK-25421] Port JDBC Sink to new Unified Sink API (FLIP-143)
1 parent 95294ff commit 92e2b5f

File tree

62 files changed

+3687
-72
lines changed

Some content is hidden

Large Commits have some content hidden by default. Use the searchbox below for content that may be hidden.

62 files changed

+3687
-72
lines changed

.github/workflows/push_pr.yml

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -25,13 +25,12 @@ jobs:
2525
compile_and_test:
2626
strategy:
2727
matrix:
28-
flink: [ 1.16-SNAPSHOT, 1.17-SNAPSHOT ]
29-
jdk: [ '8, 11' ]
3028
include:
3129
- flink: 1.18-SNAPSHOT
3230
jdk: '8, 11, 17'
3331
- flink: 1.19-SNAPSHOT
3432
jdk: '8, 11, 17, 21'
33+
3534
uses: apache/flink-connector-shared-utils/.github/workflows/ci.yml@ci_utils
3635
with:
3736
flink_version: ${{ matrix.flink }}

.github/workflows/weekly.yml

Lines changed: 4 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -27,29 +27,21 @@ jobs:
2727
strategy:
2828
matrix:
2929
flink_branches: [{
30-
flink: 1.16-SNAPSHOT,
31-
branch: main
32-
}, {
33-
flink: 1.17-SNAPSHOT,
34-
branch: main
35-
}, {
3630
flink: 1.18-SNAPSHOT,
3731
jdk: '8, 11, 17',
3832
branch: main
3933
}, {
4034
flink: 1.19-SNAPSHOT,
4135
jdk: '8, 11, 17, 21',
4236
branch: main
43-
}, {
44-
flink: 1.16.2,
45-
branch: v3.1
46-
}, {
47-
flink: 1.17.1,
48-
branch: v3.1
4937
}, {
5038
flink: 1.18.0,
5139
jdk: '8, 11, 17',
5240
branch: v3.1
41+
}, {
42+
flink: 1.19.0,
43+
jdk: '8, 11, 17, 21',
44+
branch: v3.1
5345
}]
5446
uses: apache/flink-connector-shared-utils/.github/workflows/ci.yml@ci_utils
5547
with:

flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/JdbcInputFormat.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -27,8 +27,8 @@
2727
import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
2828
import org.apache.flink.api.java.typeutils.RowTypeInfo;
2929
import org.apache.flink.configuration.Configuration;
30-
import org.apache.flink.connector.jdbc.internal.connection.JdbcConnectionProvider;
31-
import org.apache.flink.connector.jdbc.internal.connection.SimpleJdbcConnectionProvider;
30+
import org.apache.flink.connector.jdbc.datasource.connections.JdbcConnectionProvider;
31+
import org.apache.flink.connector.jdbc.datasource.connections.SimpleJdbcConnectionProvider;
3232
import org.apache.flink.connector.jdbc.split.JdbcParameterValuesProvider;
3333
import org.apache.flink.core.io.GenericInputSplit;
3434
import org.apache.flink.core.io.InputSplit;

flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/JdbcRowOutputFormat.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -19,9 +19,9 @@
1919
package org.apache.flink.connector.jdbc;
2020

2121
import org.apache.flink.annotation.Experimental;
22+
import org.apache.flink.connector.jdbc.datasource.connections.JdbcConnectionProvider;
23+
import org.apache.flink.connector.jdbc.datasource.connections.SimpleJdbcConnectionProvider;
2224
import org.apache.flink.connector.jdbc.internal.JdbcOutputFormat;
23-
import org.apache.flink.connector.jdbc.internal.connection.JdbcConnectionProvider;
24-
import org.apache.flink.connector.jdbc.internal.connection.SimpleJdbcConnectionProvider;
2525
import org.apache.flink.connector.jdbc.internal.executor.JdbcBatchStatementExecutor;
2626
import org.apache.flink.types.Row;
2727

flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/JdbcSink.java

Lines changed: 6 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -18,10 +18,11 @@
1818
package org.apache.flink.connector.jdbc;
1919

2020
import org.apache.flink.annotation.PublicEvolving;
21+
import org.apache.flink.connector.jdbc.datasource.connections.SimpleJdbcConnectionProvider;
2122
import org.apache.flink.connector.jdbc.internal.GenericJdbcSinkFunction;
2223
import org.apache.flink.connector.jdbc.internal.JdbcOutputFormat;
23-
import org.apache.flink.connector.jdbc.internal.connection.SimpleJdbcConnectionProvider;
2424
import org.apache.flink.connector.jdbc.internal.executor.JdbcBatchStatementExecutor;
25+
import org.apache.flink.connector.jdbc.sink.JdbcSinkBuilder;
2526
import org.apache.flink.connector.jdbc.xa.JdbcXaSinkFunction;
2627
import org.apache.flink.connector.jdbc.xa.XaFacade;
2728
import org.apache.flink.streaming.api.functions.sink.SinkFunction;
@@ -107,5 +108,9 @@ public static <T> SinkFunction<T> exactlyOnceSink(
107108
exactlyOnceOptions);
108109
}
109110

111+
public static <IN> JdbcSinkBuilder<IN> builder() {
112+
return org.apache.flink.connector.jdbc.sink.JdbcSink.builder();
113+
}
114+
110115
private JdbcSink() {}
111116
}
Lines changed: 9 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -15,18 +15,19 @@
1515
* limitations under the License.
1616
*/
1717

18-
package org.apache.flink.connector.jdbc.internal.connection;
18+
package org.apache.flink.connector.jdbc.datasource.connections;
1919

20-
import org.apache.flink.annotation.Internal;
20+
import org.apache.flink.annotation.PublicEvolving;
2121

2222
import javax.annotation.Nullable;
2323

24+
import java.io.Serializable;
2425
import java.sql.Connection;
2526
import java.sql.SQLException;
2627

2728
/** JDBC connection provider. */
28-
@Internal
29-
public interface JdbcConnectionProvider {
29+
@PublicEvolving
30+
public interface JdbcConnectionProvider extends Serializable, AutoCloseable {
3031
/**
3132
* Get existing connection.
3233
*
@@ -64,4 +65,8 @@ public interface JdbcConnectionProvider {
6465
* @throws ClassNotFoundException driver class not found
6566
*/
6667
Connection reestablishConnection() throws SQLException, ClassNotFoundException;
68+
69+
default void close() throws Exception {
70+
closeConnection();
71+
}
6772
}
Lines changed: 4 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -15,8 +15,9 @@
1515
* limitations under the License.
1616
*/
1717

18-
package org.apache.flink.connector.jdbc.internal.connection;
18+
package org.apache.flink.connector.jdbc.datasource.connections;
1919

20+
import org.apache.flink.annotation.Internal;
2021
import org.apache.flink.connector.jdbc.JdbcConnectionOptions;
2122
import org.apache.flink.util.Preconditions;
2223

@@ -35,6 +36,7 @@
3536

3637
/** Simple JDBC connection provider. */
3738
@NotThreadSafe
39+
@Internal
3840
public class SimpleJdbcConnectionProvider implements JdbcConnectionProvider, Serializable {
3941

4042
private static final Logger LOG = LoggerFactory.getLogger(SimpleJdbcConnectionProvider.class);
@@ -73,8 +75,7 @@ public boolean isConnectionValid() throws SQLException {
7375
&& connection.isValid(jdbcOptions.getConnectionCheckTimeoutSeconds());
7476
}
7577

76-
private static Driver loadDriver(String driverName)
77-
throws SQLException, ClassNotFoundException {
78+
private Driver loadDriver(String driverName) throws SQLException, ClassNotFoundException {
7879
Preconditions.checkNotNull(driverName);
7980
Enumeration<Driver> drivers = DriverManager.getDrivers();
8081
while (drivers.hasMoreElements()) {
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,208 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
18+
package org.apache.flink.connector.jdbc.datasource.connections.xa;
19+
20+
import org.apache.flink.annotation.Internal;
21+
import org.apache.flink.connector.jdbc.datasource.transactions.xa.exceptions.TransientXaException;
22+
import org.apache.flink.util.function.ThrowingConsumer;
23+
24+
import org.slf4j.Logger;
25+
import org.slf4j.LoggerFactory;
26+
27+
import javax.annotation.Nullable;
28+
import javax.sql.XADataSource;
29+
import javax.transaction.xa.Xid;
30+
31+
import java.io.Serializable;
32+
import java.sql.Connection;
33+
import java.sql.SQLException;
34+
import java.util.Collection;
35+
import java.util.Deque;
36+
import java.util.HashMap;
37+
import java.util.LinkedList;
38+
import java.util.Map;
39+
import java.util.function.Supplier;
40+
41+
import static org.apache.flink.util.ExceptionUtils.rethrow;
42+
import static org.apache.flink.util.Preconditions.checkState;
43+
44+
/**
45+
* A "pooling" implementation of {@link XaConnectionProvider}. Some database implement XA such that
46+
* one connection is limited to a single transaction. As a workaround, this implementation creates a
47+
* new XA resource after each xa_start call is made (and associates it with the xid to commit
48+
* later).
49+
*/
50+
@Internal
51+
public class PoolingXaConnectionProvider implements XaConnectionProvider {
52+
private static final long serialVersionUID = 1L;
53+
54+
/** A supplier of connection provider. */
55+
public interface ConnectionProviderSupplier
56+
extends Serializable, Supplier<XaConnectionProvider> {}
57+
58+
private static final Logger LOG = LoggerFactory.getLogger(PoolingXaConnectionProvider.class);
59+
private final ConnectionProviderSupplier providerSupplier;
60+
private transient XaConnectionProvider active;
61+
private transient Map<Xid, XaConnectionProvider> mappedToXids;
62+
private transient Deque<XaConnectionProvider> pooled;
63+
64+
public static PoolingXaConnectionProvider from(
65+
Supplier<XADataSource> dataSourceSupplier, Integer timeoutSec) {
66+
return from(() -> SimpleXaConnectionProvider.from(dataSourceSupplier, timeoutSec));
67+
}
68+
69+
public static PoolingXaConnectionProvider from(ConnectionProviderSupplier facadeSupplier) {
70+
return new PoolingXaConnectionProvider(facadeSupplier);
71+
}
72+
73+
private PoolingXaConnectionProvider(ConnectionProviderSupplier facadeSupplier) {
74+
this.providerSupplier = facadeSupplier;
75+
}
76+
77+
@Override
78+
public void open() throws Exception {
79+
checkState(active == null);
80+
pooled = new LinkedList<>();
81+
mappedToXids = new HashMap<>();
82+
}
83+
84+
@Override
85+
public boolean isOpen() {
86+
return active != null && active.isOpen();
87+
}
88+
89+
@Override
90+
public void start(Xid xid) throws Exception {
91+
checkState(active == null);
92+
if (pooled.isEmpty()) {
93+
active = providerSupplier.get();
94+
active.open();
95+
} else {
96+
active = pooled.poll();
97+
}
98+
active.start(xid);
99+
mappedToXids.put(xid, active);
100+
}
101+
102+
/**
103+
* Must be called after {@link #start(Xid)} with the same {@link Xid}.
104+
*
105+
* @see XaConnectionProvider#endAndPrepare(Xid)
106+
*/
107+
@Override
108+
public void endAndPrepare(Xid xid) throws Exception {
109+
checkState(active == mappedToXids.get(xid));
110+
try {
111+
active.endAndPrepare(xid);
112+
} finally {
113+
active = null;
114+
}
115+
}
116+
117+
@Override
118+
public void commit(Xid xid, boolean ignoreUnknown) throws TransientXaException {
119+
runForXid(xid, facade -> facade.commit(xid, ignoreUnknown));
120+
}
121+
122+
@Override
123+
public void rollback(Xid xid) throws TransientXaException {
124+
runForXid(xid, facade -> facade.rollback(xid));
125+
}
126+
127+
@Override
128+
public void failAndRollback(Xid xid) throws TransientXaException {
129+
runForXid(xid, facade -> facade.failAndRollback(xid));
130+
}
131+
132+
@Override
133+
public Collection<Xid> recover() throws TransientXaException {
134+
return peekPooled().recover();
135+
}
136+
137+
@Override
138+
public void close() throws Exception {
139+
for (XaConnectionProvider facade : mappedToXids.values()) {
140+
facade.close();
141+
}
142+
for (XaConnectionProvider facade : pooled) {
143+
facade.close();
144+
}
145+
if (active != null && active.isOpen()) {
146+
active.close();
147+
}
148+
}
149+
150+
@Nullable
151+
@Override
152+
public Connection getConnection() {
153+
return active.getConnection();
154+
}
155+
156+
@Override
157+
public boolean isConnectionValid() throws SQLException {
158+
return active.isConnectionValid();
159+
}
160+
161+
@Override
162+
public Connection getOrEstablishConnection() throws SQLException, ClassNotFoundException {
163+
return active.getOrEstablishConnection();
164+
}
165+
166+
@Override
167+
public void closeConnection() {
168+
active.closeConnection();
169+
}
170+
171+
@Override
172+
public Connection reestablishConnection() throws SQLException, ClassNotFoundException {
173+
return active.reestablishConnection();
174+
}
175+
176+
// WARN: action MUST leave the facade in IDLE state (i.e. not start/end/prepare any tx)
177+
private void runForXid(
178+
Xid xid, ThrowingConsumer<XaConnectionProvider, TransientXaException> action) {
179+
XaConnectionProvider mapped = mappedToXids.remove(xid);
180+
if (mapped == null) {
181+
// a transaction can be not known during recovery
182+
LOG.debug("No XA resource found associated with XID: {}", xid);
183+
action.accept(peekPooled());
184+
} else {
185+
LOG.debug("Found mapped XA resource for XID: {} {}", xid, mapped);
186+
try {
187+
action.accept(mapped);
188+
} finally {
189+
pooled.offer(mapped);
190+
}
191+
}
192+
}
193+
194+
// WARN: the returned facade MUST be left in IDLE state (i.e. not start/end/prepare any tx)
195+
private XaConnectionProvider peekPooled() {
196+
XaConnectionProvider xaFacade = pooled.peek();
197+
if (xaFacade == null) {
198+
xaFacade = providerSupplier.get();
199+
try {
200+
xaFacade.open();
201+
} catch (Exception e) {
202+
rethrow(e);
203+
}
204+
pooled.offer(xaFacade);
205+
}
206+
return xaFacade;
207+
}
208+
}

0 commit comments

Comments
 (0)