|
| 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.cdc.connectors.mysql.table; |
| 19 | + |
| 20 | +import org.apache.flink.cdc.connectors.mysql.source.MySqlSourceTestBase; |
| 21 | +import org.apache.flink.cdc.connectors.mysql.testutils.MySqlContainer; |
| 22 | +import org.apache.flink.cdc.connectors.mysql.testutils.MySqlVersion; |
| 23 | +import org.apache.flink.cdc.connectors.mysql.testutils.UniqueDatabase; |
| 24 | +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; |
| 25 | +import org.apache.flink.table.api.EnvironmentSettings; |
| 26 | +import org.apache.flink.table.api.TableResult; |
| 27 | +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; |
| 28 | +import org.apache.flink.types.Row; |
| 29 | +import org.apache.flink.util.CloseableIterator; |
| 30 | + |
| 31 | +import org.junit.After; |
| 32 | +import org.junit.Before; |
| 33 | +import org.junit.Test; |
| 34 | +import org.junit.runner.RunWith; |
| 35 | +import org.junit.runners.Parameterized; |
| 36 | +import org.slf4j.Logger; |
| 37 | +import org.slf4j.LoggerFactory; |
| 38 | +import org.testcontainers.lifecycle.Startables; |
| 39 | + |
| 40 | +import java.sql.Connection; |
| 41 | +import java.sql.Statement; |
| 42 | +import java.util.ArrayList; |
| 43 | +import java.util.Arrays; |
| 44 | +import java.util.Iterator; |
| 45 | +import java.util.List; |
| 46 | +import java.util.Random; |
| 47 | +import java.util.stream.Stream; |
| 48 | + |
| 49 | +import static org.apache.flink.api.common.JobStatus.RUNNING; |
| 50 | + |
| 51 | +/** Integration tests for MySQL Table source. */ |
| 52 | +@RunWith(Parameterized.class) |
| 53 | +public class MySqlJsonArrayAsKeyIndexITCase extends MySqlSourceTestBase { |
| 54 | + |
| 55 | + private static final Logger LOG = LoggerFactory.getLogger(MySqlJsonArrayAsKeyIndexITCase.class); |
| 56 | + |
| 57 | + private static final String TEST_USER = "mysqluser"; |
| 58 | + private static final String TEST_PASSWORD = "mysqlpw"; |
| 59 | + |
| 60 | + private final StreamExecutionEnvironment env = |
| 61 | + StreamExecutionEnvironment.getExecutionEnvironment(); |
| 62 | + private final StreamTableEnvironment tEnv = |
| 63 | + StreamTableEnvironment.create( |
| 64 | + env, EnvironmentSettings.newInstance().inStreamingMode().build()); |
| 65 | + |
| 66 | + @Parameterized.Parameters(name = "incrementalSnapshot: {0}") |
| 67 | + public static Object[] parameters() { |
| 68 | + // MySQL 8.0.17 brought the `CAST(JSON_EXTRACT AS ARRAY)` syntax firstly, and originates the |
| 69 | + // "extra 0 byte" bug. |
| 70 | + // MySQL 8.0.18 changed the TYPED_ARRAY internal enum value from 244 to 20, but didn't fix |
| 71 | + // the bug. |
| 72 | + // MySQL 8.0.19 fixed this issue (eventually). |
| 73 | + return new Object[][] { |
| 74 | + new Object[] {MySqlVersion.V8_0_17}, |
| 75 | + new Object[] {MySqlVersion.V8_0_18}, |
| 76 | + new Object[] {MySqlVersion.V8_0_19} |
| 77 | + }; |
| 78 | + } |
| 79 | + |
| 80 | + private final MySqlVersion version; |
| 81 | + private final MySqlContainer container; |
| 82 | + |
| 83 | + public MySqlJsonArrayAsKeyIndexITCase(MySqlVersion version) { |
| 84 | + this.version = version; |
| 85 | + this.container = createMySqlContainer(version, "docker/server-gtids/expire-seconds/my.cnf"); |
| 86 | + } |
| 87 | + |
| 88 | + @Before |
| 89 | + public void before() { |
| 90 | + LOG.info("Starting MySQL {} containers...", version); |
| 91 | + Startables.deepStart(Stream.of(container)).join(); |
| 92 | + LOG.info("Container MySQL {} is started.", version); |
| 93 | + } |
| 94 | + |
| 95 | + @After |
| 96 | + public void after() { |
| 97 | + LOG.info("Stopping MySQL {} containers...", version); |
| 98 | + container.stop(); |
| 99 | + LOG.info("Container MySQL {} is stopped.", version); |
| 100 | + } |
| 101 | + |
| 102 | + @Test |
| 103 | + public void testJsonArrayAsKeyIndex() { |
| 104 | + UniqueDatabase jaakiDatabase = |
| 105 | + new UniqueDatabase(container, "json_array_as_key", TEST_USER, TEST_PASSWORD); |
| 106 | + jaakiDatabase.createAndInitialize(); |
| 107 | + |
| 108 | + String sourceDDL = |
| 109 | + String.format( |
| 110 | + "CREATE TABLE json_array_as_key (\n" |
| 111 | + + " id BIGINT NOT NULL,\n" |
| 112 | + + " PRIMARY KEY(id) NOT ENFORCED" |
| 113 | + + ") WITH (" |
| 114 | + + " 'connector' = 'mysql-cdc'," |
| 115 | + + " 'hostname' = '%s'," |
| 116 | + + " 'port' = '%s'," |
| 117 | + + " 'username' = '%s'," |
| 118 | + + " 'password' = '%s'," |
| 119 | + + " 'database-name' = '%s'," |
| 120 | + + " 'table-name' = '%s'," |
| 121 | + + " 'scan.startup.mode' = 'earliest-offset'," |
| 122 | + + " 'server-time-zone' = 'UTC'," |
| 123 | + + " 'server-id' = '%s'," |
| 124 | + + " 'scan.incremental.snapshot.enabled' = 'true'" |
| 125 | + + ")", |
| 126 | + container.getHost(), |
| 127 | + container.getDatabasePort(), |
| 128 | + TEST_USER, |
| 129 | + TEST_PASSWORD, |
| 130 | + jaakiDatabase.getDatabaseName(), |
| 131 | + "json_array_as_key", |
| 132 | + getServerId()); |
| 133 | + tEnv.executeSql(sourceDDL); |
| 134 | + |
| 135 | + try (Connection connection = jaakiDatabase.getJdbcConnection(); |
| 136 | + Statement statement = connection.createStatement()) { |
| 137 | + statement.execute("INSERT INTO json_array_as_key(id) VALUES (18),(19);"); |
| 138 | + statement.execute("DELETE FROM json_array_as_key WHERE id=19;"); |
| 139 | + } catch (Exception e) { |
| 140 | + throw new RuntimeException(e); |
| 141 | + } |
| 142 | + |
| 143 | + // async submit job |
| 144 | + TableResult result = tEnv.executeSql("SELECT * FROM json_array_as_key"); |
| 145 | + |
| 146 | + try { |
| 147 | + // wait for the source startup, we don't have a better way to wait it, use sleep for |
| 148 | + // now |
| 149 | + do { |
| 150 | + Thread.sleep(5000L); |
| 151 | + } while (result.getJobClient().get().getJobStatus().get() != RUNNING); |
| 152 | + } catch (Exception e) { |
| 153 | + throw new RuntimeException(e); |
| 154 | + } |
| 155 | + |
| 156 | + CloseableIterator<Row> iterator = result.collect(); |
| 157 | + |
| 158 | + String[] expected = |
| 159 | + new String[] { |
| 160 | + // snapshot records |
| 161 | + "+I[17]", "+I[18]", "+I[19]", "-D[19]", |
| 162 | + }; |
| 163 | + |
| 164 | + assertEqualsInAnyOrder(Arrays.asList(expected), fetchRows(iterator, expected.length)); |
| 165 | + |
| 166 | + try { |
| 167 | + result.getJobClient().get().cancel().get(); |
| 168 | + } catch (Exception e) { |
| 169 | + throw new RuntimeException(e); |
| 170 | + } |
| 171 | + } |
| 172 | + |
| 173 | + private static List<String> fetchRows(Iterator<Row> iter, int size) { |
| 174 | + List<String> rows = new ArrayList<>(size); |
| 175 | + while (size > 0 && iter.hasNext()) { |
| 176 | + Row row = iter.next(); |
| 177 | + rows.add(row.toString()); |
| 178 | + size--; |
| 179 | + } |
| 180 | + return rows; |
| 181 | + } |
| 182 | + |
| 183 | + private String getServerId() { |
| 184 | + final Random random = new Random(); |
| 185 | + int serverId = random.nextInt(100) + 5400; |
| 186 | + return serverId + "-" + (serverId + env.getParallelism()); |
| 187 | + } |
| 188 | +} |
0 commit comments