Commit Graph

1707 Commits

Author SHA1 Message Date
Debezium Builder
150396e4c8 [maven-release-plugin] prepare release v2.3.0.Final 2023-06-20 13:21:31 +00:00
mfvitale
91152c4c5d DBZ-6559 Add test to cover topic name strategy 2023-06-16 11:48:34 +02:00
Jiri Pechanec
7f81f4bc6b DBZ-6559 Handle keeping the first char as exception 2023-06-16 11:48:34 +02:00
mfvitale
87217a1474 DBZ-6559 Restore backward compatibility for field that stars with a digit 2023-06-16 11:48:34 +02:00
Jiri Pechanec
067683e56a DBZ-6076 Consistent config for all connectors 2023-06-15 11:55:28 +02:00
Anil Dasari
0431a8b9fc DBZ-6076: Add support for custom SourceInfoStructMaker for adding new fields to source field 2023-06-15 11:55:28 +02:00
Chris Cranford
ef8059f3ff DBZ-6528 Fix Oracle NPE when storing only captured table schemas with signals 2023-06-14 10:48:19 +02:00
mfvitale
b7eb4b5c62 DBZ-6543 Fix NPE due to optional fields in PartitionRouting SMT 2023-06-14 07:22:27 +02:00
Andrei Isac
573a91be07 DBZ-6533: fix table order on partial match when snapshotting 2023-06-12 15:40:03 +02:00
kate
24d3425fdf DBZ-6552 Notify when incremental snapshot fails 2023-06-12 15:14:40 +02:00
mfvitale
3aaca16024 DBZ-6543 Fix NPE due to optional fields in PartitionRouting SMT 2023-06-09 13:47:18 +02:00
Debezium Builder
5d1b00588a [maven-release-plugin] prepare for next development iteration 2023-06-09 09:23:38 +00:00
Debezium Builder
d1e6107517 [maven-release-plugin] prepare release v2.3.0.CR1 2023-06-09 09:23:37 +00:00
Vojtech Juranek
b795519514 DBZ-6529 Use Murmur hash for PartitionRouting SMT
Add option to use better hash function than default Java Object::hash
function to get better hashes which would be more equally spred over
the hash space and thus more equally over the Kafka partitions.
To preserve backward compatibility, previous Java `hashCode` function
is used a default.

Add Murmur3 hash function for computing hashes of the fields.
Murmur3 implementation is taken from from Infinispan project code base.

To allow users eventually use their own make `computePartition()`
protected so it can be overriden in the subclasses.
2023-06-07 11:48:30 +02:00
mfvitale
61e75cc00d DBZ-6424 Move JMX register/unregister code from Metrics to JmxUtils 2023-06-07 07:36:58 +02:00
mfvitale
b0b968fcce DBZ-6424 Move JMX register/unregister code from Metrics to JmxUtils 2023-06-07 07:36:58 +02:00
mfvitale
394a9e32d8 DBZ-6424 Move JMX register/unregister code from Metrics to JmxUtils 2023-06-07 07:36:58 +02:00
mfvitale
7338fde84e DBZ-6424 Emit JMX Notification when Debezium generate its own notification
DBZ-6424 Emit JMX Notification when Debezium generate its own notification

DBZ-6424 Emit JMX Notification when Debezium generate its own notification
2023-06-07 07:36:58 +02:00
mfvitale
769d9361e7 DBZ-6424 Add JMX Notification channel
This channel permits to send Debezium notification to JMX bean

DBZ-6424 Add JMX Signal channel

