博客 Flink CDC2.4 整库实时同步MySql 到Doris

Flink CDC2.4 整库实时同步MySql 到Doris

   数栈君   发表于 2023-10-13 15:24  1911  0

环境
Flink 1.15.4

实现原因
目前有很多工具都支持无代码实现Mysql -> Doris 的实时同步

如:SlectDB 已发布的功能包

Dinky SeaTunnel TIS 等等

不过好多要么不支持表结构变动,要不不支持多sink,我们的业务必须支持对表结构的实时级变动,因为会对表字段级别的修改,字段类型更改,字段名字更改删除添加等

所以要支持整库同步且又要表结构的实时变动就要自己写



所需jar
flink-doris-connector-1.15-1.4.0.jar -- 实现一键万表同步

flink-sql-connector-mysql-cdc-2.4.0.jar --包含所有相关依赖,无需在导入debezium、cdc等等

流程
1、脚本创建库表

2、同步表结构程序

3、Flink cdc 程序

对比第一版本:使用 Flink CDC 实现 MySQL 数据,表结构实时入 Apache Doris 效率有所提升

首次同步时keyby 后开窗聚合导致数据倾斜

聚合数据有字符串拼接改为JsonArray 避免聚合导致背压,字符串在数据量较大时拼接效率太低

Flink cdc 代码
1、FlinkSingleSync.scala


package com.zbkj.sync

import com.alibaba.fastjson2.{JSON, JSONObject,JSONArray}
import com.ververica.cdc.connectors.mysql.source.MySqlSource
import com.ververica.cdc.connectors.mysql.table.StartupOptions
import com.ververica.cdc.connectors.shaded.org.apache.kafka.connect.json.JsonConverterConfig
import com.ververica.cdc.debezium.JsonDebeziumDeserializationSchema
import com.zbkj.util.SinkBuilder.getKafkaSink
import com.zbkj.util._
import org.apache.flink.api.common.eventtime.WatermarkStrategy
import org.apache.flink.api.common.restartstrategy.RestartStrategies
import org.apache.flink.api.java.utils.ParameterTool
import org.apache.flink.streaming.api.windowing.assigners.TumblingProcessingTimeWindows
import org.apache.flink.streaming.api.{CheckpointingMode, TimeCharacteristic}
import org.apache.flink.streaming.api.environment.CheckpointConfig.ExternalizedCheckpointCleanup
import org.apache.flink.streaming.api.windowing.time.Time
import org.apache.flink.streaming.api.scala._

import java.util.Properties

