This is an automated email from the ASF dual-hosted git repository.

wanglijie pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/flink-connector-jdbc.git

commit 508b152bf7ee50643b6b3844f20289b6461e3c25
Author: Joao Boto <b...@boto.pro>
AuthorDate: Tue Mar 28 17:18:35 2023 +0200

    [FLINK-31649] Fix archunit ITCase needs minicluster
---
 .../6b9ab1b0-c14d-4667-bab5-407b81fba98b           | 36 ----------------------
 .../apache/flink/connector/jdbc/JdbcITCase.java    |  3 +-
 .../jdbc/catalog/MySqlCatalogTestBase.java         |  3 +-
 .../jdbc/catalog/PostgresCatalogTestBase.java      |  3 +-
 ...ProviderDriverClassConcurrentLoadingITCase.java |  3 +-
 .../connector/jdbc/testutils/JdbcITCaseBase.java   | 36 ++++++++++++++++++++++
 6 files changed, 44 insertions(+), 40 deletions(-)

diff --git 
a/flink-connector-jdbc/archunit-violations/6b9ab1b0-c14d-4667-bab5-407b81fba98b 
b/flink-connector-jdbc/archunit-violations/6b9ab1b0-c14d-4667-bab5-407b81fba98b
index e54d3ff..e69de29 100644
--- 
a/flink-connector-jdbc/archunit-violations/6b9ab1b0-c14d-4667-bab5-407b81fba98b
+++ 
b/flink-connector-jdbc/archunit-violations/6b9ab1b0-c14d-4667-bab5-407b81fba98b
@@ -1,36 +0,0 @@
-org.apache.flink.connector.jdbc.JdbcITCase does not satisfy: only one of the 
following predicates match:\
-* reside in a package 'org.apache.flink.runtime.*' and contain any fields that 
are static, final, and of type InternalMiniClusterExtension and annotated with 
@RegisterExtension\
-* reside outside of package 'org.apache.flink.runtime.*' and contain any 
fields that are static, final, and of type MiniClusterExtension and annotated 
with @RegisterExtension\
-* reside in a package 'org.apache.flink.runtime.*' and is annotated with 
@ExtendWith with class InternalMiniClusterExtension\
-* reside outside of package 'org.apache.flink.runtime.*' and is annotated with 
@ExtendWith with class MiniClusterExtension\
- or contain any fields that are public, static, and of type 
MiniClusterWithClientResource and final and annotated with @ClassRule or 
contain any fields that is of type MiniClusterWithClientResource and public and 
final and not static and annotated with @Rule
-org.apache.flink.connector.jdbc.catalog.MySqlCatalogITCase does not satisfy: 
only one of the following predicates match:\
-* reside in a package 'org.apache.flink.runtime.*' and contain any fields that 
are static, final, and of type InternalMiniClusterExtension and annotated with 
@RegisterExtension\
-* reside outside of package 'org.apache.flink.runtime.*' and contain any 
fields that are static, final, and of type MiniClusterExtension and annotated 
with @RegisterExtension\
-* reside in a package 'org.apache.flink.runtime.*' and is annotated with 
@ExtendWith with class InternalMiniClusterExtension\
-* reside outside of package 'org.apache.flink.runtime.*' and is annotated with 
@ExtendWith with class MiniClusterExtension\
- or contain any fields that are public, static, and of type 
MiniClusterWithClientResource and final and annotated with @ClassRule or 
contain any fields that is of type MiniClusterWithClientResource and public and 
final and not static and annotated with @Rule
-org.apache.flink.connector.jdbc.catalog.MySql57CatalogITCase does not satisfy: 
only one of the following predicates match:\
-* reside in a package 'org.apache.flink.runtime.*' and contain any fields that 
are static, final, and of type InternalMiniClusterExtension and annotated with 
@RegisterExtension\
-* reside outside of package 'org.apache.flink.runtime.*' and contain any 
fields that are static, final, and of type MiniClusterExtension and annotated 
with @RegisterExtension\
-* reside in a package 'org.apache.flink.runtime.*' and is annotated with 
@ExtendWith with class InternalMiniClusterExtension\
-* reside outside of package 'org.apache.flink.runtime.*' and is annotated with 
@ExtendWith with class MiniClusterExtension\
- or contain any fields that are public, static, and of type 
MiniClusterWithClientResource and final and annotated with @ClassRule or 
contain any fields that is of type MiniClusterWithClientResource and public and 
final and not static and annotated with @Rule
-org.apache.flink.connector.jdbc.catalog.MySql56CatalogITCase does not satisfy: 
only one of the following predicates match:\
-* reside in a package 'org.apache.flink.runtime.*' and contain any fields that 
are static, final, and of type InternalMiniClusterExtension and annotated with 
@RegisterExtension\
-* reside outside of package 'org.apache.flink.runtime.*' and contain any 
fields that are static, final, and of type MiniClusterExtension and annotated 
with @RegisterExtension\
-* reside in a package 'org.apache.flink.runtime.*' and is annotated with 
@ExtendWith with class InternalMiniClusterExtension\
-* reside outside of package 'org.apache.flink.runtime.*' and is annotated with 
@ExtendWith with class MiniClusterExtension\
- or contain any fields that are public, static, and of type 
MiniClusterWithClientResource and final and annotated with @ClassRule or 
contain any fields that is of type MiniClusterWithClientResource and public and 
final and not static and annotated with @Rule
-org.apache.flink.connector.jdbc.catalog.PostgresCatalogITCase does not 
satisfy: only one of the following predicates match:\
-* reside in a package 'org.apache.flink.runtime.*' and contain any fields that 
are static, final, and of type InternalMiniClusterExtension and annotated with 
@RegisterExtension\
-* reside outside of package 'org.apache.flink.runtime.*' and contain any 
fields that are static, final, and of type MiniClusterExtension and annotated 
with @RegisterExtension\
-* reside in a package 'org.apache.flink.runtime.*' and is annotated with 
@ExtendWith with class InternalMiniClusterExtension\
-* reside outside of package 'org.apache.flink.runtime.*' and is annotated with 
@ExtendWith with class MiniClusterExtension\
- or contain any fields that are public, static, and of type 
MiniClusterWithClientResource and final and annotated with @ClassRule or 
contain any fields that is of type MiniClusterWithClientResource and public and 
final and not static and annotated with @Rule
-org.apache.flink.connector.jdbc.internal.connection.SimpleJdbcConnectionProviderDriverClassConcurrentLoadingITCase
 does not satisfy: only one of the following predicates match:\
