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
Improve UX of the CDC API #21536
Improve UX of the CDC API #21536
Changes from 12 commits
9c038df
13c3ae8
30ee22c
46e804a
40c56bc
ca92caa
310ed1b
267fa1c
23e1d85
f828229
4848a2e
94f559c
c6a5248
9b12103
6873e0b
534d32d
6c442d9
c240949
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 |
---|---|---|
|
@@ -28,11 +28,14 @@ | |
import com.hazelcast.jet.core.ProcessorSupplier; | ||
import com.hazelcast.jet.pipeline.Sources; | ||
import com.hazelcast.jet.pipeline.StreamSource; | ||
import org.apache.kafka.connect.source.SourceConnector; | ||
|
||
import javax.annotation.Nonnull; | ||
import java.util.Map.Entry; | ||
import java.util.Properties; | ||
|
||
import static com.hazelcast.internal.util.Preconditions.checkState; | ||
|
||
/** | ||
* Contains factory methods for creating Change Data Capture (CDC) sources. | ||
* <p> | ||
|
@@ -55,6 +58,7 @@ private DebeziumCdcSources() { | |
* | ||
* @param name the name of this source, must unique, will be passed | ||
* to the underlying Kafka Connect source | ||
* @param connectorClass name of the Debezium connector class | ||
* @return a builder you can use to set the source's properties and | ||
* then construct it | ||
*/ | ||
|
@@ -64,6 +68,24 @@ public static Builder<ChangeRecord> debezium(@Nonnull String name, @Nonnull Stri | |
(properties, eventTimePolicy) -> new ChangeRecordCdcSourceP(properties, eventTimePolicy)); | ||
} | ||
|
||
/** | ||
* Creates a CDC source that streams change data from a | ||
* Debezium-supported database to a Hazelcast Jet pipeline. | ||
* | ||
* @param name the name of this source, must unique, will be passed | ||
* to the underlying Kafka Connect source | ||
* @param connectorClass class of the Debezium Connector which will be used for the CDC | ||
* @return a builder you can use to set the source's properties and | ||
* then construct it | ||
*/ | ||
@Nonnull | ||
public static Builder<ChangeRecord> debezium( | ||
@Nonnull String name, | ||
@Nonnull Class<? extends SourceConnector> connectorClass) { | ||
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. We can provide Class<?> but not |
||
return new Builder<>(name, connectorClass.getName(), | ||
(properties, eventTimePolicy) -> new ChangeRecordCdcSourceP(properties, eventTimePolicy)); | ||
} | ||
|
||
/** | ||
* Creates a CDC source that streams change data from a | ||
* Debezium-supported database to a Hazelcast Jet pipeline. | ||
|
@@ -73,6 +95,9 @@ public static Builder<ChangeRecord> debezium(@Nonnull String name, @Nonnull Stri | |
* raw Debezium data. Just returns a pair of JSON strings, one is | ||
* the key of the Debezium CDC event, the other the value. | ||
* | ||
* @param name the name of this source, must unique, will be passed | ||
* to the underlying Kafka Connect source | ||
* @param connectorClass name of the Debezium connector class | ||
* @return a builder you can use to set the source's properties and then construct it | ||
*/ | ||
@Nonnull | ||
|
@@ -81,6 +106,30 @@ public static Builder<Entry<String, String>> debeziumJson(@Nonnull String name, | |
(properties, eventTimePolicy) -> new JsonCdcSourceP(properties, eventTimePolicy)); | ||
} | ||
|
||
/** | ||
* Creates a CDC source that streams change data from a | ||
* Debezium-supported database to a Hazelcast Jet pipeline. | ||
* <p> | ||
* Differs from the {@link #debezium(String, String) regular source} | ||
* in that it does the least possible amount of processing of the | ||
* raw Debezium data. Just returns a pair of JSON strings, one is | ||
* the key of the Debezium CDC event, the other the value. | ||
* | ||
* @param name the name of this source, must unique, will be passed | ||
* to the underlying Kafka Connect source | ||
* @param connectorClass class of the Debezium Connector which will be used for the CDC | ||
* @return a builder you can use to set the source's properties and then construct it | ||
*/ | ||
@Nonnull | ||
public static Builder<Entry<String, String>> debeziumJson( | ||
@Nonnull String name, | ||
@Nonnull Class<?> connectorClass) { | ||
checkState(SourceConnector.class.isAssignableFrom(connectorClass), "connector class must be a subclass" + | ||
" of SourceConnector"); | ||
return new Builder<>(name, connectorClass.getName(), | ||
(properties, eventTimePolicy) -> new JsonCdcSourceP(properties, eventTimePolicy)); | ||
} | ||
|
||
/** | ||
* A builder to configure a CDC source that streams the change data from | ||
* a Debezium-supported database to Hazelcast Jet. | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -24,7 +24,7 @@ | |
* @since Jet 4.2 | ||
*/ | ||
@EvolvingApi | ||
public class ParsingException extends Exception { | ||
public class ParsingException extends RuntimeException { | ||
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 this change? 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. Most of the operations now are not lazy-parsing the json, so no exception is really expected. However, if user already used methods like 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. I wonder if we should mark this class as deprecated then? 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 is still used in lazy-parsing in RecordPartImpl, although I'm not against changing this to other exception type. |
||
|
||
private static final long serialVersionUID = 1L; | ||
|
||
|
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.
The change record is marked with
@EvolvingApi
so we can remove thethrows ParsingException
if it's not the case anymore.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.
Sure, done