object FlinkSingleSync {

PropertiesManager.initUtil()
val props: PropertiesUtil = PropertiesManager.getUtil

def main(args: Array[String]): Unit = {
val env = StreamExecutionEnvironment.getExecutionEnvironment
// 并行度
env.setParallelism(props.parallelism)
env.setStreamTimeCharacteristic(TimeCharacteristic.ProcessingTime)

val parameters: ParameterTool = ParameterTool.fromArgs(args)
val memberID = parameters.getInt("memberID", 0)
val source = parameters.get("source", "")
val log = parameters.getBoolean("log", true)
if (memberID == 0) {
sys.exit(0)
}
val thisMember = "ttk_member_%d".format(memberID)
val jobName = "Sync Member %d".format(memberID)
val syncTopic = "sync_data_%d".format(memberID)
println(syncTopic)
val sourceFormat = SourceFormat.sourceFormat(source)

env.setParallelism(4)
/**
* checkpoint的相关设置 */
// 启用检查点,指定触发checkpoint的时间间隔(单位:毫秒,默认500毫秒),默认情况是不开启的
env.enableCheckpointing(1000L, CheckpointingMode.EXACTLY_ONCE)
// 设定Checkpoint超时时间,默认为10分钟
env.getCheckpointConfig.setCheckpointTimeout(600000)

/**
* 设置检查点路径 */
env.getCheckpointConfig.setCheckpointStorage("file:///data/flink-checkpoints/sync/%d".format(memberID))

/** 设定两个Checkpoint之间的最小时间间隔,防止出现例如状态数据过大而导致Checkpoint执行时间过长,从而导致Checkpoint积压过多
* 最终Flink应用密切触发Checkpoint操作,会占用了大量计算资源而影响到整个应用的性能(单位:毫秒) */
env.getCheckpointConfig.setMinPauseBetweenCheckpoints(60000)
// 默认情况下,只有一个检查点可以运行
// 根据用户指定的数量可以同时触发多个Checkpoint,进而提升Checkpoint整体的效率
//env.getCheckpointConfig.setMaxConcurrentCheckpoints(2)
/** 外部检查点
* 不会在任务正常停止的过程中清理掉检查点数据,而是会一直保存在外部系统介质中,另外也可以通过从外部检查点中对任务进行恢复 */
env.getCheckpointConfig.enableExternalizedCheckpoints(ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION)


// env.getCheckpointConfig.setPreferCheckpointForRecovery(true)
// 设置可以允许的checkpoint失败数
env.getCheckpointConfig.setTolerableCheckpointFailureNumber(3)
//设置可容忍的检查点失败数,默认值为0表示不允许容忍任何检查点失败
env.getCheckpointConfig.setTolerableCheckpointFailureNumber(2)
env.disableOperatorChaining()

/**
* 重启策略的配置
* 重启3次,每次失败后等待10000毫秒
*/
env.setRestartStrategy(RestartStrategies.fixedDelayRestart(3, 30000L))

val dataBaseList = thisMember
var tableList = thisMember + ".*"
if (!log) {
tableList = "lb_crm_customer_log|.*(?<!_log)$"
}

val dorisStreamLoad = new DorisStreamLoad2(props)

// numeric 类型转换
val customConverterConfigs = new java.util.HashMap[String, Object] {
put(JsonConverterConfig.DECIMAL_FORMAT_CONFIG, "numeric")
}
/**
* mysql source for doris */
println(dataBaseList, tableList)
val debeziumProps = new Properties()
debeziumProps.setProperty("debezium.snapshot.mode","never")
val mysqlSource = MySqlSource.builder[String]()
.hostname(sourceFormat.getString("sourceHost"))
.port(sourceFormat.getIntValue("sourcePort"))
.databaseList(dataBaseList)
//^((?!lb_admin_log|lb_bugs).)*$
// lb_admin_log、lb_bugs为不需要同步表
.tableList(props.regular_expression)
.username(sourceFormat.getString("sourceUsername"))
.password(sourceFormat.getString("sourcePassword"))
.debeziumProperties(debeziumProps)
// 全量读取
.startupOptions(StartupOptions.initial())
.includeSchemaChanges(true)
// 发现新表,加入同步任务,需要在tableList中配置
.scanNewlyAddedTableEnabled(true)
.deserializer(new JsonDebeziumDeserializationSchema(false, customConverterConfigs)).build()

val streamSource: DataStream[JSONObject] = env.fromSource(mysqlSource, WatermarkStrategy.noWatermarks(), "MySQL Source")
.map(line => JSON.parseObject(line)).setParallelism(4)

val DDLSqlStream: DataStream[JSONObject] = streamSource.filter(line => !line.containsKey("op")).uid("ddlSqlStream")
val DMLStream: DataStream[JSONObject] = streamSource.filter(line => line.containsKey("op")).uid("dmlStream")
/**
* 首次全量同步时 时间窗口内几乎为一个表数据,此时下面操作会数据倾斜
* 在binLogETLOne 中对表加随机数后缀 使其均匀分布
* 聚合操作之后再将tableName转换为实际表
*/
val DMLDataStream = FlinkCDCSyncETL.binLogETLOne(DMLStream)
val keyByDMLDataStream:DataStream[(String, String, String, JSONArray)] = DMLDataStream.keyBy(keys => (keys._1, keys._2, keys._3))
.timeWindow(Time.milliseconds(props.window_time_milliseconds))
.reduce((itemFirst, itemSecond) => (itemFirst._1, itemFirst._2, itemFirst._3,combineJsonArray(itemFirst._4,itemSecond._4)))
.map(line=>(line._1,line._2,line._3.split("-")(0),line._4))
.name("分组聚合").uid("keyBy")


keyByDMLDataStream.addSink(new SinkDoris(dorisStreamLoad)).name("数据写入Doris").uid("SinkDoris").setParallelism(4)

val DDLKafkaSink=getKafkaSink("schema_change")
DDLSqlStream.map(jsObj => jsObj.toJSONString()).sinkTo(DDLKafkaSink).name("同步DDL入Kafka").uid("SinkDDLKafka")

val kafkaSink=getKafkaSink(syncTopic)
keyByDMLDataStream.map(line=>(line._2,line._3,1)).filter(!_._2.endsWith("_sql"))
.keyBy(keys => (keys._1, keys._2))
.window(TumblingProcessingTimeWindows.of(Time.seconds(1))).sum(2)
.map(line =>{
val json = new JSONObject()
json.put("member_id", line._1)
json.put("table", line._2)
json.toJSONString()
}).sinkTo(kafkaSink).name("同步数据库表入Kafka").uid("syncDataTableToKafka")

env.execute(jobName)

}

def combineJsonArray(jsr1:JSONArray,jsr2:JSONArray): JSONArray ={
jsr1.addAll(jsr2)
jsr1
}

}

2.FlinkCDCSyncETL.scala
package com.zbkj.util

import com.alibaba.fastjson2.{JSON, JSONArray, JSONObject}
import org.apache.flink.api.scala.createTypeInformation
import org.apache.flink.streaming.api.scala.DataStream

import java.util.Random

