25 | *
26 | * {@link #recover} can be used to get abandoned prepared transactions for cleanup.
27 | */
28 | @PublicEvolving
29 | public interface XaConnectionProvider extends JdbcConnectionProvider {
30 |
31 | void open() throws Exception;
32 |
33 | boolean isOpen();
34 |
35 | /** Start a new transaction. */
36 | void start(Xid xid) throws Exception;
37 |
38 | /** End and then prepare the transaction. Transaction can't be resumed afterwards. */
39 | void endAndPrepare(Xid xid) throws Exception;
40 |
41 | /**
42 | * Commit previously prepared transaction.
43 | *
44 | * @param ignoreUnknown whether to ignore {@link XAException#XAER_NOTA XAER_NOTA} error.
45 | */
46 | void commit(Xid xid, boolean ignoreUnknown) throws TransientXaException;
47 |
48 | /** Rollback previously prepared transaction. */
49 | void rollback(Xid xid) throws TransientXaException;
50 |
51 | /**
52 | * End transaction as {@link javax.transaction.xa.XAResource#TMFAIL failed}; in case of error,
53 | * try to roll it back.
54 | */
55 | void failAndRollback(Xid xid) throws TransientXaException;
56 |
57 | /**
58 | * Note: this can block on some non-MVCC databases if there are ended not prepared transactions.
59 | */
60 | Collection recover() throws TransientXaException;
61 | }
62 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-core/src/main/java/org/apache/flink/connector/jdbc/datasource/statements/JdbcQueryStatement.java:
--------------------------------------------------------------------------------
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.statements;
19 |
20 | import org.apache.flink.annotation.PublicEvolving;
21 |
22 | import java.io.Serializable;
23 | import java.sql.PreparedStatement;
24 | import java.sql.SQLException;
25 |
26 | /**
27 | * Sets {@link PreparedStatement} parameters to use in JDBC Sink based on a specific type of record.
28 | */
29 | @PublicEvolving
30 | public interface JdbcQueryStatement extends Serializable {
31 | String query();
32 |
33 | void statement(PreparedStatement ps, T record) throws SQLException;
34 | }
35 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-core/src/main/java/org/apache/flink/connector/jdbc/datasource/statements/SimpleJdbcQueryStatement.java:
--------------------------------------------------------------------------------
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.statements;
19 |
20 | import org.apache.flink.annotation.PublicEvolving;
21 | import org.apache.flink.connector.jdbc.JdbcStatementBuilder;
22 |
23 | import java.sql.PreparedStatement;
24 | import java.sql.SQLException;
25 |
26 | /** A simple implementation for {@link JdbcQueryStatement}. */
27 | @PublicEvolving
28 | public class SimpleJdbcQueryStatement implements JdbcQueryStatement {
29 | private final String query;
30 | private final JdbcStatementBuilder statement;
31 |
32 | public SimpleJdbcQueryStatement(String query, JdbcStatementBuilder statement) {
33 | this.query = query;
34 | this.statement = statement;
35 | }
36 |
37 | @Override
38 | public String query() {
39 | return query;
40 | }
41 |
42 | @Override
43 | public void statement(PreparedStatement ps, IN data) throws SQLException {
44 | statement.accept(ps, data);
45 | }
46 | }
47 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-core/src/main/java/org/apache/flink/connector/jdbc/datasource/transactions/xa/exceptions/EmptyTransactionXaException.java:
--------------------------------------------------------------------------------
1 | package org.apache.flink.connector.jdbc.datasource.transactions.xa.exceptions;
2 |
3 | import org.apache.flink.annotation.PublicEvolving;
4 | import org.apache.flink.connector.jdbc.datasource.transactions.xa.XaTransaction;
5 | import org.apache.flink.util.FlinkRuntimeException;
6 |
7 | import javax.transaction.xa.XAException;
8 | import javax.transaction.xa.Xid;
9 |
10 | /**
11 | * Thrown by {@link XaTransaction} when RM responds with {@link
12 | * javax.transaction.xa.XAResource#XA_RDONLY XA_RDONLY} indicating that the transaction doesn't
13 | * include any changes. When such a transaction is committed RM may return an error (usually, {@link
14 | * XAException#XAER_NOTA XAER_NOTA}).
15 | */
16 | @PublicEvolving
17 | public class EmptyTransactionXaException extends FlinkRuntimeException {
18 | private final Xid xid;
19 |
20 | public EmptyTransactionXaException(Xid xid) {
21 | super("end response XA_RDONLY, xid: " + xid);
22 | this.xid = xid;
23 | }
24 |
25 | public Xid getXid() {
26 | return xid;
27 | }
28 | }
29 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-core/src/main/java/org/apache/flink/connector/jdbc/datasource/transactions/xa/exceptions/TransientXaException.java:
--------------------------------------------------------------------------------
1 | package org.apache.flink.connector.jdbc.datasource.transactions.xa.exceptions;
2 |
3 | import org.apache.flink.annotation.PublicEvolving;
4 | import org.apache.flink.util.FlinkRuntimeException;
5 |
6 | import javax.transaction.xa.XAException;
7 |
8 | /**
9 | * Indicates a transient or unknown failure from the resource manager (see {@link
10 | * XAException#XA_RBTRANSIENT XA_RBTRANSIENT}, {@link XAException#XAER_RMFAIL XAER_RMFAIL}).
11 | */
12 | @PublicEvolving
13 | public class TransientXaException extends FlinkRuntimeException {
14 | public TransientXaException(XAException cause) {
15 | super(cause);
16 | }
17 | }
18 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-core/src/main/java/org/apache/flink/connector/jdbc/derby/database/DerbyFactory.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.connector.jdbc.derby.database;
20 |
21 | import org.apache.flink.annotation.Internal;
22 | import org.apache.flink.connector.jdbc.core.database.JdbcFactory;
23 | import org.apache.flink.connector.jdbc.core.database.catalog.JdbcCatalog;
24 | import org.apache.flink.connector.jdbc.core.database.dialect.JdbcDialect;
25 | import org.apache.flink.connector.jdbc.derby.database.dialect.DerbyDialect;
26 |
27 | /** Factory for {@link DerbyDialect}. */
28 | @Internal
29 | public class DerbyFactory implements JdbcFactory {
30 | @Override
31 | public boolean acceptsURL(String url) {
32 | return url.startsWith("jdbc:derby:");
33 | }
34 |
35 | @Override
36 | public JdbcDialect createDialect() {
37 | return new DerbyDialect();
38 | }
39 |
40 | @Override
41 | public JdbcCatalog createCatalog(
42 | ClassLoader classLoader,
43 | String catalogName,
44 | String defaultDatabase,
45 | String username,
46 | String pwd,
47 | String baseUrl) {
48 | throw new UnsupportedOperationException("Catalog for Derby is not supported yet.");
49 | }
50 | }
51 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-core/src/main/java/org/apache/flink/connector/jdbc/derby/database/dialect/DerbyDialectConverter.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.connector.jdbc.derby.database.dialect;
20 |
21 | import org.apache.flink.annotation.Internal;
22 | import org.apache.flink.connector.jdbc.core.database.dialect.AbstractDialectConverter;
23 | import org.apache.flink.table.types.logical.RowType;
24 |
25 | /**
26 | * Runtime converter that responsible to convert between JDBC object and Flink internal object for
27 | * Derby.
28 | */
29 | @Internal
30 | public class DerbyDialectConverter extends AbstractDialectConverter {
31 |
32 | private static final long serialVersionUID = 1L;
33 |
34 | @Override
35 | public String converterName() {
36 | return "Derby";
37 | }
38 |
39 | public DerbyDialectConverter(RowType rowType) {
40 | super(rowType);
41 | }
42 | }
43 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-core/src/main/java/org/apache/flink/connector/jdbc/internal/JdbcOutputSerializer.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.connector.jdbc.internal;
20 |
21 | import org.apache.flink.annotation.Internal;
22 | import org.apache.flink.api.common.typeutils.TypeSerializer;
23 |
24 | import java.io.Serializable;
25 |
26 | /** A Serializer that have in account the actual configuration. */
27 | @Internal
28 | public class JdbcOutputSerializer implements Serializable {
29 |
30 | private final TypeSerializer typeSerializer;
31 | private boolean objectReuse;
32 |
33 | private JdbcOutputSerializer(TypeSerializer typeSerializer, boolean objectReuse) {
34 | this.typeSerializer = typeSerializer;
35 | this.objectReuse = objectReuse;
36 | }
37 |
38 | public static JdbcOutputSerializer of(TypeSerializer typeSerializer) {
39 | return of(typeSerializer, false);
40 | }
41 |
42 | public static JdbcOutputSerializer of(
43 | TypeSerializer typeSerializer, boolean objectReuse) {
44 | return new JdbcOutputSerializer<>(typeSerializer, objectReuse);
45 | }
46 |
47 | public JdbcOutputSerializer withObjectReuseEnabled(boolean enabled) {
48 | this.objectReuse = enabled;
49 | return this;
50 | }
51 |
52 | public T serialize(T record) {
53 | return this.objectReuse ? typeSerializer.copy(record) : record;
54 | }
55 | }
56 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-core/src/main/java/org/apache/flink/connector/jdbc/internal/executor/JdbcBatchStatementExecutor.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.connector.jdbc.internal.executor;
20 |
21 | import org.apache.flink.annotation.Internal;
22 | import org.apache.flink.connector.jdbc.JdbcStatementBuilder;
23 |
24 | import java.sql.Connection;
25 | import java.sql.SQLException;
26 | import java.util.function.Function;
27 |
28 | /** Executes the given JDBC statement in batch for the accumulated records. */
29 | @Internal
30 | public interface JdbcBatchStatementExecutor {
31 |
32 | /** Create statements from connection. */
33 | void prepareStatements(Connection connection) throws SQLException;
34 |
35 | void addToBatch(T record) throws SQLException;
36 |
37 | /** Submits a batch of commands to the database for execution. */
38 | void executeBatch() throws SQLException;
39 |
40 | /** Close JDBC related statements. */
41 | void closeStatements() throws SQLException;
42 |
43 | static JdbcBatchStatementExecutor keyed(
44 | String sql, Function keyExtractor, JdbcStatementBuilder statementBuilder) {
45 | return new KeyedBatchStatementExecutor<>(sql, keyExtractor, statementBuilder);
46 | }
47 |
48 | static JdbcBatchStatementExecutor simple(
49 | String sql, JdbcStatementBuilder paramSetter) {
50 | return new SimpleBatchStatementExecutor<>(sql, paramSetter);
51 | }
52 | }
53 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-core/src/main/java/org/apache/flink/connector/jdbc/internal/options/JdbcInsertOptions.java:
--------------------------------------------------------------------------------
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.internal.options;
19 |
20 | import org.apache.flink.util.Preconditions;
21 |
22 | import java.util.stream.IntStream;
23 |
24 | /** JDBC sink insert options. */
25 | public class JdbcInsertOptions extends JdbcTypedQueryOptions {
26 |
27 | private static final long serialVersionUID = 1L;
28 |
29 | private final String query;
30 |
31 | public JdbcInsertOptions(String query, int[] typesArray) {
32 | super(typesArray);
33 | this.query = Preconditions.checkNotNull(query, "query is empty");
34 | }
35 |
36 | public String getQuery() {
37 | return query;
38 | }
39 |
40 | public static JdbcInsertOptions from(String query, int firstFieldType, int... nextFieldTypes) {
41 | return new JdbcInsertOptions(query, concat(firstFieldType, nextFieldTypes));
42 | }
43 |
44 | private static int[] concat(int first, int... next) {
45 | if (next == null || next.length == 0) {
46 | return new int[] {first};
47 | } else {
48 | return IntStream.concat(IntStream.of(new int[] {first}), IntStream.of(next)).toArray();
49 | }
50 | }
51 | }
52 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-core/src/main/java/org/apache/flink/connector/jdbc/internal/options/JdbcTypedQueryOptions.java:
--------------------------------------------------------------------------------
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.internal.options;
19 |
20 | import javax.annotation.Nullable;
21 |
22 | import java.io.Serializable;
23 |
24 | /** Jdbc query type options. */
25 | abstract class JdbcTypedQueryOptions implements Serializable {
26 |
27 | @Nullable private final int[] fieldTypes;
28 |
29 | JdbcTypedQueryOptions(int[] fieldTypes) {
30 | this.fieldTypes = fieldTypes;
31 | }
32 |
33 | public int[] getFieldTypes() {
34 | return fieldTypes;
35 | }
36 |
37 | public abstract static class JdbcUpdateQueryOptionsBuilder<
38 | T extends JdbcUpdateQueryOptionsBuilder> {
39 | int[] fieldTypes;
40 |
41 | protected abstract T self();
42 |
43 | public T withFieldTypes(int[] fieldTypes) {
44 | this.fieldTypes = fieldTypes;
45 | return self();
46 | }
47 | }
48 | }
49 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-core/src/main/java/org/apache/flink/connector/jdbc/split/JdbcGenericParameterValuesProvider.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.connector.jdbc.split;
20 |
21 | import org.apache.flink.annotation.PublicEvolving;
22 | import org.apache.flink.connector.jdbc.JdbcInputFormat;
23 |
24 | import java.io.Serializable;
25 |
26 | /**
27 | * This splits generator actually does nothing but wrapping the query parameters computed by the
28 | * user before creating the {@link JdbcInputFormat} instance.
29 | */
30 | @PublicEvolving
31 | public class JdbcGenericParameterValuesProvider implements JdbcParameterValuesProvider {
32 |
33 | private final Serializable[][] parameters;
34 |
35 | public JdbcGenericParameterValuesProvider(Serializable[][] parameters) {
36 | this.parameters = parameters;
37 | }
38 |
39 | @Override
40 | public Serializable[][] getParameterValues() {
41 | // do nothing...precomputed externally
42 | return parameters;
43 | }
44 | }
45 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-core/src/main/java/org/apache/flink/connector/jdbc/split/JdbcParameterValuesProvider.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.connector.jdbc.split;
20 |
21 | import org.apache.flink.annotation.PublicEvolving;
22 | import org.apache.flink.connector.jdbc.JdbcInputFormat;
23 |
24 | import java.io.Serializable;
25 |
26 | /**
27 | * This interface is used by the {@link JdbcInputFormat} to compute the list of parallel query to
28 | * run (i.e. splits). Each query will be parameterized using a row of the matrix provided by each
29 | * {@link JdbcParameterValuesProvider} implementation.
30 | */
31 | @PublicEvolving
32 | public interface JdbcParameterValuesProvider extends Serializable {
33 |
34 | /** Returns the necessary parameters array to use for query in parallel a table. */
35 | Serializable[][] getParameterValues();
36 |
37 | /** Get the latest optional state data. */
38 | default Serializable getLatestOptionalState() {
39 | return null;
40 | }
41 |
42 | /** Set the optional state data. */
43 | default void setOptionalState(Serializable optionalState) {}
44 | }
45 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-core/src/main/java/org/apache/flink/connector/jdbc/statement/StatementFactory.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.connector.jdbc.statement;
20 |
21 | import java.sql.Connection;
22 | import java.sql.SQLException;
23 |
24 | /** A factory to create {@link FieldNamedPreparedStatement} with the given {@link Connection}. */
25 | public interface StatementFactory {
26 |
27 | /** Creates {@link FieldNamedPreparedStatement} with the given {@link Connection}. */
28 | FieldNamedPreparedStatement createStatement(Connection connection) throws SQLException;
29 | }
30 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-core/src/main/resources/META-INF/services/org.apache.flink.connector.jdbc.core.database.JdbcFactory:
--------------------------------------------------------------------------------
1 | # Licensed to the Apache Software Foundation (ASF) under one or more
2 | # contributor license agreements. See the NOTICE file distributed with
3 | # this work for additional information regarding copyright ownership.
4 | # The ASF licenses this file to You under the Apache License, Version 2.0
5 | # (the "License"); you may not use this file except in compliance with
6 | # the License. You may obtain a copy of the License at
7 | #
8 | # http://www.apache.org/licenses/LICENSE-2.0
9 | #
10 | # Unless required by applicable law or agreed to in writing, software
11 | # distributed under the License is distributed on an "AS IS" BASIS,
12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | # See the License for the specific language governing permissions and
14 | # limitations under the License.
15 |
16 | org.apache.flink.connector.jdbc.derby.database.DerbyFactory
17 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-core/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory:
--------------------------------------------------------------------------------
1 | # Licensed to the Apache Software Foundation (ASF) under one or more
2 | # contributor license agreements. See the NOTICE file distributed with
3 | # this work for additional information regarding copyright ownership.
4 | # The ASF licenses this file to You under the Apache License, Version 2.0
5 | # (the "License"); you may not use this file except in compliance with
6 | # the License. You may obtain a copy of the License at
7 | #
8 | # http://www.apache.org/licenses/LICENSE-2.0
9 | #
10 | # Unless required by applicable law or agreed to in writing, software
11 | # distributed under the License is distributed on an "AS IS" BASIS,
12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | # See the License for the specific language governing permissions and
14 | # limitations under the License.
15 |
16 | org.apache.flink.connector.jdbc.core.table.JdbcDynamicTableFactory
17 | org.apache.flink.connector.jdbc.core.database.catalog.factory.JdbcCatalogFactory
18 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-core/src/test/java/org/apache/flink/connector/base/source/reader/splitreader/TestingSplitsChange.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.connector.base.source.reader.splitreader;
20 |
21 | import org.apache.flink.connector.jdbc.core.datastream.source.split.JdbcSourceSplit;
22 |
23 | import java.util.List;
24 |
25 | /** Test util class for {@link SplitsChange}. */
26 | public class TestingSplitsChange extends SplitsChange {
27 |
28 | public TestingSplitsChange(List splits) {
29 | super(splits);
30 | }
31 | }
32 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-core/src/test/java/org/apache/flink/connector/jdbc/JdbcTestCheckpoint.java:
--------------------------------------------------------------------------------
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;
19 |
20 | /** Holds id and indices of items in {@link JdbcTestFixture#TEST_DATA}. */
21 | public class JdbcTestCheckpoint {
22 | public final long id;
23 | public final int[] dataItemsIdx;
24 |
25 | JdbcTestCheckpoint(long id, int... dataItemsIdx) {
26 | this.id = id;
27 | this.dataItemsIdx = dataItemsIdx;
28 | }
29 |
30 | public JdbcTestCheckpoint withCheckpointId(long id) {
31 | return new JdbcTestCheckpoint(id, dataItemsIdx);
32 | }
33 | }
34 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-core/src/test/java/org/apache/flink/connector/jdbc/core/database/catalog/AbstractJdbcCatalogTest.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.connector.jdbc.core.database.catalog;
20 |
21 | import org.junit.jupiter.api.Test;
22 |
23 | import static org.assertj.core.api.Assertions.assertThatThrownBy;
24 |
25 | /** Test for {@link AbstractJdbcCatalog}. */
26 | class AbstractJdbcCatalogTest {
27 |
28 | @Test
29 | void testJdbcUrl() {
30 | AbstractJdbcCatalog.validateJdbcUrl("jdbc:dialect://localhost:1234/");
31 | AbstractJdbcCatalog.validateJdbcUrl("jdbc:dialect://localhost:1234");
32 | }
33 |
34 | @Test
35 | void testInvalidJdbcUrl() {
36 | assertThatThrownBy(
37 | () ->
38 | AbstractJdbcCatalog.validateJdbcUrl(
39 | "jdbc:dialect://localhost:1234/db"))
40 | .isInstanceOf(IllegalArgumentException.class);
41 | }
42 | }
43 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-core/src/test/java/org/apache/flink/connector/jdbc/core/datastream/sink/AtLeastOnceJdbcSinkTest.java:
--------------------------------------------------------------------------------
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.core.datastream.sink;
19 |
20 | import org.apache.flink.connector.jdbc.JdbcConnectionOptions;
21 |
22 | /** Smoke tests for the {@link JdbcSink} and the underlying classes. */
23 | class AtLeastOnceJdbcSinkTest extends BaseJdbcSinkTest {
24 |
25 | @Override
26 | protected JdbcSink finishSink(JdbcSinkBuilder builder) {
27 | return builder.buildAtLeastOnce(
28 | new JdbcConnectionOptions.JdbcConnectionOptionsBuilder()
29 | .withUrl(getMetadata().getJdbcUrl())
30 | .withUsername(getMetadata().getUsername())
31 | .withPassword(getMetadata().getPassword())
32 | .withDriverName(getMetadata().getDriverClass())
33 | .build());
34 | }
35 | }
36 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-core/src/test/java/org/apache/flink/connector/jdbc/core/datastream/sink/ExactlyOnceJdbcSinkTest.java:
--------------------------------------------------------------------------------
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.core.datastream.sink;
19 |
20 | import org.apache.flink.connector.jdbc.JdbcExactlyOnceOptions;
21 | import org.apache.flink.connector.jdbc.JdbcExecutionOptions;
22 |
23 | /** Smoke tests for the {@link JdbcSink} and the underlying classes. */
24 | class ExactlyOnceJdbcSinkTest extends BaseJdbcSinkTest {
25 |
26 | @Override
27 | protected JdbcSink finishSink(JdbcSinkBuilder builder) {
28 | return builder.withExecutionOptions(
29 | JdbcExecutionOptions.builder().withMaxRetries(0).build())
30 | .buildExactlyOnce(
31 | JdbcExactlyOnceOptions.defaults(), getMetadata().getXaSourceSupplier());
32 | }
33 | }
34 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-core/src/test/java/org/apache/flink/connector/jdbc/core/datastream/sink/writer/JdbcWriterStateSerializerTest.java:
--------------------------------------------------------------------------------
1 | package org.apache.flink.connector.jdbc.core.datastream.sink.writer;
2 |
3 | import org.apache.flink.api.common.JobID;
4 | import org.apache.flink.connector.jdbc.datasource.transactions.xa.domain.TransactionId;
5 |
6 | import org.junit.jupiter.api.Test;
7 |
8 | import java.io.IOException;
9 | import java.util.Arrays;
10 |
11 | import static org.assertj.core.api.Assertions.assertThat;
12 |
13 | /**
14 | * Smoke test for {@link
15 | * org.apache.flink.connector.jdbc.core.datastream.sink.writer.JdbcWriterStateSerializer}.
16 | */
17 | class JdbcWriterStateSerializerTest {
18 |
19 | @Test
20 | void testBasicSerDe() throws IOException {
21 | TransactionId baseTid =
22 | TransactionId.create(
23 | JobID.fromHexString("6b64d8a9a951e2e8767ae952ad951706").getBytes(), 1, 2);
24 |
25 | JdbcWriterState original =
26 | JdbcWriterState.of(
27 | Arrays.asList(baseTid.withBranch(1001L), baseTid.withBranch(1002L)),
28 | Arrays.asList(baseTid.withBranch(2001L), baseTid.withBranch(2002L)));
29 |
30 | JdbcWriterStateSerializer tester = new JdbcWriterStateSerializer();
31 |
32 | byte[] serialized = tester.serialize(original);
33 | JdbcWriterState deserialized = tester.deserialize(tester.getVersion(), serialized);
34 |
35 | assertThat(deserialized).isEqualTo(original);
36 | }
37 | }
38 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-core/src/test/java/org/apache/flink/connector/jdbc/core/datastream/sink/writer/JdbcWriterStateTest.java:
--------------------------------------------------------------------------------
1 | package org.apache.flink.connector.jdbc.core.datastream.sink.writer;
2 |
3 | import org.apache.flink.api.common.JobID;
4 | import org.apache.flink.connector.jdbc.datasource.transactions.xa.domain.TransactionId;
5 |
6 | import org.junit.jupiter.api.Test;
7 |
8 | import java.util.Arrays;
9 |
10 | import static org.assertj.core.api.Assertions.assertThat;
11 |
12 | /**
13 | * Smoke test for {@link
14 | * org.apache.flink.connector.jdbc.core.datastream.sink.writer.JdbcWriterState}.
15 | */
16 | class JdbcWriterStateTest {
17 |
18 | @Test
19 | void testEquals() {
20 | assertThat(JdbcWriterState.empty()).isEqualTo(JdbcWriterState.empty());
21 |
22 | TransactionId baseTid =
23 | TransactionId.create(
24 | JobID.fromHexString("6b64d8a9a951e2e8767ae952ad951706").getBytes(), 1, 2);
25 |
26 | JdbcWriterState state1 =
27 | JdbcWriterState.of(
28 | Arrays.asList(baseTid.withBranch(1001L), baseTid.withBranch(1002L)),
29 | Arrays.asList(baseTid.withBranch(2001L), baseTid.withBranch(2002L)));
30 | JdbcWriterState state2 =
31 | JdbcWriterState.of(
32 | Arrays.asList(baseTid.withBranch(1001L), baseTid.withBranch(1002L)),
33 | Arrays.asList(baseTid.withBranch(2001L), baseTid.withBranch(2002L)));
34 |
35 | assertThat(state1).isEqualTo(state2);
36 | }
37 | }
38 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-core/src/test/java/org/apache/flink/connector/jdbc/datasource/transactions/xa/xid/XidSerializersTest.java:
--------------------------------------------------------------------------------
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.transactions.xa.xid;
19 |
20 | import org.apache.flink.api.common.typeutils.SerializerTestBase;
21 | import org.apache.flink.api.common.typeutils.TypeSerializer;
22 |
23 | import javax.transaction.xa.Xid;
24 |
25 | /** XaSerializersTest. */
26 | class XidSerializersTest extends SerializerTestBase {
27 |
28 | @Override
29 | protected TypeSerializer createSerializer() {
30 | return new XidSerializer();
31 | }
32 |
33 | @Override
34 | protected int getLength() {
35 | return -1;
36 | }
37 |
38 | @Override
39 | protected Class getTypeClass() {
40 | return Xid.class;
41 | }
42 |
43 | @Override
44 | protected Xid[] getTestData() {
45 | return new Xid[] {XaXidTest.XID};
46 | }
47 | }
48 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-core/src/test/java/org/apache/flink/connector/jdbc/derby/DerbyTestBase.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.connector.jdbc.derby;
20 |
21 | import org.apache.flink.connector.jdbc.derby.testutils.DerbyDatabase;
22 | import org.apache.flink.connector.jdbc.testutils.DatabaseMetadata;
23 | import org.apache.flink.connector.jdbc.testutils.DatabaseTest;
24 |
25 | import org.junit.jupiter.api.extension.ExtendWith;
26 |
27 | /** Base class for Derby testing. */
28 | @ExtendWith(DerbyDatabase.class)
29 | public interface DerbyTestBase extends DatabaseTest {
30 |
31 | @Override
32 | default DatabaseMetadata getMetadata() {
33 | return DerbyDatabase.getMetadata();
34 | }
35 | }
36 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-core/src/test/java/org/apache/flink/connector/jdbc/derby/table/DerbyDynamicTableSinkITCase.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.connector.jdbc.derby.table;
20 |
21 | import org.apache.flink.connector.jdbc.core.table.sink.JdbcDynamicTableSinkITCase;
22 | import org.apache.flink.connector.jdbc.derby.DerbyTestBase;
23 | import org.apache.flink.connector.jdbc.derby.database.dialect.DerbyDialect;
24 |
25 | /** The Table Sink ITCase for {@link DerbyDialect}. */
26 | class DerbyDynamicTableSinkITCase extends JdbcDynamicTableSinkITCase implements DerbyTestBase {}
27 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-core/src/test/java/org/apache/flink/connector/jdbc/derby/testutils/DerbyMetadata.java:
--------------------------------------------------------------------------------
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.derby.testutils;
19 |
20 | import org.apache.flink.connector.jdbc.testutils.DatabaseMetadata;
21 |
22 | import org.apache.derby.jdbc.EmbeddedXADataSource;
23 |
24 | import javax.sql.XADataSource;
25 |
26 | /** Derby Metadata. */
27 | public class DerbyMetadata implements DatabaseMetadata {
28 | private final String dbName;
29 |
30 | public DerbyMetadata(String schemaName) {
31 | dbName = "memory:" + schemaName;
32 | }
33 |
34 | public String getDbName() {
35 | return dbName;
36 | }
37 |
38 | @Override
39 | public String getJdbcUrl() {
40 | return String.format("jdbc:derby:%s", dbName);
41 | }
42 |
43 | @Override
44 | public String getJdbcUrlWithCredentials() {
45 | return getJdbcUrl();
46 | }
47 |
48 | @Override
49 | public String getUsername() {
50 | return "";
51 | }
52 |
53 | @Override
54 | public String getPassword() {
55 | return "";
56 | }
57 |
58 | @Override
59 | public XADataSource buildXaDataSource() {
60 | EmbeddedXADataSource ds = new EmbeddedXADataSource();
61 | ds.setDatabaseName(dbName);
62 | return ds;
63 | }
64 |
65 | @Override
66 | public String getDriverClass() {
67 | return "org.apache.derby.jdbc.EmbeddedDriver";
68 | }
69 |
70 | @Override
71 | public String getVersion() {
72 | return "derby:memory";
73 | }
74 | }
75 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-core/src/test/java/org/apache/flink/connector/jdbc/fakedb/FakeDBUtils.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.connector.jdbc.fakedb;
20 |
21 | /** Utilities and constants for FakeDB. */
22 | public class FakeDBUtils {
23 | public static final String URL_PREFIX = "jdbc:fake:";
24 |
25 | public static final String TEST_DB_URL = composeDBUrl("test");
26 | public static final String TEST_DB_INVALID_URL = "jdbc:no-existing-driver:test";
27 |
28 | public static final String DRIVER1_CLASS_NAME =
29 | "org.apache.flink.connector.jdbc.fakedb.driver.FakeDriver1";
30 | public static final String DRIVER2_CLASS_NAME =
31 | "org.apache.flink.connector.jdbc.fakedb.driver.FakeDriver2";
32 | public static final String DRIVER3_CLASS_NAME =
33 | "org.apache.flink.connector.jdbc.fakedb.driver.FakeDriver3";
34 |
35 | public static String composeDBUrl(String db) {
36 | return URL_PREFIX + db;
37 | }
38 |
39 | public static boolean acceptsUrl(String url) {
40 | return url.startsWith(URL_PREFIX);
41 | }
42 | }
43 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-core/src/test/java/org/apache/flink/connector/jdbc/fakedb/driver/FakeConnection1.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.connector.jdbc.fakedb.driver;
20 |
21 | import java.util.Properties;
22 |
23 | /** Sql connection created by {@link FakeDriver1#connect(String, Properties)}. */
24 | public class FakeConnection1 extends FakeConnection {}
25 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-core/src/test/java/org/apache/flink/connector/jdbc/fakedb/driver/FakeConnection2.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.connector.jdbc.fakedb.driver;
20 |
21 | import java.util.Properties;
22 |
23 | /** Sql connection created by {@link FakeDriver2#connect(String, Properties)}. */
24 | public class FakeConnection2 extends FakeConnection {}
25 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-core/src/test/java/org/apache/flink/connector/jdbc/fakedb/driver/FakeConnection3.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.connector.jdbc.fakedb.driver;
20 |
21 | import java.util.Properties;
22 |
23 | /** Sql connection created by {@link FakeDriver3#connect(String, Properties)}. */
24 | public class FakeConnection3 extends FakeConnection {}
25 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-core/src/test/java/org/apache/flink/connector/jdbc/fakedb/driver/FakeDriver3.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.connector.jdbc.fakedb.driver;
20 |
21 | import org.apache.flink.connector.jdbc.fakedb.FakeDBUtils;
22 |
23 | import java.sql.Connection;
24 | import java.sql.Driver;
25 | import java.sql.DriverPropertyInfo;
26 | import java.sql.SQLException;
27 | import java.sql.SQLFeatureNotSupportedException;
28 | import java.util.Properties;
29 | import java.util.logging.Logger;
30 |
31 | /** Yet another {@link Driver} for FakeDB. */
32 | public class FakeDriver3 implements Driver {
33 |
34 | @Override
35 | public Connection connect(String url, Properties info) throws SQLException {
36 | if (!acceptsURL(url)) {
37 | return null;
38 | }
39 | return new FakeConnection3();
40 | }
41 |
42 | @Override
43 | public boolean acceptsURL(String url) throws SQLException {
44 | return FakeDBUtils.acceptsUrl(url);
45 | }
46 |
47 | @Override
48 | public DriverPropertyInfo[] getPropertyInfo(String url, Properties info) throws SQLException {
49 | return new DriverPropertyInfo[0];
50 | }
51 |
52 | @Override
53 | public int getMajorVersion() {
54 | return 0;
55 | }
56 |
57 | @Override
58 | public int getMinorVersion() {
59 | return 0;
60 | }
61 |
62 | @Override
63 | public boolean jdbcCompliant() {
64 | return false;
65 | }
66 |
67 | @Override
68 | public Logger getParentLogger() throws SQLFeatureNotSupportedException {
69 | throw new SQLFeatureNotSupportedException();
70 | }
71 | }
72 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-core/src/test/java/org/apache/flink/connector/jdbc/h2/H2XaTestBase.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.connector.jdbc.h2;
20 |
21 | import org.apache.flink.connector.jdbc.h2.testutils.H2XaDatabase;
22 | import org.apache.flink.connector.jdbc.testutils.DatabaseMetadata;
23 | import org.apache.flink.connector.jdbc.testutils.DatabaseTest;
24 |
25 | import org.junit.jupiter.api.extension.ExtendWith;
26 |
27 | /** Base class for H2 Xa testing. */
28 | @ExtendWith(H2XaDatabase.class)
29 | public interface H2XaTestBase extends DatabaseTest {
30 |
31 | @Override
32 | default DatabaseMetadata getMetadata() {
33 | return H2XaDatabase.getMetadata();
34 | }
35 | }
36 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-core/src/test/java/org/apache/flink/connector/jdbc/h2/testutils/H2Metadata.java:
--------------------------------------------------------------------------------
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.h2.testutils;
19 |
20 | import org.apache.flink.connector.jdbc.h2.testutils.xa.H2XaDsWrapper;
21 | import org.apache.flink.connector.jdbc.testutils.DatabaseMetadata;
22 |
23 | import javax.sql.XADataSource;
24 |
25 | /** H2 Metadata. */
26 | public class H2Metadata implements DatabaseMetadata {
27 |
28 | private final String schema;
29 |
30 | public H2Metadata(String schema) {
31 | this.schema = schema;
32 | }
33 |
34 | @Override
35 | public String getJdbcUrl() {
36 | return String.format("jdbc:h2:mem:%s", schema);
37 | }
38 |
39 | @Override
40 | public String getJdbcUrlWithCredentials() {
41 | return getJdbcUrl();
42 | }
43 |
44 | @Override
45 | public String getUsername() {
46 | return "";
47 | }
48 |
49 | @Override
50 | public String getPassword() {
51 | return "";
52 | }
53 |
54 | @Override
55 | public XADataSource buildXaDataSource() {
56 | final org.h2.jdbcx.JdbcDataSource ds = new org.h2.jdbcx.JdbcDataSource();
57 | ds.setUrl(getJdbcUrl());
58 | return new H2XaDsWrapper(ds);
59 | }
60 |
61 | @Override
62 | public String getDriverClass() {
63 | return "org.h2.Driver";
64 | }
65 |
66 | @Override
67 | public String getVersion() {
68 | return "h2:mem";
69 | }
70 | }
71 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-core/src/test/java/org/apache/flink/connector/jdbc/h2/testutils/xa/package-info.java:
--------------------------------------------------------------------------------
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 | /**
19 | * This package holds some workarounds for the H2 XA client, plus {@link
20 | * org.apache.flink.connector.jdbc.h2.testutils.H2Metadata}. Used only for testing.
21 | */
22 | package org.apache.flink.connector.jdbc.h2.testutils.xa;
23 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-core/src/test/java/org/apache/flink/connector/jdbc/internal/JdbcOutputSerializerTest.java:
--------------------------------------------------------------------------------
1 | package org.apache.flink.connector.jdbc.internal;
2 |
3 | import org.apache.flink.api.common.ExecutionConfig;
4 | import org.apache.flink.api.common.typeinfo.TypeInformation;
5 | import org.apache.flink.api.common.typeutils.TypeSerializer;
6 | import org.apache.flink.types.Row;
7 |
8 | import org.junit.jupiter.api.Test;
9 |
10 | import static org.assertj.core.api.Assertions.assertThat;
11 |
12 | class JdbcOutputSerializerTest {
13 |
14 | @Test
15 | void testSerializer() {
16 | TypeInformation typeInformation = TypeInformation.of(Row.class);
17 | TypeSerializer typeSerializer =
18 | typeInformation.createSerializer(new ExecutionConfig().getSerializerConfig());
19 | JdbcOutputSerializer serializer = JdbcOutputSerializer.of(typeSerializer);
20 |
21 | Row original = Row.of(123);
22 | Row noReuse = serializer.withObjectReuseEnabled(false).serialize(original);
23 | Row withReuse = serializer.withObjectReuseEnabled(true).serialize(original);
24 |
25 | assertThat(noReuse).isEqualTo(original);
26 | assertThat(withReuse).isEqualTo(original);
27 |
28 | original.setField(0, 321);
29 |
30 | // if disable object is reusable
31 | assertThat(noReuse).isEqualTo(original);
32 | // if enabled object is duplicate
33 | assertThat(withReuse).isNotEqualTo(original);
34 | }
35 | }
36 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-core/src/test/java/org/apache/flink/connector/jdbc/testutils/DatabaseResource.java:
--------------------------------------------------------------------------------
1 | package org.apache.flink.connector.jdbc.testutils;
2 |
3 | import org.junit.jupiter.api.extension.ExtensionContext.Store.CloseableResource;
4 |
5 | /** Database resource for testing. */
6 | public interface DatabaseResource extends CloseableResource {
7 |
8 | void start();
9 |
10 | void stop();
11 |
12 | default void close() throws Throwable {
13 | stop();
14 | }
15 | }
16 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-core/src/test/java/org/apache/flink/connector/jdbc/testutils/DatabaseTest.java:
--------------------------------------------------------------------------------
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.testutils;
19 |
20 | import java.util.Collections;
21 | import java.util.List;
22 |
23 | /** Base interface for tests that have dependency in a database. */
24 | public interface DatabaseTest {
25 |
26 | DatabaseMetadata getMetadata();
27 |
28 | default List getManagedTables() {
29 | return Collections.emptyList();
30 | }
31 | }
32 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-core/src/test/java/org/apache/flink/connector/jdbc/testutils/JdbcITCaseBase.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.connector.jdbc.testutils;
20 |
21 | import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
22 | import org.apache.flink.test.junit5.MiniClusterExtension;
23 |
24 | import org.junit.jupiter.api.extension.RegisterExtension;
25 |
26 | /** A base for ITCase implementations. */
27 | public interface JdbcITCaseBase {
28 |
29 | @RegisterExtension
30 | MiniClusterExtension MINI_CLUSTER =
31 | new MiniClusterExtension(
32 | new MiniClusterResourceConfiguration.Builder()
33 | .setNumberTaskManagers(2)
34 | .setNumberSlotsPerTaskManager(2)
35 | .build());
36 | }
37 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-core/src/test/java/org/apache/flink/connector/jdbc/testutils/TableManaged.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.connector.jdbc.testutils;
20 |
21 | import java.sql.Connection;
22 | import java.sql.SQLException;
23 |
24 | /** Table that can be manage by {@link DatabaseExtension}. */
25 | public interface TableManaged {
26 |
27 | String getTableName();
28 |
29 | void createTable(Connection conn) throws SQLException;
30 |
31 | void deleteTable(Connection conn) throws SQLException;
32 |
33 | void dropTable(Connection conn) throws SQLException;
34 | }
35 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-core/src/test/java/org/apache/flink/connector/jdbc/testutils/functions/JdbcResultSetBuilder.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.connector.jdbc.testutils.functions;
20 |
21 | import java.io.Serializable;
22 | import java.sql.ResultSet;
23 | import java.sql.SQLException;
24 | import java.util.List;
25 |
26 | /** ResultSet builder. * */
27 | @FunctionalInterface
28 | public interface JdbcResultSetBuilder extends Serializable {
29 | List accept(ResultSet rs) throws SQLException;
30 | }
31 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-core/src/test/java/org/apache/flink/connector/jdbc/testutils/resources/DockerResource.java:
--------------------------------------------------------------------------------
1 | package org.apache.flink.connector.jdbc.testutils.resources;
2 |
3 | import org.apache.flink.connector.jdbc.testutils.DatabaseResource;
4 |
5 | import com.github.dockerjava.api.DockerClient;
6 | import org.slf4j.Logger;
7 | import org.slf4j.LoggerFactory;
8 | import org.testcontainers.DockerClientFactory;
9 | import org.testcontainers.containers.GenericContainer;
10 | import org.testcontainers.containers.JdbcDatabaseContainer;
11 |
12 | import java.util.Arrays;
13 |
14 | /** Docker based database resource. */
15 | public class DockerResource implements DatabaseResource {
16 |
17 | protected static final Logger LOG = LoggerFactory.getLogger(DockerResource.class);
18 |
19 | private final JdbcDatabaseContainer> container;
20 |
21 | public DockerResource(JdbcDatabaseContainer> container) {
22 | this.container = container;
23 | }
24 |
25 | @Override
26 | public void start() {
27 | this.container.start();
28 | }
29 |
30 | @Override
31 | public void stop() {
32 | this.container.stop();
33 | }
34 |
35 | @Override
36 | public void close() throws Throwable {
37 | stop();
38 | cleanContainers(container);
39 | }
40 |
41 | public static void cleanContainers(GenericContainer> container) {
42 | try {
43 | DockerClient client = DockerClientFactory.instance().client();
44 | // client.removeImageCmd(container.getDockerImageName()).exec();
45 | client.listImagesCmd().exec().stream()
46 | .filter(
47 | image ->
48 | Arrays.stream(image.getRepoTags())
49 | .anyMatch(
50 | tag ->
51 | !tag.contains("testcontainers/ryuk")
52 | && !tag.contains(
53 | container
54 | .getDockerImageName())))
55 | .forEach(image -> client.removeImageCmd(image.getId()).exec());
56 |
57 | } catch (Exception ignore) {
58 | LOG.warn("Error deleting image.");
59 | }
60 | }
61 | }
62 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-core/src/test/java/org/apache/flink/connector/jdbc/testutils/resources/MemoryResource.java:
--------------------------------------------------------------------------------
1 | package org.apache.flink.connector.jdbc.testutils.resources;
2 |
3 | import org.apache.flink.connector.jdbc.testutils.DatabaseResource;
4 |
5 | /** Memory based database resource. */
6 | public interface MemoryResource extends DatabaseResource {}
7 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-core/src/test/java/org/apache/flink/connector/jdbc/testutils/tables/TableBuilder.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.connector.jdbc.testutils.tables;
20 |
21 | import org.apache.flink.table.types.DataType;
22 |
23 | /** Table builder. * */
24 | public final class TableBuilder {
25 |
26 | public static TableRow tableRow(String name, TableField... fields) {
27 | return new TableRow(name, fields);
28 | }
29 |
30 | public static TableField field(String name, DataType dataType) {
31 | return field(name, null, dataType);
32 | }
33 |
34 | public static TableField field(String name, TableField.DbType dbType, DataType dataType) {
35 | return createField(name, dbType, dataType, false);
36 | }
37 |
38 | public static TableField pkField(String name, DataType dataType) {
39 | return pkField(name, null, dataType);
40 | }
41 |
42 | public static TableField pkField(String name, TableField.DbType dbType, DataType dataType) {
43 | return createField(name, dbType, dataType, true);
44 | }
45 |
46 | public static TableField.DbType dbType(String type) {
47 | return new TableField.DbType(type);
48 | }
49 |
50 | private static TableField createField(
51 | String name, TableField.DbType dbType, DataType dataType, boolean pkField) {
52 | return new TableField(name, dataType, dbType, pkField);
53 | }
54 | }
55 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-core/src/test/java/org/apache/flink/connector/jdbc/utils/JdbcTypeUtilTest.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.connector.jdbc.utils;
20 |
21 | import org.apache.flink.table.types.logical.LogicalTypeRoot;
22 |
23 | import org.junit.jupiter.api.Test;
24 |
25 | import java.sql.Types;
26 |
27 | import static org.apache.flink.connector.jdbc.utils.JdbcTypeUtil.logicalTypeToSqlType;
28 | import static org.assertj.core.api.Assertions.assertThat;
29 | import static org.assertj.core.api.Assertions.assertThatThrownBy;
30 |
31 | /** Testing the type conversions from Flink to SQL types. */
32 | class JdbcTypeUtilTest {
33 |
34 | @Test
35 | void testTypeConversions() {
36 | assertThat(logicalTypeToSqlType(LogicalTypeRoot.INTEGER)).isEqualTo(Types.INTEGER);
37 | testUnsupportedType(LogicalTypeRoot.RAW);
38 | testUnsupportedType(LogicalTypeRoot.MAP);
39 | }
40 |
41 | private static void testUnsupportedType(LogicalTypeRoot logicalTypeRoot) {
42 | assertThatThrownBy(() -> logicalTypeToSqlType(logicalTypeRoot))
43 | .isInstanceOf(IllegalArgumentException.class);
44 | }
45 | }
46 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-core/src/test/resources/META-INF/services/java.sql.Driver:
--------------------------------------------------------------------------------
1 | # Licensed to the Apache Software Foundation (ASF) under one or more
2 | # contributor license agreements. See the NOTICE file distributed with
3 | # this work for additional information regarding copyright ownership.
4 | # The ASF licenses this file to You under the Apache License, Version 2.0
5 | # (the "License"); you may not use this file except in compliance with
6 | # the License. You may obtain a copy of the License at
7 | #
8 | # http://www.apache.org/licenses/LICENSE-2.0
9 | #
10 | # Unless required by applicable law or agreed to in writing, software
11 | # distributed under the License is distributed on an "AS IS" BASIS,
12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | # See the License for the specific language governing permissions and
14 | # limitations under the License.
15 |
16 | org.apache.flink.connector.jdbc.fakedb.driver.FakeDriver1
17 | org.apache.flink.connector.jdbc.fakedb.driver.FakeDriver2
18 | # Comment intentionally for unregistered driver
19 | # org.apache.flink.connector.jdbc.fakedb.driver.FakeDriver3
20 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-core/src/test/resources/log4j2-test.properties:
--------------------------------------------------------------------------------
1 | ################################################################################
2 | # Licensed to the Apache Software Foundation (ASF) under one
3 | # or more contributor license agreements. See the NOTICE file
4 | # distributed with this work for additional information
5 | # regarding copyright ownership. The ASF licenses this file
6 | # to you under the Apache License, Version 2.0 (the
7 | # "License"); you may not use this file except in compliance
8 | # with the License. You may obtain a copy of the License at
9 | #
10 | # http://www.apache.org/licenses/LICENSE-2.0
11 | #
12 | # Unless required by applicable law or agreed to in writing, software
13 | # distributed under the License is distributed on an "AS IS" BASIS,
14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | # See the License for the specific language governing permissions and
16 | # limitations under the License.
17 | ################################################################################
18 |
19 | # Set root logger level to OFF to not flood build logs
20 | # set manually to INFO for debugging purposes
21 | rootLogger.level = OFF
22 | rootLogger.appenderRef.test.ref = TestLogger
23 |
24 | appender.testlogger.name = TestLogger
25 | appender.testlogger.type = CONSOLE
26 | appender.testlogger.target = SYSTEM_ERR
27 | appender.testlogger.layout.type = PatternLayout
28 | appender.testlogger.layout.pattern = %-4r [%t] %-5p %c %x - %m%n
29 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-cratedb/src/main/java/org/apache/flink/connector/jdbc/cratedb/database/CrateDBFactory.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.connector.jdbc.cratedb.database;
20 |
21 | import org.apache.flink.annotation.Internal;
22 | import org.apache.flink.connector.jdbc.core.database.JdbcFactory;
23 | import org.apache.flink.connector.jdbc.core.database.catalog.JdbcCatalog;
24 | import org.apache.flink.connector.jdbc.core.database.dialect.JdbcDialect;
25 | import org.apache.flink.connector.jdbc.cratedb.database.catalog.CrateDBCatalog;
26 | import org.apache.flink.connector.jdbc.cratedb.database.dialect.CrateDBDialect;
27 |
28 | /** Factory for {@link CrateDBDialect}. */
29 | @Internal
30 | public class CrateDBFactory implements JdbcFactory {
31 | @Override
32 | public boolean acceptsURL(String url) {
33 | return url.startsWith("jdbc:crate:");
34 | }
35 |
36 | @Override
37 | public JdbcDialect createDialect() {
38 | return new CrateDBDialect();
39 | }
40 |
41 | @Override
42 | public JdbcCatalog createCatalog(
43 | ClassLoader classLoader,
44 | String catalogName,
45 | String defaultDatabase,
46 | String username,
47 | String pwd,
48 | String baseUrl) {
49 | return new CrateDBCatalog(
50 | classLoader, catalogName, defaultDatabase, username, pwd, baseUrl);
51 | }
52 | }
53 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-cratedb/src/main/java/org/apache/flink/connector/jdbc/cratedb/database/catalog/CrateDBTypeMapper.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.connector.jdbc.cratedb.database.catalog;
20 |
21 | import org.apache.flink.annotation.Internal;
22 | import org.apache.flink.connector.jdbc.postgres.database.catalog.PostgresTypeMapper;
23 | import org.apache.flink.table.api.DataTypes;
24 | import org.apache.flink.table.types.DataType;
25 |
26 | import org.slf4j.Logger;
27 | import org.slf4j.LoggerFactory;
28 |
29 | /** CrateDBTypeMapper util class. */
30 | @Internal
31 | public class CrateDBTypeMapper extends PostgresTypeMapper {
32 |
33 | private static final Logger LOG = LoggerFactory.getLogger(CrateDBTypeMapper.class);
34 |
35 | // CrateDB jdbc driver uses very similar mapping
36 | // to PostgreSQL driver, and adds some extras:
37 | private static final String PG_STRING = "string";
38 | private static final String PG_STRING_ARRAY = "_string";
39 |
40 | @Override
41 | protected DataType getMapping(String pgType, int precision, int scale) {
42 | switch (pgType) {
43 | case PG_SERIAL:
44 | case PG_BIGSERIAL:
45 | return null;
46 | case PG_STRING:
47 | return DataTypes.STRING();
48 | case PG_STRING_ARRAY:
49 | return DataTypes.ARRAY(DataTypes.STRING());
50 | default:
51 | return super.getMapping(pgType, precision, scale);
52 | }
53 | }
54 |
55 | @Override
56 | protected String getDBType() {
57 | return "CrateDB";
58 | }
59 | }
60 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-cratedb/src/main/java/org/apache/flink/connector/jdbc/cratedb/database/dialect/CrateDBDialect.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.connector.jdbc.cratedb.database.dialect;
20 |
21 | import org.apache.flink.annotation.Internal;
22 | import org.apache.flink.connector.jdbc.postgres.database.dialect.CompatiblePostgresDialect;
23 | import org.apache.flink.table.types.logical.RowType;
24 |
25 | import java.util.Optional;
26 |
27 | /** JDBC dialect for CrateDB. */
28 | @Internal
29 | public class CrateDBDialect extends CompatiblePostgresDialect {
30 |
31 | private static final long serialVersionUID = 1L;
32 |
33 | @Override
34 | public String compatibleDialectName() {
35 | return "CrateDB";
36 | }
37 |
38 | @Override
39 | public CrateDBDialectConverter compatibleRowConverter(RowType rowType) {
40 | return new CrateDBDialectConverter(rowType);
41 | }
42 |
43 | @Override
44 | public Optional compatibleDriverName() {
45 | return Optional.of("io.crate.client.jdbc.CrateDriver");
46 | }
47 | }
48 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-cratedb/src/main/java/org/apache/flink/connector/jdbc/cratedb/database/dialect/CrateDBDialectConverter.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.connector.jdbc.cratedb.database.dialect;
20 |
21 | import org.apache.flink.annotation.Internal;
22 | import org.apache.flink.connector.jdbc.postgres.database.dialect.CompatiblePostgresDialectConverter;
23 | import org.apache.flink.table.types.logical.RowType;
24 |
25 | /**
26 | * Runtime converter that responsible to convert between JDBC object and Flink internal object for
27 | * CrateDB.
28 | */
29 | @Internal
30 | public class CrateDBDialectConverter extends CompatiblePostgresDialectConverter {
31 |
32 | private static final long serialVersionUID = 1L;
33 |
34 | public CrateDBDialectConverter(RowType rowType) {
35 | super(rowType);
36 | }
37 |
38 | @Override
39 | public String compatibleConverterName() {
40 | return "CrateDB";
41 | }
42 | }
43 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-cratedb/src/main/resources/META-INF/services/org.apache.flink.connector.jdbc.core.database.JdbcFactory:
--------------------------------------------------------------------------------
1 | # Licensed to the Apache Software Foundation (ASF) under one or more
2 | # contributor license agreements. See the NOTICE file distributed with
3 | # this work for additional information regarding copyright ownership.
4 | # The ASF licenses this file to You under the Apache License, Version 2.0
5 | # (the "License"); you may not use this file except in compliance with
6 | # the License. You may obtain a copy of the License at
7 | #
8 | # http://www.apache.org/licenses/LICENSE-2.0
9 | #
10 | # Unless required by applicable law or agreed to in writing, software
11 | # distributed under the License is distributed on an "AS IS" BASIS,
12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | # See the License for the specific language governing permissions and
14 | # limitations under the License.
15 |
16 | org.apache.flink.connector.jdbc.cratedb.database.CrateDBFactory
17 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-cratedb/src/test/java/org/apache/flink/connector/jdbc/cratedb/CrateDBTestBase.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.connector.jdbc.cratedb;
20 |
21 | import org.apache.flink.connector.jdbc.cratedb.testutils.CrateDBDatabase;
22 | import org.apache.flink.connector.jdbc.testutils.DatabaseMetadata;
23 | import org.apache.flink.connector.jdbc.testutils.DatabaseTest;
24 |
25 | import org.junit.jupiter.api.extension.ExtendWith;
26 |
27 | /** Base class for CrateDB testing. */
28 | @ExtendWith(CrateDBDatabase.class)
29 | public interface CrateDBTestBase extends DatabaseTest {
30 |
31 | @Override
32 | default DatabaseMetadata getMetadata() {
33 | return CrateDBDatabase.getMetadata();
34 | }
35 | }
36 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-cratedb/src/test/java/org/apache/flink/connector/jdbc/cratedb/testutils/CrateDBImages.java:
--------------------------------------------------------------------------------
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.cratedb.testutils;
19 |
20 | /** Postgres docker images. */
21 | public interface CrateDBImages {
22 | String CRATEDB_5 = "crate:5.7.2";
23 | }
24 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-cratedb/src/test/resources/log4j2-test.properties:
--------------------------------------------------------------------------------
1 | ################################################################################
2 | # Licensed to the Apache Software Foundation (ASF) under one
3 | # or more contributor license agreements. See the NOTICE file
4 | # distributed with this work for additional information
5 | # regarding copyright ownership. The ASF licenses this file
6 | # to you under the Apache License, Version 2.0 (the
7 | # "License"); you may not use this file except in compliance
8 | # with the License. You may obtain a copy of the License at
9 | #
10 | # http://www.apache.org/licenses/LICENSE-2.0
11 | #
12 | # Unless required by applicable law or agreed to in writing, software
13 | # distributed under the License is distributed on an "AS IS" BASIS,
14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | # See the License for the specific language governing permissions and
16 | # limitations under the License.
17 | ################################################################################
18 |
19 | # Set root logger level to OFF to not flood build logs
20 | # set manually to INFO for debugging purposes
21 | rootLogger.level = OFF
22 | rootLogger.appenderRef.test.ref = TestLogger
23 |
24 | appender.testlogger.name = TestLogger
25 | appender.testlogger.type = CONSOLE
26 | appender.testlogger.target = SYSTEM_ERR
27 | appender.testlogger.layout.type = PatternLayout
28 | appender.testlogger.layout.pattern = %-4r [%t] %-5p %c %x - %m%n
29 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-db2/src/main/java/org/apache/flink/connector/jdbc/db2/database/Db2Factory.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.connector.jdbc.db2.database;
20 |
21 | import org.apache.flink.annotation.Internal;
22 | import org.apache.flink.connector.jdbc.core.database.JdbcFactory;
23 | import org.apache.flink.connector.jdbc.core.database.catalog.JdbcCatalog;
24 | import org.apache.flink.connector.jdbc.core.database.dialect.JdbcDialect;
25 | import org.apache.flink.connector.jdbc.db2.database.dialect.Db2Dialect;
26 |
27 | /** Factory for {@link Db2Dialect}. */
28 | @Internal
29 | public class Db2Factory implements JdbcFactory {
30 | @Override
31 | public boolean acceptsURL(String url) {
32 | return url.startsWith("jdbc:db2:");
33 | }
34 |
35 | @Override
36 | public JdbcDialect createDialect() {
37 | return new Db2Dialect();
38 | }
39 |
40 | @Override
41 | public JdbcCatalog createCatalog(
42 | ClassLoader classLoader,
43 | String catalogName,
44 | String defaultDatabase,
45 | String username,
46 | String pwd,
47 | String baseUrl) {
48 | throw new UnsupportedOperationException("Catalog for DB2 is not supported yet.");
49 | }
50 | }
51 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-db2/src/main/java/org/apache/flink/connector/jdbc/db2/database/dialect/Db2DialectConverter.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.connector.jdbc.db2.database.dialect;
20 |
21 | import org.apache.flink.annotation.Internal;
22 | import org.apache.flink.connector.jdbc.core.database.dialect.AbstractDialectConverter;
23 | import org.apache.flink.table.types.logical.LogicalType;
24 | import org.apache.flink.table.types.logical.RowType;
25 |
26 | /**
27 | * Runtime converter that responsible to convert between JDBC object and Flink internal object for
28 | * Db2.
29 | */
30 | @Internal
31 | public class Db2DialectConverter extends AbstractDialectConverter {
32 |
33 | private static final long serialVersionUID = 1L;
34 |
35 | @Override
36 | public String converterName() {
37 | return "Db2";
38 | }
39 |
40 | public Db2DialectConverter(RowType rowType) {
41 | super(rowType);
42 | }
43 |
44 | @Override
45 | protected JdbcDeserializationConverter createInternalConverter(LogicalType type) {
46 | return super.createInternalConverter(type);
47 | }
48 | }
49 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-db2/src/main/resources/META-INF/services/org.apache.flink.connector.jdbc.core.database.JdbcFactory:
--------------------------------------------------------------------------------
1 | # Licensed to the Apache Software Foundation (ASF) under one or more
2 | # contributor license agreements. See the NOTICE file distributed with
3 | # this work for additional information regarding copyright ownership.
4 | # The ASF licenses this file to You under the Apache License, Version 2.0
5 | # (the "License"); you may not use this file except in compliance with
6 | # the License. You may obtain a copy of the License at
7 | #
8 | # http://www.apache.org/licenses/LICENSE-2.0
9 | #
10 | # Unless required by applicable law or agreed to in writing, software
11 | # distributed under the License is distributed on an "AS IS" BASIS,
12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | # See the License for the specific language governing permissions and
14 | # limitations under the License.
15 |
16 | org.apache.flink.connector.jdbc.db2.database.Db2Factory
17 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-db2/src/test/java/org/apache/flink/connector/jdbc/db2/Db2TestBase.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.connector.jdbc.db2;
20 |
21 | import org.apache.flink.connector.jdbc.db2.testutils.Db2Database;
22 | import org.apache.flink.connector.jdbc.testutils.DatabaseMetadata;
23 | import org.apache.flink.connector.jdbc.testutils.DatabaseTest;
24 |
25 | import org.junit.jupiter.api.extension.ExtendWith;
26 |
27 | /** Base class for Db2 testing. */
28 | @ExtendWith(Db2Database.class)
29 | public interface Db2TestBase extends DatabaseTest {
30 |
31 | @Override
32 | default DatabaseMetadata getMetadata() {
33 | return Db2Database.getMetadata();
34 | }
35 | }
36 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-db2/src/test/java/org/apache/flink/connector/jdbc/db2/database/dialect/Db2DialectTest.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.connector.jdbc.db2.database.dialect;
20 |
21 | import org.apache.flink.connector.jdbc.core.database.dialect.JdbcDialectTest;
22 | import org.apache.flink.connector.jdbc.db2.Db2TestBase;
23 |
24 | import java.util.Arrays;
25 | import java.util.List;
26 |
27 | class Db2DialectTest extends JdbcDialectTest implements Db2TestBase {
28 |
29 | @Override
30 | protected List testData() {
31 | return Arrays.asList(
32 | createTestItem("CHAR"),
33 | createTestItem("VARCHAR"),
34 | createTestItem("BOOLEAN"),
35 | createTestItem("TINYINT"),
36 | createTestItem("SMALLINT"),
37 | createTestItem("INTEGER"),
38 | createTestItem("BIGINT"),
39 | createTestItem("FLOAT"),
40 | createTestItem("DOUBLE"),
41 | createTestItem("DECIMAL(10, 4)"),
42 | createTestItem("DECIMAL(31, 18)"),
43 | createTestItem("DATE"),
44 | createTestItem("TIME"),
45 | createTestItem("TIMESTAMP(3)"),
46 | createTestItem("TIMESTAMP WITHOUT TIME ZONE"),
47 |
48 | // Not valid data
49 | createTestItem("BINARY", "The Db2 dialect doesn't support type: BINARY(1)."),
50 | createTestItem(
51 | "VARBINARY(10)", "The Db2 dialect doesn't support type: VARBINARY(10)."));
52 | }
53 | }
54 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-db2/src/test/java/org/apache/flink/connector/jdbc/db2/testutils/Db2Images.java:
--------------------------------------------------------------------------------
1 | package org.apache.flink.connector.jdbc.db2.testutils;
2 |
3 | /** DB2 docker images. */
4 | public interface Db2Images {
5 | String DB2_11 = "icr.io/db2_community/db2:11.5.8.0";
6 | }
7 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-db2/src/test/resources/log4j2-test.properties:
--------------------------------------------------------------------------------
1 | ################################################################################
2 | # Licensed to the Apache Software Foundation (ASF) under one
3 | # or more contributor license agreements. See the NOTICE file
4 | # distributed with this work for additional information
5 | # regarding copyright ownership. The ASF licenses this file
6 | # to you under the Apache License, Version 2.0 (the
7 | # "License"); you may not use this file except in compliance
8 | # with the License. You may obtain a copy of the License at
9 | #
10 | # http://www.apache.org/licenses/LICENSE-2.0
11 | #
12 | # Unless required by applicable law or agreed to in writing, software
13 | # distributed under the License is distributed on an "AS IS" BASIS,
14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | # See the License for the specific language governing permissions and
16 | # limitations under the License.
17 | ################################################################################
18 |
19 | # Set root logger level to OFF to not flood build logs
20 | # set manually to INFO for debugging purposes
21 | rootLogger.level = OFF
22 | rootLogger.appenderRef.test.ref = TestLogger
23 |
24 | appender.testlogger.name = TestLogger
25 | appender.testlogger.type = CONSOLE
26 | appender.testlogger.target = SYSTEM_ERR
27 | appender.testlogger.layout.type = PatternLayout
28 | appender.testlogger.layout.pattern = %-4r [%t] %-5p %c %x - %m%n
29 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-mysql/src/main/java/org/apache/flink/connector/jdbc/mysql/database/MySqlFactory.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.connector.jdbc.mysql.database;
20 |
21 | import org.apache.flink.annotation.Internal;
22 | import org.apache.flink.connector.jdbc.core.database.JdbcFactory;
23 | import org.apache.flink.connector.jdbc.core.database.catalog.JdbcCatalog;
24 | import org.apache.flink.connector.jdbc.core.database.dialect.JdbcDialect;
25 | import org.apache.flink.connector.jdbc.mysql.database.catalog.MySqlCatalog;
26 | import org.apache.flink.connector.jdbc.mysql.database.dialect.MySqlDialect;
27 |
28 | /** Factory for {@link MySqlDialect}. */
29 | @Internal
30 | public class MySqlFactory implements JdbcFactory {
31 | @Override
32 | public boolean acceptsURL(String url) {
33 | return url.startsWith("jdbc:mysql:");
34 | }
35 |
36 | @Override
37 | public JdbcDialect createDialect() {
38 | return new MySqlDialect();
39 | }
40 |
41 | @Override
42 | public JdbcCatalog createCatalog(
43 | ClassLoader classLoader,
44 | String catalogName,
45 | String defaultDatabase,
46 | String username,
47 | String pwd,
48 | String baseUrl) {
49 | return new MySqlCatalog(classLoader, catalogName, defaultDatabase, username, pwd, baseUrl);
50 | }
51 | }
52 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-mysql/src/main/java/org/apache/flink/connector/jdbc/mysql/database/dialect/MySQLDialectConverter.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.connector.jdbc.mysql.database.dialect;
20 |
21 | import org.apache.flink.annotation.Internal;
22 | import org.apache.flink.connector.jdbc.core.database.dialect.AbstractDialectConverter;
23 | import org.apache.flink.table.types.logical.RowType;
24 |
25 | /**
26 | * Runtime converter that responsible to convert between JDBC object and Flink internal object for
27 | * MySQL.
28 | */
29 | @Internal
30 | public class MySQLDialectConverter extends AbstractDialectConverter {
31 |
32 | private static final long serialVersionUID = 1L;
33 |
34 | @Override
35 | public String converterName() {
36 | return "MySQL";
37 | }
38 |
39 | public MySQLDialectConverter(RowType rowType) {
40 | super(rowType);
41 | }
42 | }
43 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-mysql/src/main/resources/META-INF/services/org.apache.flink.connector.jdbc.core.database.JdbcFactory:
--------------------------------------------------------------------------------
1 | # Licensed to the Apache Software Foundation (ASF) under one or more
2 | # contributor license agreements. See the NOTICE file distributed with
3 | # this work for additional information regarding copyright ownership.
4 | # The ASF licenses this file to You under the Apache License, Version 2.0
5 | # (the "License"); you may not use this file except in compliance with
6 | # the License. You may obtain a copy of the License at
7 | #
8 | # http://www.apache.org/licenses/LICENSE-2.0
9 | #
10 | # Unless required by applicable law or agreed to in writing, software
11 | # distributed under the License is distributed on an "AS IS" BASIS,
12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | # See the License for the specific language governing permissions and
14 | # limitations under the License.
15 |
16 | org.apache.flink.connector.jdbc.mysql.database.MySqlFactory
--------------------------------------------------------------------------------
/flink-connector-jdbc-mysql/src/test/java/org/apache/flink/connector/jdbc/mysql/MySqlTestBase.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.connector.jdbc.mysql;
20 |
21 | import org.apache.flink.connector.jdbc.mysql.testutils.MySqlDatabase;
22 | import org.apache.flink.connector.jdbc.testutils.DatabaseMetadata;
23 | import org.apache.flink.connector.jdbc.testutils.DatabaseTest;
24 |
25 | import org.junit.jupiter.api.extension.ExtendWith;
26 |
27 | /** Base class for MySql testing. */
28 | @ExtendWith(MySqlDatabase.class)
29 | public interface MySqlTestBase extends DatabaseTest {
30 |
31 | @Override
32 | default DatabaseMetadata getMetadata() {
33 | return MySqlDatabase.getMetadata();
34 | }
35 | }
36 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-mysql/src/test/java/org/apache/flink/connector/jdbc/mysql/database/catalog/MySqlCatalogITCase.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.connector.jdbc.mysql.database.catalog;
20 |
21 | import org.apache.flink.connector.jdbc.mysql.MySqlTestBase;
22 |
23 | /** E2E test for {@link MySqlCatalog}. */
24 | class MySqlCatalogITCase extends MySqlCatalogTestBase implements MySqlTestBase {}
25 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-mysql/src/test/java/org/apache/flink/connector/jdbc/mysql/testutils/MySqlDatabase.java:
--------------------------------------------------------------------------------
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.mysql.testutils;
19 |
20 | import org.apache.flink.connector.jdbc.testutils.DatabaseExtension;
21 | import org.apache.flink.connector.jdbc.testutils.DatabaseMetadata;
22 | import org.apache.flink.connector.jdbc.testutils.DatabaseResource;
23 | import org.apache.flink.connector.jdbc.testutils.resources.DockerResource;
24 | import org.apache.flink.util.FlinkRuntimeException;
25 |
26 | import org.testcontainers.containers.MySQLContainer;
27 |
28 | /** A MySql database for testing. */
29 | public class MySqlDatabase extends DatabaseExtension implements MySqlImages {
30 |
31 | private static final MySQLContainer> CONTAINER =
32 | new MySqlContainer(MYSQL_8)
33 | .withXa()
34 | .withLockWaitTimeout(50_000L)
35 | .withCommand("--character-set-server=utf8")
36 | .withEnv("MYSQL_ROOT_HOST", "%");
37 |
38 | private static MySqlMetadata metadata;
39 |
40 | public static MySqlMetadata getMetadata() {
41 | if (!CONTAINER.isRunning()) {
42 | throw new FlinkRuntimeException("Container is stopped.");
43 | }
44 | if (metadata == null) {
45 | metadata = new MySqlMetadata(CONTAINER, true);
46 | }
47 | return metadata;
48 | }
49 |
50 | @Override
51 | protected DatabaseMetadata getMetadataDB() {
52 | return getMetadata();
53 | }
54 |
55 | @Override
56 | protected DatabaseResource getResource() {
57 | return new DockerResource(CONTAINER);
58 | }
59 | }
60 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-mysql/src/test/java/org/apache/flink/connector/jdbc/mysql/testutils/MySqlImages.java:
--------------------------------------------------------------------------------
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.mysql.testutils;
19 |
20 | /** MySql docker images. */
21 | public interface MySqlImages {
22 | String MYSQL_8 = "mysql:9.0.0";
23 | }
24 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-mysql/src/test/resources/log4j2-test.properties:
--------------------------------------------------------------------------------
1 | ################################################################################
2 | # Licensed to the Apache Software Foundation (ASF) under one
3 | # or more contributor license agreements. See the NOTICE file
4 | # distributed with this work for additional information
5 | # regarding copyright ownership. The ASF licenses this file
6 | # to you under the Apache License, Version 2.0 (the
7 | # "License"); you may not use this file except in compliance
8 | # with the License. You may obtain a copy of the License at
9 | #
10 | # http://www.apache.org/licenses/LICENSE-2.0
11 | #
12 | # Unless required by applicable law or agreed to in writing, software
13 | # distributed under the License is distributed on an "AS IS" BASIS,
14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | # See the License for the specific language governing permissions and
16 | # limitations under the License.
17 | ################################################################################
18 |
19 | # Set root logger level to OFF to not flood build logs
20 | # set manually to INFO for debugging purposes
21 | rootLogger.level = OFF
22 | rootLogger.appenderRef.test.ref = TestLogger
23 |
24 | appender.testlogger.name = TestLogger
25 | appender.testlogger.type = CONSOLE
26 | appender.testlogger.target = SYSTEM_ERR
27 | appender.testlogger.layout.type = PatternLayout
28 | appender.testlogger.layout.pattern = %-4r [%t] %-5p %c %x - %m%n
29 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-oceanbase/src/main/java/org/apache/flink/connector/jdbc/oceanbase/database/dialect/OceanBaseCompatibleMode.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.connector.jdbc.oceanbase.database.dialect;
20 |
21 | /** Compatible mode of OceanBase. */
22 | public enum OceanBaseCompatibleMode {
23 | MySQL,
24 | Oracle;
25 |
26 | public static OceanBaseCompatibleMode parse(String text) {
27 | if (text == null || text.trim().isEmpty()) {
28 | return OceanBaseCompatibleMode.MySQL;
29 | }
30 | switch (text.trim().toLowerCase()) {
31 | case "mysql":
32 | return OceanBaseCompatibleMode.MySQL;
33 | case "oracle":
34 | return OceanBaseCompatibleMode.Oracle;
35 | default:
36 | throw new IllegalArgumentException("Unsupported compatible mode: " + text);
37 | }
38 | }
39 |
40 | public boolean isMySQLMode() {
41 | return this == MySQL;
42 | }
43 |
44 | @Override
45 | public String toString() {
46 | return this.name().toLowerCase();
47 | }
48 | }
49 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-oceanbase/src/main/resources/META-INF/services/org.apache.flink.connector.jdbc.core.database.JdbcFactory:
--------------------------------------------------------------------------------
1 | # Licensed to the Apache Software Foundation (ASF) under one or more
2 | # contributor license agreements. See the NOTICE file distributed with
3 | # this work for additional information regarding copyright ownership.
4 | # The ASF licenses this file to You under the Apache License, Version 2.0
5 | # (the "License"); you may not use this file except in compliance with
6 | # the License. You may obtain a copy of the License at
7 | #
8 | # http://www.apache.org/licenses/LICENSE-2.0
9 | #
10 | # Unless required by applicable law or agreed to in writing, software
11 | # distributed under the License is distributed on an "AS IS" BASIS,
12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | # See the License for the specific language governing permissions and
14 | # limitations under the License.
15 |
16 | org.apache.flink.connector.jdbc.oceanbase.database.OceanBaseFactory
17 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-oceanbase/src/test/java/org/apache/flink/connector/jdbc/oceanbase/OceanBaseMysqlTestBase.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.connector.jdbc.oceanbase;
20 |
21 | import org.apache.flink.connector.jdbc.oceanbase.database.dialect.OceanBaseCompatibleMode;
22 | import org.apache.flink.connector.jdbc.oceanbase.table.OceanBaseTableRow;
23 | import org.apache.flink.connector.jdbc.oceanbase.testutils.OceanBaseDatabase;
24 | import org.apache.flink.connector.jdbc.testutils.DatabaseMetadata;
25 | import org.apache.flink.connector.jdbc.testutils.DatabaseTest;
26 | import org.apache.flink.connector.jdbc.testutils.tables.TableField;
27 | import org.apache.flink.connector.jdbc.testutils.tables.TableRow;
28 |
29 | import org.junit.jupiter.api.extension.ExtendWith;
30 |
31 | /** Base class for OceanBase Mysql mode testing. */
32 | @ExtendWith(OceanBaseDatabase.class)
33 | public interface OceanBaseMysqlTestBase extends DatabaseTest {
34 |
35 | static TableRow tableRow(String name, TableField... fields) {
36 | return new OceanBaseTableRow(OceanBaseCompatibleMode.MySQL, name, fields);
37 | }
38 |
39 | @Override
40 | default DatabaseMetadata getMetadata() {
41 | return OceanBaseDatabase.getMetadata();
42 | }
43 | }
44 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-oceanbase/src/test/java/org/apache/flink/connector/jdbc/oceanbase/OceanBaseOracleTestBase.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.connector.jdbc.oceanbase;
20 |
21 | import org.apache.flink.connector.jdbc.oceanbase.database.dialect.OceanBaseCompatibleMode;
22 | import org.apache.flink.connector.jdbc.oceanbase.table.OceanBaseTableRow;
23 | import org.apache.flink.connector.jdbc.oceanbase.testutils.OceanBaseMetadata;
24 | import org.apache.flink.connector.jdbc.testutils.DatabaseMetadata;
25 | import org.apache.flink.connector.jdbc.testutils.DatabaseTest;
26 | import org.apache.flink.connector.jdbc.testutils.tables.TableField;
27 | import org.apache.flink.connector.jdbc.testutils.tables.TableRow;
28 |
29 | /** Base class for OceanBase Oracle mode testing. */
30 | public interface OceanBaseOracleTestBase extends DatabaseTest {
31 |
32 | static TableRow tableRow(String name, TableField... fields) {
33 | return new OceanBaseTableRow(OceanBaseCompatibleMode.Oracle, name, fields);
34 | }
35 |
36 | @Override
37 | default DatabaseMetadata getMetadata() {
38 | return new OceanBaseMetadata(
39 | System.getenv("test.oceanbase.username"),
40 | System.getenv("test.oceanbase.password"),
41 | System.getenv("test.oceanbase.url"),
42 | "com.oceanbase.jdbc.Driver",
43 | "test");
44 | }
45 | }
46 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-oceanbase/src/test/java/org/apache/flink/connector/jdbc/oceanbase/database/dialect/OceanBaseDialectTest.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.connector.jdbc.oceanbase.database.dialect;
20 |
21 | import org.junit.jupiter.api.Test;
22 |
23 | import static org.assertj.core.api.Assertions.assertThat;
24 |
25 | /** Tests for {@link OceanBaseDialect}. */
26 | class OceanBaseDialectTest {
27 |
28 | @Test
29 | void testMysqlAppendDefaultUrlProperties() {
30 | OceanBaseDialect dialect = new OceanBaseDialect(OceanBaseCompatibleMode.MySQL);
31 | String jdbcUrl = "jdbc:oceanbase://localhost:2883/foo";
32 |
33 | assertThat(dialect.appendDefaultUrlProperties(jdbcUrl))
34 | .isEqualTo(jdbcUrl + "?rewriteBatchedStatements=true");
35 |
36 | assertThat(dialect.appendDefaultUrlProperties(jdbcUrl + "?foo=bar"))
37 | .isEqualTo(jdbcUrl + "?foo=bar&rewriteBatchedStatements=true");
38 |
39 | assertThat(
40 | dialect.appendDefaultUrlProperties(
41 | jdbcUrl + "?foo=bar&rewriteBatchedStatements=false"))
42 | .isEqualTo(jdbcUrl + "?foo=bar&rewriteBatchedStatements=false");
43 | }
44 |
45 | @Test
46 | void testOracleAppendDefaultUrlProperties() {
47 | OceanBaseDialect dialect = new OceanBaseDialect(OceanBaseCompatibleMode.Oracle);
48 | String jdbcUrl = "jdbc:oceanbase://localhost:2883/foo";
49 |
50 | assertThat(dialect.appendDefaultUrlProperties(jdbcUrl)).isEqualTo(jdbcUrl);
51 | }
52 | }
53 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-oceanbase/src/test/java/org/apache/flink/connector/jdbc/oceanbase/table/OceanBaseTableRow.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.connector.jdbc.oceanbase.table;
20 |
21 | import org.apache.flink.connector.jdbc.oceanbase.database.dialect.OceanBaseCompatibleMode;
22 | import org.apache.flink.connector.jdbc.testutils.DatabaseMetadata;
23 | import org.apache.flink.connector.jdbc.testutils.tables.TableField;
24 | import org.apache.flink.connector.jdbc.testutils.tables.TableRow;
25 |
26 | import java.util.ArrayList;
27 | import java.util.List;
28 |
29 | /** TableRow for OceanBase. */
30 | public class OceanBaseTableRow extends TableRow {
31 |
32 | private final OceanBaseCompatibleMode compatibleMode;
33 |
34 | public OceanBaseTableRow(
35 | OceanBaseCompatibleMode compatibleMode, String name, TableField[] fields) {
36 | super(name, fields);
37 | this.compatibleMode = compatibleMode;
38 | }
39 |
40 | @Override
41 | public String getCreateQueryForFlink(
42 | DatabaseMetadata metadata,
43 | String newName,
44 | List newFields,
45 | List withParams) {
46 | List params = new ArrayList<>(withParams);
47 | params.add("'compatible-mode'='" + compatibleMode + "'");
48 | return super.getCreateQueryForFlink(metadata, newName, newFields, params);
49 | }
50 | }
51 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-oceanbase/src/test/java/org/apache/flink/connector/jdbc/oceanbase/testutils/OceanBaseImages.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.connector.jdbc.oceanbase.testutils;
20 |
21 | /** OceanBase docker images. */
22 | public interface OceanBaseImages {
23 | String OCEANBASE_CE_4 = "oceanbase/oceanbase-ce:4.2.1-lts";
24 | }
25 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-oceanbase/src/test/resources/log4j2-test.properties:
--------------------------------------------------------------------------------
1 | ################################################################################
2 | # Licensed to the Apache Software Foundation (ASF) under one
3 | # or more contributor license agreements. See the NOTICE file
4 | # distributed with this work for additional information
5 | # regarding copyright ownership. The ASF licenses this file
6 | # to you under the Apache License, Version 2.0 (the
7 | # "License"); you may not use this file except in compliance
8 | # with the License. You may obtain a copy of the License at
9 | #
10 | # http://www.apache.org/licenses/LICENSE-2.0
11 | #
12 | # Unless required by applicable law or agreed to in writing, software
13 | # distributed under the License is distributed on an "AS IS" BASIS,
14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | # See the License for the specific language governing permissions and
16 | # limitations under the License.
17 | ################################################################################
18 |
19 | # Set root logger level to OFF to not flood build logs
20 | # set manually to INFO for debugging purposes
21 | rootLogger.level = OFF
22 | rootLogger.appenderRef.test.ref = TestLogger
23 |
24 | appender.testlogger.name = TestLogger
25 | appender.testlogger.type = CONSOLE
26 | appender.testlogger.target = SYSTEM_ERR
27 | appender.testlogger.layout.type = PatternLayout
28 | appender.testlogger.layout.pattern = %-4r [%t] %-5p %c %x - %m%n
29 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-oracle/src/main/java/org/apache/flink/connector/jdbc/oracle/database/OracleFactory.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.connector.jdbc.oracle.database;
20 |
21 | import org.apache.flink.annotation.Internal;
22 | import org.apache.flink.connector.jdbc.core.database.JdbcFactory;
23 | import org.apache.flink.connector.jdbc.core.database.catalog.JdbcCatalog;
24 | import org.apache.flink.connector.jdbc.core.database.dialect.JdbcDialect;
25 | import org.apache.flink.connector.jdbc.oracle.database.dialect.OracleDialect;
26 |
27 | /** Factory for {@link OracleDialect}. */
28 | @Internal
29 | public class OracleFactory implements JdbcFactory {
30 | @Override
31 | public boolean acceptsURL(String url) {
32 | return url.startsWith("jdbc:oracle:");
33 | }
34 |
35 | @Override
36 | public JdbcDialect createDialect() {
37 | return new OracleDialect();
38 | }
39 |
40 | @Override
41 | public JdbcCatalog createCatalog(
42 | ClassLoader classLoader,
43 | String catalogName,
44 | String defaultDatabase,
45 | String username,
46 | String pwd,
47 | String baseUrl) {
48 | throw new UnsupportedOperationException("Catalog for Oracle is not supported yet.");
49 | }
50 | }
51 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-oracle/src/main/resources/META-INF/services/org.apache.flink.connector.jdbc.core.database.JdbcFactory:
--------------------------------------------------------------------------------
1 | # Licensed to the Apache Software Foundation (ASF) under one or more
2 | # contributor license agreements. See the NOTICE file distributed with
3 | # this work for additional information regarding copyright ownership.
4 | # The ASF licenses this file to You under the Apache License, Version 2.0
5 | # (the "License"); you may not use this file except in compliance with
6 | # the License. You may obtain a copy of the License at
7 | #
8 | # http://www.apache.org/licenses/LICENSE-2.0
9 | #
10 | # Unless required by applicable law or agreed to in writing, software
11 | # distributed under the License is distributed on an "AS IS" BASIS,
12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | # See the License for the specific language governing permissions and
14 | # limitations under the License.
15 |
16 | org.apache.flink.connector.jdbc.oracle.database.OracleFactory
17 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-oracle/src/test/java/org/apache/flink/connector/jdbc/oracle/OracleTestBase.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.connector.jdbc.oracle;
20 |
21 | import org.apache.flink.connector.jdbc.oracle.testutils.OracleDatabase;
22 | import org.apache.flink.connector.jdbc.testutils.DatabaseMetadata;
23 | import org.apache.flink.connector.jdbc.testutils.DatabaseTest;
24 |
25 | import org.junit.jupiter.api.extension.ExtendWith;
26 |
27 | /** Base class for Oracle testing. */
28 | @ExtendWith(OracleDatabase.class)
29 | public interface OracleTestBase extends DatabaseTest {
30 |
31 | @Override
32 | default DatabaseMetadata getMetadata() {
33 | return OracleDatabase.getMetadata();
34 | }
35 | }
36 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-oracle/src/test/java/org/apache/flink/connector/jdbc/oracle/testutils/OracleDatabase.java:
--------------------------------------------------------------------------------
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.oracle.testutils;
19 |
20 | import org.apache.flink.connector.jdbc.testutils.DatabaseExtension;
21 | import org.apache.flink.connector.jdbc.testutils.DatabaseMetadata;
22 | import org.apache.flink.connector.jdbc.testutils.DatabaseResource;
23 | import org.apache.flink.connector.jdbc.testutils.resources.DockerResource;
24 | import org.apache.flink.util.FlinkRuntimeException;
25 |
26 | import org.testcontainers.containers.OracleContainer;
27 |
28 | /** A Oracle database for testing. */
29 | public class OracleDatabase extends DatabaseExtension implements OracleImages {
30 |
31 | private static final OracleContainer CONTAINER =
32 | new OracleContainer(ORACLE_21)
33 | .withStartupTimeoutSeconds(240)
34 | .withConnectTimeoutSeconds(120)
35 | .usingSid();
36 |
37 | private static OracleMetadata metadata;
38 |
39 | public static OracleMetadata getMetadata() {
40 | if (!CONTAINER.isRunning()) {
41 | throw new FlinkRuntimeException("Container is stopped.");
42 | }
43 | if (metadata == null) {
44 | metadata = new OracleMetadata(CONTAINER, true);
45 | }
46 | return metadata;
47 | }
48 |
49 | @Override
50 | protected DatabaseMetadata getMetadataDB() {
51 | return getMetadata();
52 | }
53 |
54 | @Override
55 | protected DatabaseResource getResource() {
56 | return new DockerResource(CONTAINER);
57 | }
58 | }
59 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-oracle/src/test/java/org/apache/flink/connector/jdbc/oracle/testutils/OracleImages.java:
--------------------------------------------------------------------------------
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.oracle.testutils;
19 |
20 | /** Oracle docker images. */
21 | public interface OracleImages {
22 | String ORACLE_21 = "gvenzl/oracle-xe:21.3.0-slim-faststart";
23 | }
24 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-oracle/src/test/resources/log4j2-test.properties:
--------------------------------------------------------------------------------
1 | ################################################################################
2 | # Licensed to the Apache Software Foundation (ASF) under one
3 | # or more contributor license agreements. See the NOTICE file
4 | # distributed with this work for additional information
5 | # regarding copyright ownership. The ASF licenses this file
6 | # to you under the Apache License, Version 2.0 (the
7 | # "License"); you may not use this file except in compliance
8 | # with the License. You may obtain a copy of the License at
9 | #
10 | # http://www.apache.org/licenses/LICENSE-2.0
11 | #
12 | # Unless required by applicable law or agreed to in writing, software
13 | # distributed under the License is distributed on an "AS IS" BASIS,
14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | # See the License for the specific language governing permissions and
16 | # limitations under the License.
17 | ################################################################################
18 |
19 | # Set root logger level to OFF to not flood build logs
20 | # set manually to INFO for debugging purposes
21 | rootLogger.level = OFF
22 | rootLogger.appenderRef.test.ref = TestLogger
23 |
24 | appender.testlogger.name = TestLogger
25 | appender.testlogger.type = CONSOLE
26 | appender.testlogger.target = SYSTEM_ERR
27 | appender.testlogger.layout.type = PatternLayout
28 | appender.testlogger.layout.pattern = %-4r [%t] %-5p %c %x - %m%n
29 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-postgres/src/main/java/org/apache/flink/connector/jdbc/postgres/database/PostgresFactory.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.connector.jdbc.postgres.database;
20 |
21 | import org.apache.flink.annotation.Internal;
22 | import org.apache.flink.connector.jdbc.core.database.JdbcFactory;
23 | import org.apache.flink.connector.jdbc.core.database.catalog.JdbcCatalog;
24 | import org.apache.flink.connector.jdbc.core.database.dialect.JdbcDialect;
25 | import org.apache.flink.connector.jdbc.postgres.database.catalog.PostgresCatalog;
26 | import org.apache.flink.connector.jdbc.postgres.database.dialect.PostgresDialect;
27 |
28 | /** Factory for {@link PostgresDialect}. */
29 | @Internal
30 | public class PostgresFactory implements JdbcFactory {
31 | @Override
32 | public boolean acceptsURL(String url) {
33 | return url.startsWith("jdbc:postgresql:");
34 | }
35 |
36 | @Override
37 | public JdbcDialect createDialect() {
38 | return new PostgresDialect();
39 | }
40 |
41 | @Override
42 | public JdbcCatalog createCatalog(
43 | ClassLoader classLoader,
44 | String catalogName,
45 | String defaultDatabase,
46 | String username,
47 | String pwd,
48 | String baseUrl) {
49 | return new PostgresCatalog(
50 | classLoader, catalogName, defaultDatabase, username, pwd, baseUrl);
51 | }
52 | }
53 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-postgres/src/main/java/org/apache/flink/connector/jdbc/postgres/database/dialect/CompatiblePostgresDialect.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.connector.jdbc.postgres.database.dialect;
20 |
21 | import org.apache.flink.annotation.PublicEvolving;
22 | import org.apache.flink.table.types.logical.RowType;
23 |
24 | import java.util.Optional;
25 |
26 | /** JDBC dialect for PostgreSQL compatible databases. */
27 | @PublicEvolving
28 | public abstract class CompatiblePostgresDialect extends PostgresDialect {
29 |
30 | private static final long serialVersionUID = 1L;
31 |
32 | protected abstract String compatibleDialectName();
33 |
34 | protected abstract CompatiblePostgresDialectConverter compatibleRowConverter(RowType rowType);
35 |
36 | protected abstract Optional compatibleDriverName();
37 |
38 | @Override
39 | public String dialectName() {
40 | return compatibleDialectName();
41 | }
42 |
43 | @Override
44 | public CompatiblePostgresDialectConverter getRowConverter(RowType rowType) {
45 | return compatibleRowConverter(rowType);
46 | }
47 |
48 | @Override
49 | public Optional defaultDriverName() {
50 | return compatibleDriverName();
51 | }
52 | }
53 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-postgres/src/main/java/org/apache/flink/connector/jdbc/postgres/database/dialect/CompatiblePostgresDialectConverter.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.connector.jdbc.postgres.database.dialect;
20 |
21 | import org.apache.flink.annotation.Internal;
22 | import org.apache.flink.table.types.logical.RowType;
23 |
24 | /** JDBC converter for PostgreSQL compatible databases. */
25 | @Internal
26 | public abstract class CompatiblePostgresDialectConverter extends PostgresDialectConverter {
27 |
28 | private static final long serialVersionUID = 1L;
29 |
30 | protected CompatiblePostgresDialectConverter(RowType rowType) {
31 | super(rowType);
32 | }
33 |
34 | protected abstract String compatibleConverterName();
35 |
36 | @Override
37 | public String converterName() {
38 | return compatibleConverterName();
39 | }
40 | }
41 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-postgres/src/main/resources/META-INF/services/org.apache.flink.connector.jdbc.core.database.JdbcFactory:
--------------------------------------------------------------------------------
1 | # Licensed to the Apache Software Foundation (ASF) under one or more
2 | # contributor license agreements. See the NOTICE file distributed with
3 | # this work for additional information regarding copyright ownership.
4 | # The ASF licenses this file to You under the Apache License, Version 2.0
5 | # (the "License"); you may not use this file except in compliance with
6 | # the License. You may obtain a copy of the License at
7 | #
8 | # http://www.apache.org/licenses/LICENSE-2.0
9 | #
10 | # Unless required by applicable law or agreed to in writing, software
11 | # distributed under the License is distributed on an "AS IS" BASIS,
12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | # See the License for the specific language governing permissions and
14 | # limitations under the License.
15 |
16 | org.apache.flink.connector.jdbc.postgres.database.PostgresFactory
17 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-postgres/src/test/java/org/apache/flink/connector/jdbc/postgres/PostgresTestBase.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.connector.jdbc.postgres;
20 |
21 | import org.apache.flink.connector.jdbc.postgres.testutils.PostgresDatabase;
22 | import org.apache.flink.connector.jdbc.testutils.DatabaseMetadata;
23 | import org.apache.flink.connector.jdbc.testutils.DatabaseTest;
24 |
25 | import org.junit.jupiter.api.extension.ExtendWith;
26 |
27 | /** Base class for Postgres testing. */
28 | @ExtendWith(PostgresDatabase.class)
29 | public interface PostgresTestBase extends DatabaseTest {
30 |
31 | @Override
32 | default DatabaseMetadata getMetadata() {
33 | return PostgresDatabase.getMetadata();
34 | }
35 | }
36 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-postgres/src/test/java/org/apache/flink/connector/jdbc/postgres/table/PostgresDynamicTableSinkITCase.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.connector.jdbc.postgres.table;
20 |
21 | import org.apache.flink.connector.jdbc.core.table.sink.JdbcDynamicTableSinkITCase;
22 | import org.apache.flink.connector.jdbc.postgres.PostgresTestBase;
23 | import org.apache.flink.connector.jdbc.postgres.database.dialect.PostgresDialect;
24 |
25 | /** The Table Sink ITCase for {@link PostgresDialect}. */
26 | class PostgresDynamicTableSinkITCase extends JdbcDynamicTableSinkITCase
27 | implements PostgresTestBase {}
28 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-postgres/src/test/java/org/apache/flink/connector/jdbc/postgres/testutils/PostgresImages.java:
--------------------------------------------------------------------------------
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.postgres.testutils;
19 |
20 | /** Postgres docker images. */
21 | public interface PostgresImages {
22 | String POSTGRES_15 = "postgres:15.7";
23 | String POSTGRES_16 = "postgres:16.3";
24 | }
25 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-postgres/src/test/resources/log4j2-test.properties:
--------------------------------------------------------------------------------
1 | ################################################################################
2 | # Licensed to the Apache Software Foundation (ASF) under one
3 | # or more contributor license agreements. See the NOTICE file
4 | # distributed with this work for additional information
5 | # regarding copyright ownership. The ASF licenses this file
6 | # to you under the Apache License, Version 2.0 (the
7 | # "License"); you may not use this file except in compliance
8 | # with the License. You may obtain a copy of the License at
9 | #
10 | # http://www.apache.org/licenses/LICENSE-2.0
11 | #
12 | # Unless required by applicable law or agreed to in writing, software
13 | # distributed under the License is distributed on an "AS IS" BASIS,
14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | # See the License for the specific language governing permissions and
16 | # limitations under the License.
17 | ################################################################################
18 |
19 | # Set root logger level to OFF to not flood build logs
20 | # set manually to INFO for debugging purposes
21 | rootLogger.level = OFF
22 | rootLogger.appenderRef.test.ref = TestLogger
23 |
24 | appender.testlogger.name = TestLogger
25 | appender.testlogger.type = CONSOLE
26 | appender.testlogger.target = SYSTEM_ERR
27 | appender.testlogger.layout.type = PatternLayout
28 | appender.testlogger.layout.pattern = %-4r [%t] %-5p %c %x - %m%n
29 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-sqlserver/src/main/java/org/apache/flink/connector/jdbc/sqlserver/database/SqlServerFactory.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.connector.jdbc.sqlserver.database;
20 |
21 | import org.apache.flink.annotation.Internal;
22 | import org.apache.flink.connector.jdbc.core.database.JdbcFactory;
23 | import org.apache.flink.connector.jdbc.core.database.catalog.JdbcCatalog;
24 | import org.apache.flink.connector.jdbc.core.database.dialect.JdbcDialect;
25 | import org.apache.flink.connector.jdbc.sqlserver.database.dialect.SqlServerDialect;
26 |
27 | /** Factory for {@link SqlServerDialect}. */
28 | @Internal
29 | public class SqlServerFactory implements JdbcFactory {
30 | @Override
31 | public boolean acceptsURL(String url) {
32 | return url.startsWith("jdbc:sqlserver:");
33 | }
34 |
35 | @Override
36 | public JdbcDialect createDialect() {
37 | return new SqlServerDialect();
38 | }
39 |
40 | @Override
41 | public JdbcCatalog createCatalog(
42 | ClassLoader classLoader,
43 | String catalogName,
44 | String defaultDatabase,
45 | String username,
46 | String pwd,
47 | String baseUrl) {
48 | throw new UnsupportedOperationException("Catalog for SqlServer is not supported yet.");
49 | }
50 | }
51 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-sqlserver/src/main/java/org/apache/flink/connector/jdbc/sqlserver/database/dialect/SqlServerDialectConverter.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.connector.jdbc.sqlserver.database.dialect;
20 |
21 | import org.apache.flink.annotation.Internal;
22 | import org.apache.flink.connector.jdbc.core.database.dialect.AbstractDialectConverter;
23 | import org.apache.flink.table.types.logical.LogicalType;
24 | import org.apache.flink.table.types.logical.RowType;
25 |
26 | /**
27 | * Runtime converter that responsible to convert between JDBC object and Flink internal object for
28 | * MsSql.
29 | */
30 | @Internal
31 | public class SqlServerDialectConverter extends AbstractDialectConverter {
32 |
33 | private static final long serialVersionUID = 1L;
34 |
35 | @Override
36 | public String converterName() {
37 | return "SqlServer";
38 | }
39 |
40 | public SqlServerDialectConverter(RowType rowType) {
41 | super(rowType);
42 | }
43 |
44 | @Override
45 | protected JdbcDeserializationConverter createInternalConverter(LogicalType type) {
46 | switch (type.getTypeRoot()) {
47 | case TINYINT:
48 | return val -> ((Short) val).byteValue();
49 | default:
50 | return super.createInternalConverter(type);
51 | }
52 | }
53 | }
54 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-sqlserver/src/main/resources/META-INF/services/org.apache.flink.connector.jdbc.core.database.JdbcFactory:
--------------------------------------------------------------------------------
1 | # Licensed to the Apache Software Foundation (ASF) under one or more
2 | # contributor license agreements. See the NOTICE file distributed with
3 | # this work for additional information regarding copyright ownership.
4 | # The ASF licenses this file to You under the Apache License, Version 2.0
5 | # (the "License"); you may not use this file except in compliance with
6 | # the License. You may obtain a copy of the License at
7 | #
8 | # http://www.apache.org/licenses/LICENSE-2.0
9 | #
10 | # Unless required by applicable law or agreed to in writing, software
11 | # distributed under the License is distributed on an "AS IS" BASIS,
12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | # See the License for the specific language governing permissions and
14 | # limitations under the License.
15 |
16 | org.apache.flink.connector.jdbc.sqlserver.database.SqlServerFactory
17 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-sqlserver/src/test/java/org/apache/flink/connector/jdbc/sqlserver/SqlServerTestBase.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.connector.jdbc.sqlserver;
20 |
21 | import org.apache.flink.connector.jdbc.sqlserver.testutils.SqlServerDatabase;
22 | import org.apache.flink.connector.jdbc.testutils.DatabaseMetadata;
23 | import org.apache.flink.connector.jdbc.testutils.DatabaseTest;
24 |
25 | import org.junit.jupiter.api.extension.ExtendWith;
26 |
27 | /** Base class for SqlServer testing. */
28 | @ExtendWith(SqlServerDatabase.class)
29 | public interface SqlServerTestBase extends DatabaseTest {
30 |
31 | @Override
32 | default DatabaseMetadata getMetadata() {
33 | return SqlServerDatabase.getMetadata();
34 | }
35 | }
36 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-sqlserver/src/test/java/org/apache/flink/connector/jdbc/sqlserver/testutils/SqlServerImages.java:
--------------------------------------------------------------------------------
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.sqlserver.testutils;
19 |
20 | import org.testcontainers.utility.DockerImageName;
21 |
22 | /** SqlServer docker images. */
23 | public interface SqlServerImages {
24 | DockerImageName MSSQL_AZURE_SQL_EDGE =
25 | DockerImageName.parse("mcr.microsoft.com/azure-sql-edge")
26 | .asCompatibleSubstituteFor("mcr.microsoft.com/mssql/server");
27 | }
28 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-sqlserver/src/test/resources/log4j2-test.properties:
--------------------------------------------------------------------------------
1 | ################################################################################
2 | # Licensed to the Apache Software Foundation (ASF) under one
3 | # or more contributor license agreements. See the NOTICE file
4 | # distributed with this work for additional information
5 | # regarding copyright ownership. The ASF licenses this file
6 | # to you under the Apache License, Version 2.0 (the
7 | # "License"); you may not use this file except in compliance
8 | # with the License. You may obtain a copy of the License at
9 | #
10 | # http://www.apache.org/licenses/LICENSE-2.0
11 | #
12 | # Unless required by applicable law or agreed to in writing, software
13 | # distributed under the License is distributed on an "AS IS" BASIS,
14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | # See the License for the specific language governing permissions and
16 | # limitations under the License.
17 | ################################################################################
18 |
19 | # Set root logger level to OFF to not flood build logs
20 | # set manually to INFO for debugging purposes
21 | rootLogger.level = OFF
22 | rootLogger.appenderRef.test.ref = TestLogger
23 |
24 | appender.testlogger.name = TestLogger
25 | appender.testlogger.type = CONSOLE
26 | appender.testlogger.target = SYSTEM_ERR
27 | appender.testlogger.layout.type = PatternLayout
28 | appender.testlogger.layout.pattern = %-4r [%t] %-5p %c %x - %m%n
29 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-trino/src/main/java/org/apache/flink/connector/jdbc/trino/database/TrinoFactory.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.connector.jdbc.trino.database;
20 |
21 | import org.apache.flink.annotation.Internal;
22 | import org.apache.flink.connector.jdbc.core.database.JdbcFactory;
23 | import org.apache.flink.connector.jdbc.core.database.catalog.JdbcCatalog;
24 | import org.apache.flink.connector.jdbc.core.database.dialect.JdbcDialect;
25 | import org.apache.flink.connector.jdbc.trino.database.dialect.TrinoDialect;
26 |
27 | /** Factory for {@link TrinoDialect}. */
28 | @Internal
29 | public class TrinoFactory implements JdbcFactory {
30 | @Override
31 | public boolean acceptsURL(String url) {
32 | return url.startsWith("jdbc:trino:");
33 | }
34 |
35 | @Override
36 | public JdbcDialect createDialect() {
37 | return new TrinoDialect();
38 | }
39 |
40 | @Override
41 | public JdbcCatalog createCatalog(
42 | ClassLoader classLoader,
43 | String catalogName,
44 | String defaultDatabase,
45 | String username,
46 | String pwd,
47 | String baseUrl) {
48 | throw new UnsupportedOperationException("Catalog for Trino is not supported yet.");
49 | }
50 | }
51 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-trino/src/main/resources/META-INF/services/org.apache.flink.connector.jdbc.core.database.JdbcFactory:
--------------------------------------------------------------------------------
1 | # Licensed to the Apache Software Foundation (ASF) under one or more
2 | # contributor license agreements. See the NOTICE file distributed with
3 | # this work for additional information regarding copyright ownership.
4 | # The ASF licenses this file to You under the Apache License, Version 2.0
5 | # (the "License"); you may not use this file except in compliance with
6 | # the License. You may obtain a copy of the License at
7 | #
8 | # http://www.apache.org/licenses/LICENSE-2.0
9 | #
10 | # Unless required by applicable law or agreed to in writing, software
11 | # distributed under the License is distributed on an "AS IS" BASIS,
12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | # See the License for the specific language governing permissions and
14 | # limitations under the License.
15 |
16 | org.apache.flink.connector.jdbc.trino.database.TrinoFactory
17 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-trino/src/test/java/org/apache/flink/connector/jdbc/trino/TrinoTestBase.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.connector.jdbc.trino;
20 |
21 | import org.apache.flink.connector.jdbc.testutils.DatabaseMetadata;
22 | import org.apache.flink.connector.jdbc.testutils.DatabaseTest;
23 | import org.apache.flink.connector.jdbc.trino.testutils.TrinoDatabase;
24 |
25 | import org.junit.jupiter.api.extension.ExtendWith;
26 |
27 | /** Base class for Trino testing. */
28 | @ExtendWith(TrinoDatabase.class)
29 | public interface TrinoTestBase extends DatabaseTest {
30 |
31 | @Override
32 | default DatabaseMetadata getMetadata() {
33 | return TrinoDatabase.getMetadata();
34 | }
35 |
36 | default DatabaseMetadata getMetadataDatabase() {
37 | return TrinoDatabase.getDatabaseMetadata();
38 | }
39 | }
40 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-trino/src/test/java/org/apache/flink/connector/jdbc/trino/testutils/TrinoImages.java:
--------------------------------------------------------------------------------
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.trino.testutils;
19 |
20 | /** Postgres docker images. */
21 | public interface TrinoImages {
22 | String TRINO_IMAGE = "trinodb/trino:451";
23 | }
24 |
--------------------------------------------------------------------------------
/flink-connector-jdbc-trino/src/test/resources/log4j2-test.properties:
--------------------------------------------------------------------------------
1 | ################################################################################
2 | # Licensed to the Apache Software Foundation (ASF) under one
3 | # or more contributor license agreements. See the NOTICE file
4 | # distributed with this work for additional information
5 | # regarding copyright ownership. The ASF licenses this file
6 | # to you under the Apache License, Version 2.0 (the
7 | # "License"); you may not use this file except in compliance
8 | # with the License. You may obtain a copy of the License at
9 | #
10 | # http://www.apache.org/licenses/LICENSE-2.0
11 | #
12 | # Unless required by applicable law or agreed to in writing, software
13 | # distributed under the License is distributed on an "AS IS" BASIS,
14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | # See the License for the specific language governing permissions and
16 | # limitations under the License.
17 | ################################################################################
18 |
19 | # Set root logger level to OFF to not flood build logs
20 | # set manually to INFO for debugging purposes
21 | rootLogger.level = OFF
22 | rootLogger.appenderRef.test.ref = TestLogger
23 |
24 | appender.testlogger.name = TestLogger
25 | appender.testlogger.type = CONSOLE
26 | appender.testlogger.target = SYSTEM_ERR
27 | appender.testlogger.layout.type = PatternLayout
28 | appender.testlogger.layout.pattern = %-4r [%t] %-5p %c %x - %m%n
29 |
--------------------------------------------------------------------------------
/tools/ci/log4j.properties:
--------------------------------------------------------------------------------
1 | ################################################################################
2 | # Licensed to the Apache Software Foundation (ASF) under one
3 | # or more contributor license agreements. See the NOTICE file
4 | # distributed with this work for additional information
5 | # regarding copyright ownership. The ASF licenses this file
6 | # to you under the Apache License, Version 2.0 (the
7 | # "License"); you may not use this file except in compliance
8 | # with the License. You may obtain a copy of the License at
9 | #
10 | # http://www.apache.org/licenses/LICENSE-2.0
11 | #
12 | # Unless required by applicable law or agreed to in writing, software
13 | # distributed under the License is distributed on an "AS IS" BASIS,
14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | # See the License for the specific language governing permissions and
16 | # limitations under the License.
17 | ################################################################################
18 |
19 | # Set root logger level to OFF to not flood build logs
20 | # set manually to INFO for debugging purposes
21 | rootLogger.level = OFF
22 | rootLogger.appenderRef.out.ref = ConsoleAppender
23 |
24 | # -----------------------------------------------------------------------------
25 | # Console (use 'console')
26 | # -----------------------------------------------------------------------------
27 |
28 | appender.console.name = ConsoleAppender
29 | appender.console.type = CONSOLE
30 | appender.console.layout.type = PatternLayout
31 | appender.console.layout.pattern = %d{HH:mm:ss,SSS} [%20t] %-5p %-60c %x - %m%n
32 |
33 | # -----------------------------------------------------------------------------
34 | # File (use 'file')
35 | # -----------------------------------------------------------------------------
36 | appender.file.name = FileAppender
37 | appender.file.type = FILE
38 | appender.file.fileName = ${sys:log.dir}/mvn-${sys:mvn.forkNumber:-output}.log
39 | appender.file.layout.type = PatternLayout
40 | appender.file.layout.pattern = %d{HH:mm:ss,SSS} [%20t] %-5p %-60c %x - %m%n
41 | appender.file.createOnDemand = true
42 |
43 | # suppress the irrelevant (wrong) warnings from the netty channel handler
44 | logger.netty.name = org.jboss.netty.channel.DefaultChannelPipeline
45 | logger.netty.level = ERROR
46 |
--------------------------------------------------------------------------------
/tools/maven/suppressions.xml:
--------------------------------------------------------------------------------
1 |
2 |
20 |
21 |
24 |
25 |
26 |
27 |
--------------------------------------------------------------------------------