support mysql binlog sync to pulsar by canal#2998
Conversation
modify canal.client 1.1.1 for support flatMessage
|
|
||
| protected void start() { | ||
| Assert.notNull(connector, "connector is null"); | ||
| thread = new Thread(new Runnable() { |
There was a problem hiding this comment.
Maybe assign a name to thread so it's easier to debug with jstack
| @@ -0,0 +1,152 @@ | |||
| package org.apache.pulsar.io.canal; | |||
There was a problem hiding this comment.
This needs to have Apache headers. You can use mvn license:format to automatically add
sijie
left a comment
There was a problem hiding this comment.
overall looks good. it is a great contributions. have some review comments and also need some clarifications from you.
|
|
||
| @Override | ||
| public void uncaughtException(Thread t, Throwable e) { | ||
| log.error("parse events has an error", e); |
There was a problem hiding this comment.
nit: it is good to add thread name in the logging message for troubleshooting purpose.
|
|
||
| private CanalSourceConfig canalSourceConfig; | ||
|
|
||
| protected Thread.UncaughtExceptionHandler handler = new Thread.UncaughtExceptionHandler() { |
| new InetSocketAddress(canalSourceConfig.getSingleHostname(), canalSourceConfig.getSinglePort()), | ||
| canalSourceConfig.getDestination(), canalSourceConfig.getUsername(), canalSourceConfig.getPassword()); | ||
| } | ||
| log.info("start canal connect"); |
There was a problem hiding this comment.
can you improve the logging here?
-
in cluster mode, can you print out the zookeeper user and destination?
-
in single mode, can you print out the single host name, port and destination?
| } | ||
|
|
||
| protected void start() { | ||
| Assert.notNull(connector, "connector is null"); |
There was a problem hiding this comment.
nit: Use java builtin objects.requirenonnull or guava checkNotNull?
| connector.disconnect(); | ||
| } | ||
|
|
||
| MDC.remove("destination"); |
There was a problem hiding this comment.
can you define destination as a constant string?
| static private class CanalRecord implements Record<byte[]> { | ||
|
|
||
| private final byte[] record; | ||
| private final Long id; |
There was a problem hiding this comment.
is this id incremental? if you might consider using it as record sequence.
| } | ||
| } | ||
|
|
||
| connector.ack(batchId); |
There was a problem hiding this comment.
if you are acking here, it only supports at-most-once semantic. you might consider implementing ack() in CanalRecord
pulsar-io/pom.xml
Outdated
| <module>elastic-search</module> | ||
| <module>kafka-connect-adaptor</module> | ||
| <module>debezium</module> | ||
| <module>canal</module> |
add thread name add log info set to constant string destination Class CanalRecord add getRecordSequence
add Batch consume
Delete import Assert
|
Please take the time to review my code again. It has been changed and tested locally. |
|
run integration tests |
…om/AmateurEvents/pulsar into feature/support-mysql-binlog-canal
|
run integration tests |
|
run integration tests org.apache.pulsar.tests.integration.offload.TestS3Offload.tiered-storage-test-suite |
1 similar comment
|
run integration tests org.apache.pulsar.tests.integration.offload.TestS3Offload.tiered-storage-test-suite |
jiazhai
left a comment
There was a problem hiding this comment.
lgtm. Thanks for the contribute.
|
run integration tests |
|
run integration tests |
2 similar comments
|
run integration tests |
|
run integration tests |
|
@tuteng thank you so much for your contribution. |
Motivation
support alibaba canal
https://github.com/alibaba/canal/wiki
Modifications
Integrated canal client
Result
support binlog sync to pulsar
use python pulsar-client to consume