-* reside in a package 'org.apache.flink.runtime.*' and contain any fields that 
are static, final, and of type InternalMiniClusterExtension and annotated with 
@RegisterExtension\
-* reside outside of package 'org.apache.flink.runtime.*' and contain any 
fields that are static, final, and of type MiniClusterExtension and annotated 
with @RegisterExtension\
-* reside in a package 'org.apache.flink.runtime.*' and is annotated with 
@ExtendWith with class InternalMiniClusterExtension\
-* reside outside of package 'org.apache.flink.runtime.*' and is annotated with 
@ExtendWith with class MiniClusterExtension\
- or contain any fields that are public, static, and of type 
MiniClusterWithClientResource and final and annotated with @ClassRule or 
contain any fields that is of type MiniClusterWithClientResource and public and 
final and not static and annotated with @Rule
\ No newline at end of file
diff --git 
a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/JdbcITCase.java
 
b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/JdbcITCase.java
index 8c521dc..7dfc03f 100644
--- 
a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/JdbcITCase.java
+++ 
b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/JdbcITCase.java
@@ -20,6 +20,7 @@ package org.apache.flink.connector.jdbc;
 import org.apache.flink.api.common.restartstrategy.RestartStrategies;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.connector.jdbc.databases.DatabaseMetadata;
+import org.apache.flink.connector.jdbc.testutils.JdbcITCaseBase;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.util.function.FunctionWithException;
 
@@ -48,7 +49,7 @@ import static 
org.apache.flink.util.Preconditions.checkNotNull;
 import static org.assertj.core.api.Assertions.assertThat;
 
 /** Smoke tests for the {@link JdbcSink} and the underlying classes. */
