Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

🎉 source: implementation for mysql cdc #3505

Merged
merged 15 commits into from
May 24, 2021
Merged
Original file line number Diff line number Diff line change
Expand Up @@ -79,6 +79,8 @@ public abstract class StandardSourceTest {
public static final String CDC_LSN = "_ab_cdc_lsn";
public static final String CDC_UPDATED_AT = "_ab_cdc_updated_at";
public static final String CDC_DELETED_AT = "_ab_cdc_deleted_at";
public static final String CDC_LOG_FILE = "_ab_cdc_log_file";
public static final String CDC_LOG_POS = "_ab_cdc_log_pos";

private static final long JOB_ID = 0L;
private static final int JOB_ATTEMPT = 0;
Expand Down Expand Up @@ -383,7 +385,7 @@ private List<AirbyteRecordMessage> filterRecords(Collection<AirbyteMessage> mess
.collect(Collectors.toList());
}

private ConfiguredAirbyteCatalog withSourceDefinedCursors(ConfiguredAirbyteCatalog catalog) {
public ConfiguredAirbyteCatalog withSourceDefinedCursors(ConfiguredAirbyteCatalog catalog) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

why are you making this public and overriding it?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ah! I guess I was playing around with something and forgot to revert the change. Will fix this. Its not required

final ConfiguredAirbyteCatalog clone = Jsons.clone(catalog);
for (ConfiguredAirbyteStream configuredStream : clone.getStreams()) {
if (configuredStream.getSyncMode() == INCREMENTAL
Expand Down Expand Up @@ -472,6 +474,8 @@ private AirbyteRecordMessage pruneEmittedAt(AirbyteRecordMessage m) {
private AirbyteRecordMessage pruneCdcMetadata(AirbyteRecordMessage m) {
final AirbyteRecordMessage clone = Jsons.clone(m);
((ObjectNode) clone.getData()).remove(CDC_LSN);
((ObjectNode) clone.getData()).remove(CDC_LOG_FILE);
((ObjectNode) clone.getData()).remove(CDC_LOG_POS);
((ObjectNode) clone.getData()).remove(CDC_UPDATED_AT);
((ObjectNode) clone.getData()).remove(CDC_DELETED_AT);
return clone;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -87,6 +87,8 @@ public abstract class AbstractJdbcSource extends BaseConnector implements Source
public static final String CDC_LSN = "_ab_cdc_lsn";
public static final String CDC_UPDATED_AT = "_ab_cdc_updated_at";
public static final String CDC_DELETED_AT = "_ab_cdc_deleted_at";
public static final String CDC_LOG_FILE = "_ab_cdc_log_file";
public static final String CDC_LOG_POS = "_ab_cdc_log_pos";

private static final String JDBC_COLUMN_DATABASE_NAME = "TABLE_CAT";
private static final String JDBC_COLUMN_SCHEMA_NAME = "TABLE_SCHEM";
Expand Down
5 changes: 4 additions & 1 deletion airbyte-integrations/connectors/source-mysql/build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -16,13 +16,16 @@ dependencies {

implementation 'mysql:mysql-connector-java:8.0.22'
implementation 'org.apache.commons:commons-lang3:3.11'
implementation 'io.debezium:debezium-embedded:1.4.2.Final'
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: alphabetise

implementation 'io.debezium:debezium-api:1.4.2.Final'
implementation 'io.debezium:debezium-connector-mysql:1.4.2.Final'

testImplementation testFixtures(project(':airbyte-integrations:connectors:source-jdbc'))

testImplementation 'org.apache.commons:commons-lang3:3.11'
testImplementation 'org.testcontainers:mysql:1.15.1'

integrationTestJavaImplementation project(':airbyte-integrations:bases:standard-source-test')
integrationTestJavaImplementation project(':airbyte-integrations:connectors:source-mysql')

implementation files(project(':airbyte-integrations:bases:base-java').airbyteDocker.outputs)
integrationTestJavaImplementation files(project(':airbyte-integrations:bases:base-java').airbyteDocker.outputs)
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,147 @@
/*
* MIT License
*
* Copyright (c) 2020 Airbyte
*
* Permission is hereby granted, free of charge, to any person obtaining a copy
* of this software and associated documentation files (the "Software"), to deal
* in the Software without restriction, including without limitation the rights
* to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
* copies of the Software, and to permit persons to whom the Software is
* furnished to do so, subject to the following conditions:
*
* The above copyright notice and this permission notice shall be included in all
* copies or substantial portions of the Software.
*
* THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
* IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
* FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
* AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
* LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
* OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
* SOFTWARE.
*/

package io.airbyte.integrations.source.mysql;

import static io.airbyte.integrations.source.mysql.MySqlSource.MYSQL_DB_HISTORY;

import io.airbyte.commons.json.Jsons;
import io.airbyte.integrations.source.jdbc.models.CdcState;
import io.debezium.document.Document;
import io.debezium.document.DocumentReader;
import io.debezium.document.DocumentWriter;
import io.debezium.relational.history.HistoryRecord;
import java.io.BufferedWriter;
import java.io.IOException;
import java.nio.charset.Charset;
import java.nio.charset.StandardCharsets;
import java.nio.file.FileAlreadyExistsException;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.StandardOpenOption;
import java.util.function.Consumer;
import org.apache.commons.io.FileUtils;

public class AirbyteFileDatabaseHistoryStorageOperations {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

maybe just AirbyteSchemaHistoryStorage? or something a little shorter? nbd if you prefer the verbose name.


private final Path path;
private static final Charset UTF8 = StandardCharsets.UTF_8;
private final DocumentReader reader = DocumentReader.defaultReader();
private final DocumentWriter writer = DocumentWriter.defaultWriter();

public AirbyteFileDatabaseHistoryStorageOperations(final Path path) {
this.path = path;
}

public Path getPath() {
return path;
}

/**
* This implementation is is kind of similar to
* {@link io.debezium.relational.history.FileDatabaseHistory#recoverRecords(Consumer)} ()}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
* {@link io.debezium.relational.history.FileDatabaseHistory#recoverRecords(Consumer)} ()}
* {@link io.debezium.relational.history.FileDatabaseHistory#recoverRecords(Consumer)}

*/
public String read() {
StringBuilder fileAsString = new StringBuilder();
try {
for (String line : Files.readAllLines(path, UTF8)) {
if (line != null && !line.isEmpty()) {
Document record = reader.read(line);
String recordAsString = writer.write(record);
fileAsString.append(recordAsString);
fileAsString.append(System.lineSeparator());
}
}
return fileAsString.toString();
} catch (IOException e) {
throw new RuntimeException(e);
}
}

/**
* This implementation is is kind of similar to
* {@link io.debezium.relational.history.FileDatabaseHistory#start()}
*/
private void makeSureFileExists() {
try {
// Make sure the file exists ...
if (!Files.exists(path)) {
// Create parent directories if we have them ...
if (path.getParent() != null) {
Files.createDirectories(path.getParent());
}
try {
Files.createFile(path);
} catch (FileAlreadyExistsException e) {
// do nothing
}
}
} catch (IOException e) {
throw new RuntimeException(
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

IllegaltStateException?

"Unable to create history file at " + path + ": " + e.getMessage(), e);
}
}

public void persist(CdcState cdcState) {
String fileAsString = cdcState != null && cdcState.getState() != null ? Jsons
.object(cdcState.getState().get(MYSQL_DB_HISTORY), String.class) : null;

if (fileAsString == null || fileAsString.isEmpty()) {
return;
}

FileUtils.deleteQuietly(path.toFile());
makeSureFileExists();
writeToFile(fileAsString);
}

/**
* This implementation is kind of similar to
* {@link io.debezium.relational.history.FileDatabaseHistory#storeRecord(HistoryRecord)}
*
* @param fileAsString Represents the contents of the file saved in state from previous syncs
*/
private void writeToFile(String fileAsString) {
try {
String[] split = fileAsString.split(System.lineSeparator());
for (String element : split) {
Document read = reader.read(element);
String line = writer.write(read);

try (BufferedWriter historyWriter = Files
.newBufferedWriter(path, StandardOpenOption.APPEND)) {
try {
historyWriter.append(line);
historyWriter.newLine();
} catch (IOException e) {
throw new RuntimeException(e);
}
}
}
} catch (IOException e) {
throw new RuntimeException(e);
}
}

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,162 @@
/*
* MIT License
*
* Copyright (c) 2020 Airbyte
*
* Permission is hereby granted, free of charge, to any person obtaining a copy
* of this software and associated documentation files (the "Software"), to deal
* in the Software without restriction, including without limitation the rights
* to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
* copies of the Software, and to permit persons to whom the Software is
* furnished to do so, subject to the following conditions:
*
* The above copyright notice and this permission notice shall be included in all
* copies or substantial portions of the Software.
*
* THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
* IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
* FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
* AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
* LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
* OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
* SOFTWARE.
*/

package io.airbyte.integrations.source.mysql;

import static io.airbyte.integrations.source.mysql.MySqlSource.MYSQL_CDC_OFFSET;

import com.fasterxml.jackson.databind.JsonNode;
import com.google.common.base.Preconditions;
import io.airbyte.commons.json.Jsons;
import io.airbyte.integrations.source.jdbc.models.CdcState;
import java.io.EOFException;
import java.io.IOException;
import java.io.ObjectOutputStream;
import java.nio.ByteBuffer;
import java.nio.charset.StandardCharsets;
import java.nio.file.Files;
import java.nio.file.NoSuchFileException;
import java.nio.file.Path;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import java.util.stream.Collectors;
import org.apache.commons.io.FileUtils;
import org.apache.kafka.connect.errors.ConnectException;
import org.apache.kafka.connect.util.SafeObjectInputStream;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

/**
* This class handles reading and writing a debezium offset file. In many cases it is duplicating
* logic in debezium because that logic is not exposed in the public API. We mostly treat the
* contents of this state file like a black box. We know it is a Map<ByteBuffer, Bytebuffer>. We
* deserialize it to a Map<String, String> so that the state file can be human readable. If we ever
* discover that any of the contents of these offset files is not string serializable we will likely
* have to drop the human readability support and just base64 encode it.
*/
public class AirbyteFileOffsetBackingStore {

private static final Logger LOGGER = LoggerFactory.getLogger(AirbyteFileOffsetBackingStore.class);

private final Path offsetFilePath;

public AirbyteFileOffsetBackingStore(final Path offsetFilePath) {
this.offsetFilePath = offsetFilePath;
}

public Path getOffsetFilePath() {
return offsetFilePath;
}

public CdcState read() {
final Map<ByteBuffer, ByteBuffer> raw = load();

final Map<String, String> mappedAsStrings = raw.entrySet().stream().collect(Collectors.toMap(
e -> byteBufferToString(e.getKey()),
e -> byteBufferToString(e.getValue())));
final JsonNode asJson = Jsons.jsonNode(mappedAsStrings);

LOGGER.info("debezium state: {}", asJson);

return new CdcState().withState(asJson);
}

public Map<String, String> readMap() {
final Map<ByteBuffer, ByteBuffer> raw = load();

return raw.entrySet().stream().collect(Collectors.toMap(
e -> byteBufferToString(e.getKey()),
e -> byteBufferToString(e.getValue())));
}

@SuppressWarnings("unchecked")
public void persist(CdcState cdcState) {
final Map<String, String> mapAsString =
cdcState != null && cdcState.getState() != null ? Jsons.object(cdcState.getState().get(MYSQL_CDC_OFFSET), Map.class) : Collections.emptyMap();
final Map<ByteBuffer, ByteBuffer> mappedAsStrings = mapAsString.entrySet().stream().collect(Collectors.toMap(
e -> stringToByteBuffer(e.getKey()),
e -> stringToByteBuffer(e.getValue())));

FileUtils.deleteQuietly(offsetFilePath.toFile());
save(mappedAsStrings);
}

private static String byteBufferToString(ByteBuffer byteBuffer) {
Preconditions.checkNotNull(byteBuffer);
return new String(byteBuffer.array(), StandardCharsets.UTF_8);
}

private static ByteBuffer stringToByteBuffer(String s) {
Preconditions.checkNotNull(s);
return ByteBuffer.wrap(s.getBytes(StandardCharsets.UTF_8));
}

/**
* See FileOffsetBackingStore#load - logic is mostly borrowed from here. duplicated because this
* method is not public.
*/
@SuppressWarnings("unchecked")
private Map<ByteBuffer, ByteBuffer> load() {
try (final SafeObjectInputStream is = new SafeObjectInputStream(Files.newInputStream(offsetFilePath))) {
final Object obj = is.readObject();
if (!(obj instanceof HashMap))
throw new ConnectException("Expected HashMap but found " + obj.getClass());
final Map<byte[], byte[]> raw = (Map<byte[], byte[]>) obj;
final Map<ByteBuffer, ByteBuffer> data = new HashMap<>();
for (Map.Entry<byte[], byte[]> mapEntry : raw.entrySet()) {
final ByteBuffer key = (mapEntry.getKey() != null) ? ByteBuffer.wrap(mapEntry.getKey()) : null;
final ByteBuffer value = (mapEntry.getValue() != null) ? ByteBuffer.wrap(mapEntry.getValue()) : null;
data.put(key, value);
}

return data;
} catch (NoSuchFileException | EOFException e) {
// NoSuchFileException: Ignore, may be new.
// EOFException: Ignore, this means the file was missing or corrupt
return Collections.emptyMap();
} catch (IOException | ClassNotFoundException e) {
throw new ConnectException(e);
}
}

/**
* See FileOffsetBackingStore#save - logic is mostly borrowed from here. duplicated because this
* method is not public.
*/
private void save(Map<ByteBuffer, ByteBuffer> data) {
try (ObjectOutputStream os = new ObjectOutputStream(Files.newOutputStream(offsetFilePath))) {
Map<byte[], byte[]> raw = new HashMap<>();
for (Map.Entry<ByteBuffer, ByteBuffer> mapEntry : data.entrySet()) {
byte[] key = (mapEntry.getKey() != null) ? mapEntry.getKey().array() : null;
byte[] value = (mapEntry.getValue() != null) ? mapEntry.getValue().array() : null;
raw.put(key, value);
}
os.writeObject(raw);
} catch (IOException e) {
throw new ConnectException(e);
}
}

}
Loading