object FlinkCDCSyncETL {

def binLogETLOne(dataStreamSource: DataStream[JSONObject]): DataStream[(String, String, String, JSONArray)] = {
/**
* 根据不同日志类型 匹配load doris方式
*/
val tupleData: DataStream[(String, String, String, JSONArray)] = dataStreamSource.map(line => {
var data: JSONObject = new JSONObject()
var jsr: JSONArray = new JSONArray()
var mergeType = "APPEND"
val source = line.getJSONObject("source")
val db = source.getString("db")
val table = source.getString("table")
val op=line.getString("op")
if ("d" == op) {
data = line.getJSONObject("before")
mergeType = "DELETE"
} else if ("u" == op) {
data = line.getJSONObject("after")
mergeType = "APPEND"
} else if ("c" == op) {
data = line.getJSONObject("after")
} else if ("r" == op) {
data = line.getJSONObject("after")
mergeType = "APPEND"
}
jsr.add(data)
Tuple4(mergeType, db, table+ "-" + new Random().nextInt(4), jsr)
})
tupleData
}



}

3.DorisStreamLoad2.scala
package com.zbkj.util

import org.apache.doris.flink.exception.StreamLoadException
import org.apache.doris.flink.sink.HttpPutBuilder
import org.apache.http.client.methods.CloseableHttpResponse
import org.apache.http.entity.StringEntity
import org.apache.http.impl.client.{DefaultRedirectStrategy, HttpClientBuilder, HttpClients}
import org.apache.http.util.EntityUtils
import org.slf4j.{Logger, LoggerFactory}

import java.util.{Properties, UUID}

class DorisStreamLoad2(props: PropertiesUtil) extends Serializable {
private val logger: Logger = LoggerFactory.getLogger(classOf[DorisStreamLoad2])

private lazy val httpClientBuilder: HttpClientBuilder = HttpClients.custom.setRedirectStrategy(new DefaultRedirectStrategy() {
override protected def isRedirectable(method: String): Boolean = {
// If the connection target is FE, you need to deal with 307 redirect。
true
}
})


def loadJson(jsonData: String, mergeType: String, db: String, table: String): Unit = try {
val loadUrlPattern = "http://%s/api/%s/%s/_stream_load?"
val entity = new StringEntity(jsonData, "UTF-8")
val streamLoadProp = new Properties()
streamLoadProp.setProperty("merge_type", mergeType)
streamLoadProp.setProperty("format", "json")
streamLoadProp.setProperty("column_separator", ",")
streamLoadProp.setProperty("line_delimiter", ",")
streamLoadProp.setProperty("strip_outer_array", "true")
streamLoadProp.setProperty("exec_mem_limit", "6442450944")
streamLoadProp.setProperty("strict_mode", "true")
val httpClient = httpClientBuilder.build
val loadUrlStr = String.format(loadUrlPattern, props.doris_load_host, db, table)
try {
val builder = new HttpPutBuilder()
val label = UUID.randomUUID.toString
builder.setUrl(loadUrlStr)
.baseAuth(props.doris_user, props.doris_password)
.addCommonHeader()
.setLabel(label)
.setEntity(entity)
.addProperties(streamLoadProp)
handlePreCommitResponse(httpClient.execute(builder.build()))
}

def handlePreCommitResponse(response: CloseableHttpResponse): Unit = {
val statusCode: Int = response.getStatusLine.getStatusCode
if (statusCode == 200 && response.getEntity != null) {
val loadResult: String = EntityUtils.toString(response.getEntity)
logger.info("load Result {}", loadResult)
} else {
throw new StreamLoadException("stream load error: " + response.getStatusLine.toString)
}


}


}
}

4.SinkDoris.scala
package com.zbkj.util

import com.alibaba.fastjson2.JSONArray
import org.apache.flink.configuration.Configuration
import org.apache.flink.streaming.api.functions.sink.RichSinkFunction

class SinkDoris(dorisStreamLoad:DorisStreamLoad2) extends RichSinkFunction[(String, String, String, JSONArray)] {


override def open(parameters: Configuration): Unit = {}
/**
* 每个元素的插入都要调用一次invoke()方法进行插入操作
*/
override def invoke(value:(String, String, String, JSONArray)): Unit = {
dorisStreamLoad.loadJson(value._4.toString,value._1,value._2,value._3)
}

override def close(): Unit = {}
}


免责申明:


本文系转载,版权归原作者所有,如若侵权请联系我们进行删除!

《数据治理行业实践白皮书》下载地址:https://fs80.cn/4w2atu

《数栈V6.0产品白皮书》下载地址:
https://fs80.cn/cw0iw1

想了解或咨询更多有关袋鼠云大数据产品、行业解决方案、客户案例的朋友,浏览袋鼠云官网:
https://www.dtstack.com/?src=bbs

同时,欢迎对大数据开源项目有兴趣的同学加入「袋鼠云开源框架钉钉技术群」,交流最新开源技术信息,群号码:30537511,项目地址:
https://github.com/DTStack

0条评论
社区公告
  • 大数据领域最专业的产品&技术交流社区,专注于探讨与分享大数据领域有趣又火热的信息,专业又专注的数据人园地

最新活动更多
微信扫码获取数字化转型资料
钉钉扫码加入技术交流群