This channel permits to send signal to Debezium through the JMX operation
2023-06-07 07:36:58 +02:00
mfvitale
f93affc3ca DBZ-6511 Add java docs for SignalProcessor getSignalChannel method 2023-06-05 17:33:07 +02:00
mfvitale
ba8a3e5062 DBZ-6511 Expose reset method on SignalChannelReader to reset processed signals 2023-06-05 17:33:07 +02:00
mfvitale
4d56bbd15a DBZ-6511 Restore Kafka signal offset only when KafkaSignalChannel is enabled 2023-06-05 17:33:07 +02:00
Ronak Jain
3f3967c2bb DBZ-6366: Code Improvements for skip.messages.without.change
Addresses/Closes DBZ-6366
2023-06-05 14:50:25 +02:00
Jiri Pechanec
8a1cd95086 DBZ-6515 Add units to the description 2023-06-05 11:27:57 +02:00
mfvitale
155207dc9b DBZ-6509 Add FileSignalChannel to META-INF/services 2023-05-31 14:39:35 +02:00
mfvitale
e1c1e65336 DBZ-6408 Add documentation for notifications 2023-05-30 17:17:52 +02:00
Debezium Builder
aa5b2131fc [maven-release-plugin] prepare for next development iteration 2023-05-26 10:35:20 +00:00
Debezium Builder
9cc9b3c29a [maven-release-plugin] prepare release v2.3.0.Beta1 2023-05-26 10:35:20 +00:00
mfvitale
ed2157bfc9 DBZ-6486 ExtractNewRecordState skip fields that not exists on payload 2023-05-26 11:30:37 +02:00
Jiri Pechanec
b72f084eac DBZ-6486 Check schema fields 2023-05-26 11:30:37 +02:00
mfvitale
08122371eb DBZ-6486 ExtractNewRecordState can now take arbitrary fields from payload 2023-05-26 11:30:37 +02:00
mfvitale
7ba9f57035 DBZ-6496 Fix signal.poll.interval.ms default value 2023-05-24 18:51:49 +02:00
mfvitale
dc00f64bbf DBZ-6447 Incremental snapshot notification ids are now equal to the id sent in the execute-snapshot signal. 2023-05-23 09:57:05 +02:00
harveyyue
016daf6fd8 DBZ-6490 Convert the field BigDecimal value to bytes failure of ApproximateStructSizeCalculator 2023-05-23 08:38:06 +02:00
Chris Cranford
d40da4f810 DBZ-6457 Use PDB for parallel snapshot connections 2023-05-19 14:04:14 +02:00
Ismail Simsek
2683b31394 DBZ-4027 implement FileSignalChannel 2023-05-15 15:34:43 +02:00
Vojtech Juranek
ee4ebaa957 DBZ-6439 Don't scan all the tables when loading schema
Currently we scan all the tables, which may result into a substantial
delay in initial snapshot when the database is very large. We need to
filter out tables which we are not interested in.

Add back table filter when loading schema of tables. As per comment of
this block of code, passing all tables and table filter should be faster
than passing only list of tables we are interested in.
2023-05-15 14:11:47 +02:00
Vojtech Juranek
f4596fa2e0 DBZ-6460 Enable setting log level in tests via log interceptor 2023-05-15 14:11:47 +02:00
mfvitale
85278e76af DBZ-6462 Stop SignalProcessor after test execution in SignalProcessorTest 2023-05-15 08:51:02 +02:00
mfvitale
e41bbe7191 DBZ-6462 Schedule SignalProcessor task with a more relaxed scheduling. Removes parallelStreams while processing signalChannelReaders 2023-05-15 08:51:02 +02:00
Debezium Builder
a5b44767bc [maven-release-plugin] prepare for next development iteration 2023-05-11 11:05:24 +00:00
Debezium Builder
1b1a7531f2 [maven-release-plugin] prepare release v2.3.0.Alpha1 2023-05-11 11:05:24 +00:00
mfvitale
7429b4e6f1 DBZ-1973 Add incremental snapshot notifications 2023-05-10 12:22:02 +02:00
mfvitale
811b800f55 DBZ-1973 Add support for Notifications
DBZ-1973 Add more tests for other connectors

DBZ-1973 Add send method with offset parameter

DBZ-1973 Instantiate NotificationService in the task class

DBZ-1973 Instantiate NotificationService in the task class