-public class JdbcITCase extends JdbcTestBase {
+public class JdbcITCase extends JdbcTestBase implements JdbcITCaseBase {
 
     public static final JdbcStatementBuilder<TestEntry> 
TEST_ENTRY_JDBC_STATEMENT_BUILDER =
             (ps, t) -> {
diff --git 
a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/MySqlCatalogTestBase.java
 
b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/MySqlCatalogTestBase.java
index e46b029..a521d90 100644
--- 
a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/MySqlCatalogTestBase.java
+++ 
b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/MySqlCatalogTestBase.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.connector.jdbc.catalog;
 
+import org.apache.flink.connector.jdbc.testutils.JdbcITCaseBase;
 import org.apache.flink.table.api.DataTypes;
 import org.apache.flink.table.api.EnvironmentSettings;
 import org.apache.flink.table.api.Schema;
@@ -56,7 +57,7 @@ import static org.assertj.core.api.Assertions.assertThat;
 import static org.assertj.core.api.Assertions.assertThatThrownBy;
 
 /** Test base for {@link MySqlCatalog}. */
-abstract class MySqlCatalogTestBase {
+abstract class MySqlCatalogTestBase implements JdbcITCaseBase {
 
     public static final Logger LOG = 
LoggerFactory.getLogger(MySqlCatalogTestBase.class);
     protected static final String TEST_CATALOG_NAME = "mysql_catalog";
diff --git 
a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/PostgresCatalogTestBase.java
 
b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/PostgresCatalogTestBase.java
index bd3b982..f5a5e34 100644
--- 
a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/PostgresCatalogTestBase.java
+++ 
b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/PostgresCatalogTestBase.java
@@ -19,6 +19,7 @@
 package org.apache.flink.connector.jdbc.catalog;
 
 import org.apache.flink.connector.jdbc.databases.postgres.PostgresDatabase;
+import org.apache.flink.connector.jdbc.testutils.JdbcITCaseBase;
 import org.apache.flink.table.api.DataTypes;
 import org.apache.flink.table.api.Schema;
 import org.apache.flink.table.types.logical.DecimalType;
@@ -33,7 +34,7 @@ import java.sql.SQLException;
 import java.sql.Statement;
 
 /** Test base for {@link PostgresCatalog}. */
-class PostgresCatalogTestBase implements PostgresDatabase {
+class PostgresCatalogTestBase implements PostgresDatabase, JdbcITCaseBase {
 
     public static final Logger LOG = 
LoggerFactory.getLogger(PostgresCatalogTestBase.class);
 
diff --git 
a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/internal/connection/SimpleJdbcConnectionProviderDriverClassConcurrentLoadingITCase.java
 
b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/internal/connection/SimpleJdbcConnectionProviderDriverClassConcurrentLoadingITCase.java
index 5fb3676..da66317 100644
--- 
a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/internal/connection/SimpleJdbcConnectionProviderDriverClassConcurrentLoadingITCase.java
+++ 
b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/internal/connection/SimpleJdbcConnectionProviderDriverClassConcurrentLoadingITCase.java
@@ -20,6 +20,7 @@ package org.apache.flink.connector.jdbc.internal.connection;
 
 import org.apache.flink.connector.jdbc.JdbcConnectionOptions;
 import org.apache.flink.connector.jdbc.fakedb.FakeDBUtils;
+import org.apache.flink.connector.jdbc.testutils.JdbcITCaseBase;
 import org.apache.flink.core.testutils.CheckedThread;
 
 import org.junit.jupiter.api.Test;
@@ -37,7 +38,7 @@ import static org.assertj.core.api.Assertions.assertThat;
  * This test deals with sql driver class loading issues; run as an ITCase so 
it won't be interfered
  * with by other tests.
  */
-class SimpleJdbcConnectionProviderDriverClassConcurrentLoadingITCase {
+class SimpleJdbcConnectionProviderDriverClassConcurrentLoadingITCase 
implements JdbcITCaseBase {
     private static boolean isClassLoaded(ClassLoader classLoader, String 
className)
             throws Exception {
         do {
diff --git 
a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/JdbcITCaseBase.java
 
b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/JdbcITCaseBase.java
new file mode 100644
index 0000000..0ce74f4
--- /dev/null
+++ 
b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/JdbcITCaseBase.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.jdbc.testutils;
+
+import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
+import org.apache.flink.test.junit5.MiniClusterExtension;
+
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+/** A base for ITCase implementations. */
+public interface JdbcITCaseBase {
+
+    @RegisterExtension
+    MiniClusterExtension MINI_CLUSTER =
+            new MiniClusterExtension(
+                    new MiniClusterResourceConfiguration.Builder()
+                            .setNumberTaskManagers(2)
+                            .setNumberSlotsPerTaskManager(2)
+                            .build());
+}

Reply via email to