[mysql] Fix the restoration failure when checkpoint happened in one MySQL transaction
parent
22f2ac8a75
commit
a785efab5a
@ -0,0 +1,45 @@
|
||||
/*
|
||||
* 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 com.ververica.cdc.connectors.mysql.source.offset;
|
||||
|
||||
import org.apache.flink.annotation.Internal;
|
||||
|
||||
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
|
||||
/** Serializer implementation for a {@link BinlogOffset}. */
|
||||
@Internal
|
||||
public class BinlogOffsetSerializer {
|
||||
|
||||
public static final BinlogOffsetSerializer INSTANCE = new BinlogOffsetSerializer();
|
||||
|
||||
public byte[] serialize(BinlogOffset binlogOffset) throws IOException {
|
||||
// use JSON serialization
|
||||
ObjectMapper objectMapper = new ObjectMapper();
|
||||
return objectMapper.writeValueAsBytes(binlogOffset.getOffset());
|
||||
}
|
||||
|
||||
public BinlogOffset deserialize(byte[] bytes) throws IOException {
|
||||
ObjectMapper objectMapper = new ObjectMapper();
|
||||
Map<String, String> offset = objectMapper.readValue(bytes, Map.class);
|
||||
return new BinlogOffset(offset);
|
||||
}
|
||||
}
|
@ -0,0 +1,251 @@
|
||||
/*
|
||||
* 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 com.ververica.cdc.connectors.mysql.source.reader;
|
||||
|
||||
import org.apache.flink.api.common.eventtime.Watermark;
|
||||
import org.apache.flink.api.common.typeinfo.TypeInformation;
|
||||
import org.apache.flink.api.connector.source.ReaderOutput;
|
||||
import org.apache.flink.api.connector.source.SourceOutput;
|
||||
import org.apache.flink.api.connector.source.SourceReaderContext;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
|
||||
import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue;
|
||||
import org.apache.flink.connector.testutils.source.reader.TestingReaderContext;
|
||||
import org.apache.flink.table.api.DataTypes;
|
||||
import org.apache.flink.table.types.DataType;
|
||||
import org.apache.flink.util.Collector;
|
||||
|
||||
import com.ververica.cdc.connectors.mysql.debezium.EmbeddedFlinkDatabaseHistory;
|
||||
import com.ververica.cdc.connectors.mysql.debezium.task.context.StatefulTaskContext;
|
||||
import com.ververica.cdc.connectors.mysql.source.MySqlParallelSourceTestBase;
|
||||
import com.ververica.cdc.connectors.mysql.source.assigners.MySqlBinlogSplitAssigner;
|
||||
import com.ververica.cdc.connectors.mysql.source.metrics.MySqlSourceReaderMetrics;
|
||||
import com.ververica.cdc.connectors.mysql.source.split.MySqlSplit;
|
||||
import com.ververica.cdc.connectors.mysql.testutils.RecordsFormatter;
|
||||
import com.ververica.cdc.connectors.mysql.testutils.UniqueDatabase;
|
||||
import com.ververica.cdc.debezium.DebeziumDeserializationSchema;
|
||||
import io.debezium.jdbc.JdbcConnection;
|
||||
import io.debezium.relational.TableId;
|
||||
import org.apache.kafka.connect.source.SourceRecord;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.sql.SQLException;
|
||||
import java.time.ZoneId;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.UUID;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
|
||||
/** Tests for {@link MySqlSourceReader}. */
|
||||
public class MySqlSourceReaderTest extends MySqlParallelSourceTestBase {
|
||||
|
||||
private final UniqueDatabase customerDatabase =
|
||||
new UniqueDatabase(MYSQL_CONTAINER, "customer", "mysqluser", "mysqlpw");
|
||||
|
||||
@Test
|
||||
public void testBinlogReadFailoverCrossTransaction() throws Exception {
|
||||
customerDatabase.createAndInitialize();
|
||||
final Configuration configuration = getConfig(new String[] {"customers"});
|
||||
final DataType dataType =
|
||||
DataTypes.ROW(
|
||||
DataTypes.FIELD("id", DataTypes.BIGINT()),
|
||||
DataTypes.FIELD("name", DataTypes.STRING()),
|
||||
DataTypes.FIELD("address", DataTypes.STRING()),
|
||||
DataTypes.FIELD("phone_number", DataTypes.STRING()));
|
||||
MySqlSplit binlogSplit = createBinlogSplit(configuration);
|
||||
|
||||
MySqlSourceReader<SourceRecord> reader = createReader(configuration);
|
||||
reader.start();
|
||||
reader.addSplits(Arrays.asList(binlogSplit));
|
||||
|
||||
// step-1: make 6 change events in one MySQL transaction
|
||||
TableId tableId = binlogSplit.getTableSchemas().keySet().iterator().next();
|
||||
makeBinlogEventsInOneTransaction(configuration, tableId.toString());
|
||||
|
||||
// step-2: fetch the first 2 records belong to the MySQL transaction
|
||||
String[] expectedRecords =
|
||||
new String[] {
|
||||
"-U[103, user_3, Shanghai, 123567891234]",
|
||||
"+U[103, user_3, Hangzhou, 123567891234]"
|
||||
};
|
||||
// the 2 records are produced by 1 operations
|
||||
List<String> actualRecords = consumeRecords(reader, dataType, 1);
|
||||
assertEquals(
|
||||
Arrays.stream(expectedRecords).sorted().collect(Collectors.toList()),
|
||||
actualRecords);
|
||||
List<MySqlSplit> splitsState = reader.snapshotState(1L);
|
||||
// check the binlog split state
|
||||
assertEquals(1, splitsState.size());
|
||||
reader.close();
|
||||
|
||||
// step-3: mock failover from a restored state
|
||||
MySqlSourceReader<SourceRecord> restartReader = createReader(configuration);
|
||||
restartReader.start();
|
||||
restartReader.addSplits(splitsState);
|
||||
|
||||
// step-4: fetch the rest 4 records belong to the MySQL transaction
|
||||
String[] expectedRestRecords =
|
||||
new String[] {
|
||||
"-D[102, user_2, Shanghai, 123567891234]",
|
||||
"+I[102, user_2, Shanghai, 123567891234]",
|
||||
"-U[103, user_3, Hangzhou, 123567891234]",
|
||||
"+U[103, user_3, Shanghai, 123567891234]"
|
||||
};
|
||||
// the 4 records are produced by 3 operations
|
||||
List<String> restRecords = consumeRecords(restartReader, dataType, 3);
|
||||
assertEquals(
|
||||
Arrays.stream(expectedRestRecords).sorted().collect(Collectors.toList()),
|
||||
restRecords);
|
||||
restartReader.close();
|
||||
}
|
||||
|
||||
private MySqlSourceReader<SourceRecord> createReader(Configuration configuration) {
|
||||
final FutureCompletingBlockingQueue<RecordsWithSplitIds<SourceRecord>> elementsQueue =
|
||||
new FutureCompletingBlockingQueue<>();
|
||||
final SourceReaderContext readerContext = new TestingReaderContext();
|
||||
final MySqlRecordEmitter<SourceRecord> recordEmitter =
|
||||
new MySqlRecordEmitter<>(
|
||||
new ForwardDeserializeSchema(),
|
||||
new MySqlSourceReaderMetrics(readerContext.metricGroup()));
|
||||
return new MySqlSourceReader<>(
|
||||
elementsQueue,
|
||||
() -> createSplitReader(configuration),
|
||||
recordEmitter,
|
||||
configuration,
|
||||
readerContext);
|
||||
}
|
||||
|
||||
private MySqlSplitReader createSplitReader(Configuration configuration) {
|
||||
return new MySqlSplitReader(configuration, 0);
|
||||
}
|
||||
|
||||
private void makeBinlogEventsInOneTransaction(Configuration configuration, String tableId)
|
||||
throws SQLException {
|
||||
JdbcConnection connection = StatefulTaskContext.getConnection(configuration);
|
||||
// make 6 binlog events by 4 operations
|
||||
connection.setAutoCommit(false);
|
||||
connection.execute(
|
||||
"UPDATE " + tableId + " SET address = 'Hangzhou' where id = 103",
|
||||
"DELETE FROM " + tableId + " where id = 102",
|
||||
"INSERT INTO " + tableId + " VALUES(102, 'user_2','Shanghai','123567891234')",
|
||||
"UPDATE " + tableId + " SET address = 'Shanghai' where id = 103");
|
||||
connection.commit();
|
||||
connection.close();
|
||||
}
|
||||
|
||||
private MySqlSplit createBinlogSplit(Configuration configuration) {
|
||||
MySqlBinlogSplitAssigner binlogSplitAssigner = new MySqlBinlogSplitAssigner(configuration);
|
||||
binlogSplitAssigner.open();
|
||||
return binlogSplitAssigner.getNext().get();
|
||||
}
|
||||
|
||||
private Configuration getConfig(String[] captureTables) {
|
||||
Map<String, String> properties = new HashMap<>();
|
||||
properties.put("database.server.name", "embedded-test");
|
||||
properties.put("database.hostname", MYSQL_CONTAINER.getHost());
|
||||
properties.put("database.port", String.valueOf(MYSQL_CONTAINER.getDatabasePort()));
|
||||
properties.put("database.user", customerDatabase.getUsername());
|
||||
properties.put("database.password", customerDatabase.getPassword());
|
||||
properties.put("database.whitelist", customerDatabase.getDatabaseName());
|
||||
properties.put("database.history.skip.unparseable.ddl", "true");
|
||||
properties.put("database.serverTimezone", ZoneId.of("UTC").toString());
|
||||
properties.put("snapshot.mode", "initial");
|
||||
properties.put("database.history", EmbeddedFlinkDatabaseHistory.class.getCanonicalName());
|
||||
properties.put("database.history.instance.name", UUID.randomUUID().toString());
|
||||
List<String> captureTableIds =
|
||||
Arrays.stream(captureTables)
|
||||
.map(tableName -> customerDatabase.getDatabaseName() + "." + tableName)
|
||||
.collect(Collectors.toList());
|
||||
properties.put("table.whitelist", String.join(",", captureTableIds));
|
||||
properties.put("scan.incremental.snapshot.chunk.size", "10");
|
||||
properties.put("scan.snapshot.fetch.size", "2");
|
||||
properties.put("scan.startup.mode", "latest-offset");
|
||||
|
||||
return Configuration.fromMap(properties);
|
||||
}
|
||||
|
||||
private List<String> consumeRecords(
|
||||
MySqlSourceReader<SourceRecord> sourceReader, DataType recordType, int changeEventNum)
|
||||
throws Exception {
|
||||
// Poll all the n records of the single split.
|
||||
final SimpleReaderOutput output = new SimpleReaderOutput();
|
||||
while (output.getResults().size() < changeEventNum) {
|
||||
sourceReader.pollNext(output);
|
||||
}
|
||||
final RecordsFormatter formatter = new RecordsFormatter(recordType);
|
||||
return formatter.format(output.getResults());
|
||||
}
|
||||
|
||||
// ------------------------------------------------------------------------
|
||||
// test utilities
|
||||
// ------------------------------------------------------------------------
|
||||
private static class SimpleReaderOutput implements ReaderOutput<SourceRecord> {
|
||||
|
||||
private final List<SourceRecord> results = new ArrayList<>();
|
||||
|
||||
@Override
|
||||
public void collect(SourceRecord record) {
|
||||
results.add(record);
|
||||
}
|
||||
|
||||
public List<SourceRecord> getResults() {
|
||||
return results;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void collect(SourceRecord record, long timestamp) {
|
||||
collect(record);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void emitWatermark(Watermark watermark) {}
|
||||
|
||||
@Override
|
||||
public void markIdle() {}
|
||||
|
||||
@Override
|
||||
public SourceOutput<SourceRecord> createOutputForSplit(java.lang.String splitId) {
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void releaseOutputForSplit(java.lang.String splitId) {}
|
||||
}
|
||||
|
||||
private static class ForwardDeserializeSchema
|
||||
implements DebeziumDeserializationSchema<SourceRecord> {
|
||||
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
@Override
|
||||
public void deserialize(SourceRecord record, Collector<SourceRecord> out) throws Exception {
|
||||
out.collect(record);
|
||||
}
|
||||
|
||||
@Override
|
||||
public TypeInformation<SourceRecord> getProducedType() {
|
||||
return TypeInformation.of(SourceRecord.class);
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,108 @@
|
||||
/*
|
||||
* 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 com.ververica.cdc.connectors.mysql.testutils;
|
||||
|
||||
import org.apache.flink.api.common.typeinfo.TypeInformation;
|
||||
import org.apache.flink.table.data.RowData;
|
||||
import org.apache.flink.table.data.conversion.RowRowConverter;
|
||||
import org.apache.flink.table.types.DataType;
|
||||
import org.apache.flink.table.types.logical.RowType;
|
||||
import org.apache.flink.table.types.utils.TypeConversions;
|
||||
import org.apache.flink.types.Row;
|
||||
import org.apache.flink.util.Collector;
|
||||
|
||||
import com.ververica.cdc.debezium.DebeziumDeserializationSchema;
|
||||
import com.ververica.cdc.debezium.table.RowDataDebeziumDeserializeSchema;
|
||||
import org.apache.kafka.connect.source.SourceRecord;
|
||||
|
||||
import java.time.ZoneId;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static com.ververica.cdc.connectors.mysql.source.utils.RecordUtils.isSchemaChangeEvent;
|
||||
import static com.ververica.cdc.connectors.mysql.source.utils.RecordUtils.isWatermarkEvent;
|
||||
|
||||
/** Formatter that formats the {@link org.apache.kafka.connect.source.SourceRecord} to String. */
|
||||
public class RecordsFormatter {
|
||||
|
||||
private final DataType dataType;
|
||||
private final ZoneId zoneId;
|
||||
|
||||
private TypeInformation<RowData> typeInfo;
|
||||
private DebeziumDeserializationSchema<RowData> deserializationSchema;
|
||||
private SimpleCollector collector;
|
||||
private RowRowConverter rowRowConverter;
|
||||
|
||||
public RecordsFormatter(DataType dataType) {
|
||||
this(dataType, ZoneId.of("UTC"));
|
||||
}
|
||||
|
||||
public RecordsFormatter(DataType dataType, ZoneId zoneId) {
|
||||
this.dataType = dataType;
|
||||
this.zoneId = zoneId;
|
||||
this.typeInfo =
|
||||
(TypeInformation<RowData>) TypeConversions.fromDataTypeToLegacyInfo(dataType);
|
||||
this.deserializationSchema =
|
||||
new RowDataDebeziumDeserializeSchema(
|
||||
(RowType) dataType.getLogicalType(),
|
||||
typeInfo,
|
||||
((rowData, rowKind) -> {}),
|
||||
ZoneId.of("UTC"));
|
||||
this.collector = new SimpleCollector();
|
||||
this.rowRowConverter = RowRowConverter.create(dataType);
|
||||
rowRowConverter.open(Thread.currentThread().getContextClassLoader());
|
||||
}
|
||||
|
||||
public List<String> format(List<SourceRecord> records) {
|
||||
records.stream()
|
||||
// filter signal event
|
||||
.filter(r -> !isWatermarkEvent(r))
|
||||
// filter schema change event
|
||||
.filter(r -> !isSchemaChangeEvent(r))
|
||||
.forEach(
|
||||
r -> {
|
||||
try {
|
||||
deserializationSchema.deserialize(r, collector);
|
||||
} catch (Exception e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
});
|
||||
return collector.list.stream()
|
||||
.map(rowRowConverter::toExternal)
|
||||
.map(Row::toString)
|
||||
.sorted()
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
|
||||
private static class SimpleCollector implements Collector<RowData> {
|
||||
|
||||
private List<RowData> list = new ArrayList<>();
|
||||
|
||||
@Override
|
||||
public void collect(RowData record) {
|
||||
list.add(record);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
// do nothing
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,63 @@
|
||||
# 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.
|
||||
|
||||
# For advice on how to change settings please see
|
||||
# http://dev.mysql.com/doc/refman/5.7/en/server-configuration-defaults.html
|
||||
|
||||
[mysqld]
|
||||
#
|
||||
# Remove leading # and set to the amount of RAM for the most important data
|
||||
# cache in MySQL. Start at 70% of total RAM for dedicated server, else 10%.
|
||||
# innodb_buffer_pool_size = 128M
|
||||
#
|
||||
# Remove leading # to turn on a very important data integrity option: logging
|
||||
# changes to the binary log between backups.
|
||||
# log_bin
|
||||
#
|
||||
# Remove leading # to set options mainly useful for reporting servers.
|
||||
# The server defaults are faster for transactions and fast SELECTs.
|
||||
# Adjust sizes as needed, experiment to find the optimal values.
|
||||
# join_buffer_size = 128M
|
||||
# sort_buffer_size = 2M
|
||||
# read_rnd_buffer_size = 2M
|
||||
skip-host-cache
|
||||
skip-name-resolve
|
||||
#datadir=/var/lib/mysql
|
||||
#socket=/var/lib/mysql/mysql.sock
|
||||
#secure-file-priv=/var/lib/mysql-files
|
||||
user=mysql
|
||||
|
||||
# Disabling symbolic-links is recommended to prevent assorted security risks
|
||||
symbolic-links=0
|
||||
|
||||
#log-error=/var/log/mysqld.log
|
||||
#pid-file=/var/run/mysqld/mysqld.pid
|
||||
|
||||
# ----------------------------------------------
|
||||
# Enable the binlog for replication & CDC
|
||||
# ----------------------------------------------
|
||||
|
||||
# Enable binary replication log and set the prefix, expiration, and log format.
|
||||
# The prefix is arbitrary, expiration can be short for integration tests but would
|
||||
# be longer on a production system. Row-level info is required for ingest to work.
|
||||
# Server ID is required, but this will vary on production systems
|
||||
server-id = 223344
|
||||
log_bin = mysql-bin
|
||||
expire_logs_days = 1
|
||||
binlog_format = row
|
||||
|
||||
# enable gtid mode
|
||||
gtid_mode = on
|
||||
enforce_gtid_consistency = on
|
Loading…
Reference in New Issue