DBZ-4027 Move specific sink channel configuration to SinkNotificationChannel

DBZ-4027 Remove not used SPI file

DBZ-1973 Moved SPI file definition to debezium-core

DBZ-1973 Rename KafkaNotificationChannel to more generic SinkNotificationChannel

DBZ-1973 Code refactor

DBZ-1973 Improve configuration property description

DBZ-1973 Improve test

DBZ-1973 Add close method to NotificationChannel

DBZ-1973 Implement KafkaNotificationChannel

DBZ-1973 Add NotificationService and LogNotificationChannel
2023-05-10 12:22:02 +02:00
mfvitale
0ca87d8031 DBZ-4027 During event dispatch process only signal from source channel 2023-05-05 13:35:40 +02:00
mfvitale
952340286e DBZ-4027 Make signalling channel configurable
DBZ-4027 Add an Incremental snapshot test with kafka signaling

DBZ-4027 Add an Incremental snapshot test with kafka signaling

DBZ-4027 Add an Incremental snapshot test with kafka signaling

DBZ-4027 Code style

DBZ-4027 Make SignalPayload more generic and extensible

DBZ-4027 Rename DatabaseSignalChannel to SourceSignalChannel

DBZ-4027 Improve logging

DBZ-4027 Moved SPI file definition to debezium-core

DBZ-4027 Move SignalProcessor synchronization point to be processed only when a signal cdc event arrives.

DBZ-4027 Add EventDispatcher constructor without signalProcessor for spanner connector

DBZ-4027 Fix NPE

DBZ-4027 Fix NPE

DBZ-4027 Formatting

DBZ-4027 Correctly manage signal on not supported connector

DBZ-4027 Use the correct MongoDbOffset

DBZ-4027 Correctly initialize offset for Oracle and SqlServer connectors

DBZ-4027 Register SPI implementations

DBZ-4027 Improve SignalProcessor instantiation

DBZ-4027 Pass source info in case of SchemaChanges action

DBZ-4027 Manage close event in a synchronous way

DBZ-4027 Correctly init offset context also in case of snapshot mode 'never'

DBZ-4027 Fix MySqlMetricsIT test

DBZ-4027 Move KafkaSignalChannel to core

DBZ-4027 Move KafkaSignalChannel to core

DBZ-4027 Set pass offset context after initial snapshot to SignalProcessor

DBZ-4027 Pass OffsetContext to signal processor

DBZ-4027 Pass CommonConnectorConfig to SignalChannelReader init method

DBZ-4027 Move Incremental snapshot window actions to dedicated package

DBZ-4027 Align SignalsIT test with new code

DBZ-4027 Fix SignalsIT test

DBZ-4027 Fix SignalProcessor scheduling

DBZ-4027 Moved DatabaseSignalChannel and SignalChannelReader to dedicated package

DBZ-4027 Start SignalProcessor from ChangeEventSourceCoordinator

DBZ-4027 Create SignalProcessor and renamed Signal to DatabaseSignalChannel

DBZ-4027 Initial refactoring of signal feature
2023-05-05 13:35:40 +02:00
Jiri Pechanec
58ef4f0b98 DBZ-6157 Disable potentially dangerous MySQL JDBC props 2023-05-04 11:54:04 +02:00
Jiri Pechanec
6b1455666a DBZ-6411 Message content logged only at TRACE 2023-05-03 06:51:28 +02:00
mfvitale
e5d4332835 DBZ-6411 Fix HeaderToValue schema update cache.
When the connector starts for the first time with initial snapshot record with read operation will be created. In this phase the ExtractChangedRecordState will produce no headers and since HeaderToValue was not skipping record without headers the cache of the new schema will be created only with the old fields.
2023-05-03 06:51:28 +02:00
harveyyue
3860a4faca DBZ-6388 __source_ts_ms r (read) operation date is set to future for SQL Server 2023-05-02 08:11:02 -04:00