-
Notifications
You must be signed in to change notification settings - Fork 3.7k
[fix][io] Upgrade Debezium connector version to 2.6.1.Final #23078
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
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -26,12 +26,12 @@ | |
| import io.debezium.config.Configuration; | ||
| import io.debezium.config.Field; | ||
| import io.debezium.document.DocumentReader; | ||
| import io.debezium.relational.history.AbstractDatabaseHistory; | ||
| import io.debezium.relational.history.DatabaseHistory; | ||
| import io.debezium.relational.history.DatabaseHistoryException; | ||
| import io.debezium.relational.history.DatabaseHistoryListener; | ||
| import io.debezium.relational.history.AbstractSchemaHistory; | ||
| import io.debezium.relational.history.HistoryRecord; | ||
| import io.debezium.relational.history.HistoryRecordComparator; | ||
| import io.debezium.relational.history.SchemaHistory; | ||
| import io.debezium.relational.history.SchemaHistoryException; | ||
| import io.debezium.relational.history.SchemaHistoryListener; | ||
| import java.io.IOException; | ||
| import java.util.Collections; | ||
| import java.util.HashMap; | ||
|
|
@@ -52,12 +52,12 @@ | |
| import org.apache.pulsar.client.api.Schema; | ||
|
|
||
| /** | ||
| * A {@link DatabaseHistory} implementation that records schema changes as normal pulsar messages on the specified | ||
| * A {@link SchemaHistory} implementation that records schema changes as normal pulsar messages on the specified | ||
| * topic, and that recovers the history by establishing a Kafka Consumer re-processing all messages on that topic. | ||
| */ | ||
| @Slf4j | ||
| @ThreadSafe | ||
| public final class PulsarDatabaseHistory extends AbstractDatabaseHistory { | ||
| public final class PulsarDatabaseHistory extends AbstractSchemaHistory { | ||
|
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. Why is this change needed? I don't know Debezium very well, but it seems that "database history" and "schema history" are separate concepts. 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. 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. It might be more consistent naming if 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. They renamed DatabaseHistory to SchemaHistory in 2.0 version of debezium. See here |
||
|
|
||
| public static final Field TOPIC = Field.create(CONFIGURATION_FIELD_PREFIX_STRING + "pulsar.topic") | ||
| .withDisplayName("Database history topic name") | ||
|
|
@@ -94,11 +94,11 @@ public final class PulsarDatabaseHistory extends AbstractDatabaseHistory { | |
| .withValidation(Field::isOptional); | ||
|
|
||
| public static final Field.Set ALL_FIELDS = Field.setOf( | ||
| TOPIC, | ||
| SERVICE_URL, | ||
| CLIENT_BUILDER, | ||
| DatabaseHistory.NAME, | ||
| READER_CONFIG); | ||
| TOPIC, | ||
| SERVICE_URL, | ||
| CLIENT_BUILDER, | ||
| SchemaHistory.NAME, | ||
| READER_CONFIG); | ||
|
|
||
| private final ObjectMapper mapper = new ObjectMapper(); | ||
| private final DocumentReader reader = DocumentReader.defaultReader(); | ||
|
|
@@ -113,7 +113,7 @@ public final class PulsarDatabaseHistory extends AbstractDatabaseHistory { | |
| public void configure( | ||
| Configuration config, | ||
| HistoryRecordComparator comparator, | ||
| DatabaseHistoryListener listener, | ||
| SchemaHistoryListener listener, | ||
| boolean useCatalogBeforeSchema) { | ||
| super.configure(config, comparator, listener, useCatalogBeforeSchema); | ||
| if (!config.validateAndRecord(ALL_FIELDS, logger::error)) { | ||
|
|
@@ -148,7 +148,7 @@ public void configure( | |
| } | ||
|
|
||
| // Copy the relevant portions of the configuration and add useful defaults ... | ||
| this.dbHistoryName = config.getString(DatabaseHistory.NAME, UUID.randomUUID().toString()); | ||
| this.dbHistoryName = config.getString(SchemaHistory.NAME, UUID.randomUUID().toString()); | ||
|
|
||
| log.info("Configure to store the debezium database history {} to pulsar topic {}", | ||
| dbHistoryName, topicName); | ||
|
|
@@ -201,7 +201,7 @@ public void start() { | |
| } | ||
|
|
||
| @Override | ||
| protected void storeRecord(HistoryRecord record) throws DatabaseHistoryException { | ||
| protected void storeRecord(HistoryRecord record) throws SchemaHistoryException { | ||
| if (this.producer == null) { | ||
| throw new IllegalStateException("No producer is available. Ensure that 'start()'" | ||
| + " is called before storing database history records."); | ||
|
|
@@ -212,7 +212,7 @@ protected void storeRecord(HistoryRecord record) throws DatabaseHistoryException | |
| try { | ||
| producer.send(record.toString()); | ||
| } catch (PulsarClientException e) { | ||
| throw new DatabaseHistoryException(e); | ||
| throw new SchemaHistoryException(e); | ||
| } | ||
| } | ||
|
|
||
|
|
||
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.
According to https://debezium.io/ , latest stable Debezium release series is 2.7.x . Could we upgrade directly to 2.7.x instead?
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.
From 2.6.2.Final onwards the postgres tester is not working. The consumer is expecting a message event just after the insertion into the table but the source is taking some time to process the event. I have tested this locally as well.
I think it is due to PostgreSQL offset flush race condition. More details are here:
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.
DBZ-7816 is fixed in 2.6.2.Final. Perhaps something needs to be adjusted in the Pulsar Debezium integration?