diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/test/java/org/apache/flink/cdc/connectors/oracle/source/OracleSourceTestBase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/test/java/org/apache/flink/cdc/connectors/oracle/source/OracleSourceTestBase.java index 7f8b83b49..63c34dd46 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/test/java/org/apache/flink/cdc/connectors/oracle/source/OracleSourceTestBase.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/test/java/org/apache/flink/cdc/connectors/oracle/source/OracleSourceTestBase.java @@ -29,6 +29,7 @@ import org.junit.BeforeClass; import org.junit.Rule; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.testcontainers.DockerClientFactory; import org.testcontainers.containers.OracleContainer; import org.testcontainers.containers.output.Slf4jLogConsumer; import org.testcontainers.lifecycle.Startables; @@ -73,7 +74,16 @@ public class OracleSourceTestBase extends TestLogger { public static final OracleContainer ORACLE_CONTAINER = new OracleContainer( - DockerImageName.parse("goodboy008/oracle-19.3.0-ee").withTag("non-cdb")) + DockerImageName.parse("goodboy008/oracle-19.3.0-ee") + .withTag( + DockerClientFactory.instance() + .client() + .versionCmd() + .exec() + .getArch() + .equals("amd64") + ? "non-cdb" + : "arm-non-cdb")) .withUsername(CONNECTOR_USER) .withPassword(CONNECTOR_PWD) .withDatabaseName(ORACLE_DATABASE) diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/test/java/org/apache/flink/cdc/connectors/oracle/table/OracleConnectorITCase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/test/java/org/apache/flink/cdc/connectors/oracle/table/OracleConnectorITCase.java index 4e2d38088..c15ce81e1 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/test/java/org/apache/flink/cdc/connectors/oracle/table/OracleConnectorITCase.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/test/java/org/apache/flink/cdc/connectors/oracle/table/OracleConnectorITCase.java @@ -843,6 +843,9 @@ public class OracleConnectorITCase { @Test public void testSnapshotToStreamingSwitchPendingTransactions() throws Exception { + + createAndInitialize("product.sql"); + Assume.assumeFalse(parallelismSnapshot); CompletableFuture finishFuture = createRecordInserters(); diff --git a/flink-cdc-e2e-tests/flink-cdc-source-e2e-tests/src/test/java/org/apache/flink/cdc/connectors/tests/OracleE2eITCase.java b/flink-cdc-e2e-tests/flink-cdc-source-e2e-tests/src/test/java/org/apache/flink/cdc/connectors/tests/OracleE2eITCase.java index f72f03be7..f2a6e9979 100644 --- a/flink-cdc-e2e-tests/flink-cdc-source-e2e-tests/src/test/java/org/apache/flink/cdc/connectors/tests/OracleE2eITCase.java +++ b/flink-cdc-e2e-tests/flink-cdc-source-e2e-tests/src/test/java/org/apache/flink/cdc/connectors/tests/OracleE2eITCase.java @@ -26,6 +26,7 @@ import org.junit.Before; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.testcontainers.DockerClientFactory; import org.testcontainers.containers.OracleContainer; import org.testcontainers.containers.output.Slf4jLogConsumer; import org.testcontainers.lifecycle.Startables; @@ -66,7 +67,17 @@ public class OracleE2eITCase extends FlinkContainerTestEnvironment { LOG.info("Starting containers..."); oracle = - new OracleContainer(DockerImageName.parse(ORACLE_IMAGE).withTag("non-cdb")) + new OracleContainer( + DockerImageName.parse(ORACLE_IMAGE) + .withTag( + DockerClientFactory.instance() + .client() + .versionCmd() + .exec() + .getArch() + .equals("amd64") + ? "non-cdb" + : "arm-non-cdb")) .withUsername(CONNECTOR_USER) .withPassword(CONNECTOR_PWD) .withDatabaseName(ORACLE_DATABASE)