-
Notifications
You must be signed in to change notification settings - Fork 4.3k
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
introduce common abstraction for CDC via debezium #4580
Merged
Merged
Changes from 10 commits
Commits
Show all changes
20 commits
Select commit
Hold shift + click to select a range
0495779
wip
subodh1810 fbc52cd
add file
subodh1810 3d34b9b
final structure
subodh1810 ec59346
few more updates
subodh1810 557fb7a
undo unwanted changes
subodh1810 071a445
add abstract test + more refinement
subodh1810 8fa0e5e
remove CDC metadata to debezium
subodh1810 eaf3b0b
Merge branch 'master' into cdc-abstraction
subodh1810 8ea4e99
Merge branch 'master' into cdc-abstraction
subodh1810 0b7581b
rename class + add missing property
subodh1810 356c884
Merge branch 'master' into cdc-abstraction
subodh1810 f164eb0
move debezium to bases + upgrade debezium version + review comments
subodh1810 1547ab0
downgrade version + minor fixes
subodh1810 eca4c7a
Merge branch 'master' into cdc-abstraction
subodh1810 924598e
reset to minutes
subodh1810 a6f7236
fix build
subodh1810 4826d0b
Merge branch 'master' into cdc-abstraction
subodh1810 c16076d
address review comments
subodh1810 e542b89
should return Optional
subodh1810 fbc318f
use common abstraction for CDC via debezium for mysql (#4604)
subodh1810 File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
23 changes: 23 additions & 0 deletions
23
airbyte-integrations/connectors/source-debezium/build.gradle
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,23 @@ | ||
plugins { | ||
id "java-test-fixtures" | ||
} | ||
|
||
project.configurations { | ||
testFixturesImplementation.extendsFrom implementation | ||
} | ||
dependencies { | ||
implementation project(':airbyte-protocol:models') | ||
|
||
implementation 'io.debezium:debezium-api:1.4.2.Final' | ||
implementation 'io.debezium:debezium-embedded:1.4.2.Final' | ||
implementation 'io.debezium:debezium-connector-mysql:1.4.2.Final' | ||
implementation 'io.debezium:debezium-connector-postgres:1.4.2.Final' | ||
|
||
testFixturesImplementation project(':airbyte-db') | ||
testFixturesImplementation project(':airbyte-integrations:bases:base-java') | ||
|
||
testFixturesImplementation 'org.junit.jupiter:junit-jupiter-engine:5.4.2' | ||
testFixturesImplementation 'org.junit.jupiter:junit-jupiter-api:5.4.2' | ||
testFixturesImplementation 'org.junit.jupiter:junit-jupiter-params:5.4.2' | ||
|
||
} | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. nit: newline |
136 changes: 136 additions & 0 deletions
136
...ebezium/src/main/java/io/airbyte/integrations/source/debezium/AirbyteDebeziumHandler.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,136 @@ | ||
/* | ||
* 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.debezium; | ||
|
||
import com.fasterxml.jackson.databind.JsonNode; | ||
import io.airbyte.commons.util.AutoCloseableIterator; | ||
import io.airbyte.commons.util.AutoCloseableIterators; | ||
import io.airbyte.commons.util.CompositeIterator; | ||
import io.airbyte.commons.util.MoreIterators; | ||
import io.airbyte.integrations.source.debezium.interfaces.CdcConnectorMetadata; | ||
import io.airbyte.integrations.source.debezium.interfaces.CdcSavedInfo; | ||
import io.airbyte.integrations.source.debezium.interfaces.CdcStateHandler; | ||
import io.airbyte.integrations.source.debezium.interfaces.CdcTargetPosition; | ||
import io.airbyte.protocol.models.AirbyteMessage; | ||
import io.airbyte.protocol.models.ConfiguredAirbyteCatalog; | ||
import io.debezium.engine.ChangeEvent; | ||
import java.time.Instant; | ||
import java.util.Collections; | ||
import java.util.Iterator; | ||
import java.util.List; | ||
import java.util.Map; | ||
import java.util.Properties; | ||
import java.util.concurrent.LinkedBlockingQueue; | ||
import java.util.function.Supplier; | ||
import org.slf4j.Logger; | ||
import org.slf4j.LoggerFactory; | ||
|
||
public class AirbyteDebeziumHandler { | ||
|
||
private static final Logger LOGGER = LoggerFactory.getLogger(AirbyteDebeziumHandler.class); | ||
/** | ||
* We use 10000 as capacity cause the default queue size and batch size of debezium is : | ||
* {@link io.debezium.config.CommonConnectorConfig#DEFAULT_MAX_BATCH_SIZE}is 2048 | ||
* {@link io.debezium.config.CommonConnectorConfig#DEFAULT_MAX_QUEUE_SIZE} is 8192 | ||
*/ | ||
private static final int QUEUE_CAPACITY = 10000; | ||
|
||
private final Properties connectorProperties; | ||
private final JsonNode config; | ||
private final CdcTargetPosition cdcTargetPosition; | ||
private final ConfiguredAirbyteCatalog catalog; | ||
private final boolean trackSchemaHistory; | ||
|
||
private final LinkedBlockingQueue<ChangeEvent<String, String>> queue; | ||
|
||
public AirbyteDebeziumHandler(JsonNode config, | ||
CdcTargetPosition cdcTargetPosition, | ||
Properties connectorProperties, | ||
ConfiguredAirbyteCatalog catalog, | ||
boolean trackSchemaHistory) { | ||
this.config = config; | ||
this.cdcTargetPosition = cdcTargetPosition; | ||
this.connectorProperties = connectorProperties; | ||
this.catalog = catalog; | ||
this.trackSchemaHistory = trackSchemaHistory; | ||
this.queue = new LinkedBlockingQueue<>(QUEUE_CAPACITY); | ||
} | ||
|
||
public List<AutoCloseableIterator<AirbyteMessage>> getIncrementalIterators(CdcSavedInfo cdcSavedInfo, | ||
CdcStateHandler cdcStateHandler, | ||
CdcConnectorMetadata cdcConnectorMetadata, | ||
Instant emittedAt) { | ||
LOGGER.info("using CDC: {}", true); | ||
// TODO: Figure out how to set the isCDC of stateManager to true. Its always false | ||
final AirbyteFileOffsetBackingStore offsetManager = AirbyteFileOffsetBackingStore.initializeState(cdcSavedInfo.getSavedOffset()); | ||
final AirbyteSchemaHistoryStorage schemaHistoryManager = schemaHistoryManager(cdcSavedInfo); | ||
final DebeziumRecordPublisher publisher = new DebeziumRecordPublisher(connectorProperties, config, catalog, offsetManager, | ||
schemaHistoryManager); | ||
publisher.start(queue); | ||
|
||
// handle state machine around pub/sub logic. | ||
final AutoCloseableIterator<ChangeEvent<String, String>> eventIterator = new DebeziumRecordIterator( | ||
queue, | ||
cdcTargetPosition, | ||
publisher::hasClosed, | ||
publisher::close); | ||
|
||
// convert to airbyte message. | ||
final AutoCloseableIterator<AirbyteMessage> messageIterator = AutoCloseableIterators | ||
.transform( | ||
eventIterator, | ||
(event) -> DebeziumEventUtils.toAirbyteMessage(event, cdcConnectorMetadata, emittedAt)); | ||
|
||
// our goal is to get the state at the time this supplier is called (i.e. after all message records | ||
// have been produced) | ||
final Supplier<AirbyteMessage> stateMessageSupplier = () -> { | ||
Map<String, String> offset = offsetManager.read(); | ||
String dbHistory = trackSchemaHistory ? schemaHistoryManager.read() : null; | ||
|
||
return cdcStateHandler.state(offset, dbHistory); | ||
}; | ||
|
||
// wrap the supplier in an iterator so that we can concat it to the message iterator. | ||
final Iterator<AirbyteMessage> stateMessageIterator = MoreIterators.singletonIteratorFromSupplier(stateMessageSupplier); | ||
|
||
// this structure guarantees that the debezium engine will be closed, before we attempt to emit the | ||
// state file. we want this so that we have a guarantee that the debezium offset file (which we use | ||
// to produce the state file) is up-to-date. | ||
final CompositeIterator<AirbyteMessage> messageIteratorWithStateDecorator = | ||
AutoCloseableIterators.concatWithEagerClose(messageIterator, AutoCloseableIterators.fromIterator(stateMessageIterator)); | ||
|
||
return Collections.singletonList(messageIteratorWithStateDecorator); | ||
} | ||
|
||
private AirbyteSchemaHistoryStorage schemaHistoryManager(CdcSavedInfo cdcSavedInfo) { | ||
if (trackSchemaHistory) { | ||
FilteredFileDatabaseHistory.setDatabaseName(config.get("database").asText()); | ||
return AirbyteSchemaHistoryStorage.initializeDBHistory(cdcSavedInfo.getSavedSchemaHistory()); | ||
} | ||
|
||
return null; | ||
} | ||
|
||
} |
160 changes: 160 additions & 0 deletions
160
.../src/main/java/io/airbyte/integrations/source/debezium/AirbyteFileOffsetBackingStore.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,160 @@ | ||
/* | ||
* 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.debezium; | ||
|
||
import com.fasterxml.jackson.databind.JsonNode; | ||
import com.google.common.base.Preconditions; | ||
import io.airbyte.commons.json.Jsons; | ||
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 Map<String, String> read() { | ||
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(JsonNode cdcState) { | ||
final Map<String, String> mapAsString = | ||
cdcState != null ? Jsons.object(cdcState, 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); | ||
} | ||
} | ||
|
||
static AirbyteFileOffsetBackingStore initializeState(JsonNode cdcState) { | ||
final Path cdcWorkingDir; | ||
try { | ||
cdcWorkingDir = Files.createTempDirectory(Path.of("/tmp"), "cdc-state-offset"); | ||
} catch (IOException e) { | ||
throw new RuntimeException(e); | ||
} | ||
final Path cdcOffsetFilePath = cdcWorkingDir.resolve("offset.dat"); | ||
|
||
final AirbyteFileOffsetBackingStore offsetManager = new AirbyteFileOffsetBackingStore(cdcOffsetFilePath); | ||
offsetManager.persist(cdcState); | ||
return offsetManager; | ||
} | ||
|
||
} |
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
for mssql:
implementation 'io.debezium:debezium-connector-sqlserver:1.4.2.Final'
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
(if doing 1.5 upgrade within this PR then of course these will all need updating accordingly)